Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support TIMESTAMP_NTZ, DATE and TIME datatype #14398

Open
wants to merge 7 commits into
base: master
Choose a base branch
from

Conversation

chunxiaozheng
Copy link

Instructions:

  1. The PR has to be tagged with at least one of the following labels (i):
    1. feature
    2. bugfix
    3. performance
    4. ui
    5. backward-incompat
    6. release-notes (**)

This PR add TIMESTAMP_NTZ, DATE and TIME datatype.
For flink, TIMESTAMP does not have time zone, which corresponds to TIMESTAMP_NTZ in pinot, TIMESTAMP_LTZ has local time zone, which corresponds to TIMESTAMP in pinot.
For spark, TIMESTAMP has local time zone, which corresponds to TIMESTAMP in pinot, TIMESTAMP_NTZ does not have time zone, which corresponds to TIMESTAMP_NTZ in pinot.

In my case, the schema of my table is as follows
image

and the query result is as follows
image

the schema json of TIME, DATE, TIMESTAMP_NTZ and TIMESTAMP is as follows

"dimensionFieldSpecs": [
    {   
      "name": "time_field",
      "dataType": "TIME"
    },  
    {   
      "name": "date_field",
      "dataType": "DATE"
    },  
    {   
      "name": "timestamp_ntz_field",
      "dataType": "TIMESTAMP_NTZ"
    },  
    {                                                                                                                      
      "name": "timestamp_ltz_field",
      "dataType": "TIMESTAMP"
    }   
  ]

@codecov-commenter
Copy link

codecov-commenter commented Nov 6, 2024

Codecov Report

Attention: Patch coverage is 20.42079% with 643 lines in your changes missing coverage. Please review.

Project coverage is 63.51%. Comparing base (59551e4) to head (36de3e2).
Report is 1343 commits behind head on master.

Files with missing lines Patch % Lines
...a/org/apache/pinot/common/utils/PinotDataType.java 23.31% 122 Missing and 3 partials ⚠️
...ator/transform/function/CastTransformFunction.java 3.80% 99 Missing and 2 partials ⚠️
...java/org/apache/pinot/common/utils/DataSchema.java 18.10% 92 Missing and 3 partials ⚠️
...ava/org/apache/pinot/spi/utils/ArrayCopyUtils.java 0.00% 48 Missing ⚠️
...sform/function/ScalarTransformFunctionWrapper.java 0.00% 40 Missing ⚠️
.../function/ParentExprMinMaxAggregationFunction.java 0.00% 27 Missing ⚠️
...mon/request/context/predicate/BaseInPredicate.java 0.00% 24 Missing ⚠️
...core/operator/filter/predicate/PredicateUtils.java 0.00% 24 Missing ⚠️
...ot/calcite/rel/rules/PinotEvaluateLiteralRule.java 8.33% 15 Missing and 7 partials ⚠️
...aggregation/function/AggregationFunctionUtils.java 0.00% 21 Missing ⚠️
... and 16 more
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #14398      +/-   ##
============================================
+ Coverage     61.75%   63.51%   +1.76%     
- Complexity      207     1568    +1361     
============================================
  Files          2436     2662     +226     
  Lines        133233   146806   +13573     
  Branches      20636    22451    +1815     
============================================
+ Hits          82274    93241   +10967     
- Misses        44911    46650    +1739     
- Partials       6048     6915     +867     
Flag Coverage Δ
custom-integration1 ?
integration 100.00% <ø> (+99.99%) ⬆️
integration1 100.00% <ø> (+99.99%) ⬆️
integration2 0.00% <ø> (ø)
java-11 63.49% <20.42%> (+1.78%) ⬆️
java-21 63.40% <20.42%> (+1.77%) ⬆️
skip-bytebuffers-false 63.50% <20.42%> (+1.76%) ⬆️
skip-bytebuffers-true 63.37% <20.42%> (+35.65%) ⬆️
temurin 63.51% <20.42%> (+1.76%) ⬆️
unittests 63.50% <20.42%> (+1.76%) ⬆️
unittests1 55.22% <19.30%> (+8.33%) ⬆️
unittests2 33.98% <5.44%> (+6.25%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@gortiz
Copy link
Contributor

gortiz commented Nov 6, 2024

Cool! It looks like the PR is not yet finished (there are tests failing due to this new type) but it looks promising! I'll try to review it soon.

@Jackie-Jiang Jackie-Jiang added release-notes Referenced by PRs that need attention when compiling the next release notes Configuration Config changes (addition/deletion/change in behavior) labels Nov 6, 2024
@Jackie-Jiang
Copy link
Contributor

Related to #13004, #9575

Thanks for working on this!
A high level design covering the following points can help explain the code and guide the review:

  1. What is the storage type and format for each of them
  2. What is the supported input formats for each of them
  3. What is the output format for each of them
  4. What is the expected behavior when comparing different types, e.g. comparing TIMESTAMP with TIMESTAMP_NTZ, DATE with numeric type etc. We should try to match PostgreSQL behavior.

@chunxiaozheng
Copy link
Author

Cool! It looks like the PR is not yet finished (there are tests failing due to this new type) but it looks promising! I'll try to review it soon.

Thanks! I will fix the tests today

@gortiz
Copy link
Contributor

gortiz commented Nov 7, 2024

As suggested by @Jackie-Jiang, I think a design document would be great. There we can discuss the nomenclature, which from my point of view is not clear. I would like to open the discussion to use the same definitions used by Postgres, which would mean to have a TIMESTAMP and TIMESTAMP_WITH_TIME_ZONE (or TIMESTAMP_TZ for short, which is how it is called in Calcite)

Comment on lines 280 to 291
TIMESTAMP(LONG, NullValuePlaceHolder.LONG) {
@Override
public RelDataType toType(RelDataTypeFactory typeFactory) {
return typeFactory.createSqlType(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
}
},
TIMESTAMP_NTZ(LONG, NullValuePlaceHolder.LONG) {
@Override
public RelDataType toType(RelDataTypeFactory typeFactory) {
return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
}
},
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess you did this to follow the Spark terminology, but apart from my preference to following Postgres terminology I think this is breaking backward compatibility. Before this PR a Pinot TIMESTAMP was converted to Calcite TIMESTAMP and now we are converting it into a TIMESTAMP WITH LOCAL TIME ZONE.

I may be wrong but I think what we should be doing here is to return that our TIMESTAMP_TZ returns a Calcite TIMESTAMP_TZ

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess you did this to follow the Spark terminology, but apart from my preference to following Postgres terminology I think this is breaking backward compatibility. Before this PR a Pinot TIMESTAMP was converted to Calcite TIMESTAMP and now we are converting it into a TIMESTAMP WITH LOCAL TIME ZONE.

I may be wrong but I think what we should be doing here is to return that our TIMESTAMP_TZ returns a Calcite TIMESTAMP_TZ

Thanks for your review! From my understanding, the TIMESTAMP type in Calcite has no time zone, but the TIMESTAMP type in pinot is all using java.sql.Timestamp to explain, which is with the local time zone, so i made changes here.
In order to minimize modifications to the TIMESTAMP type in pinot, i referred to the data type in Spark and extended the TIMESTAMP_NTZ type.

@@ -490,12 +540,24 @@ public Object toInternal(Object value) {
return ((boolean) value) ? 1 : 0;
case TIMESTAMP:
return ((Timestamp) value).getTime();
case TIMESTAMP_NTZ:
return ((LocalDateTime) value).atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@bziobrowski IIRC you knew a way to do this without allocating so many objects. Am I right?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That time I used joda.time's MutableDateTime, not java classes.
I agree that this line creates way too many temporary objects and should be rewritten.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In this specific case, this can be rewritten with (AFAIK) zero allocation with:

Suggested change
return ((LocalDateTime) value).atZone(ZoneId.of("UTC")).toInstant().toEpochMilli();
return ((LocalDateTime) value).toEpochSecond(ZoneOffset.UTC) * 1000;

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In this specific case, this can be rewritten with (AFAIK) zero allocation with:

Thanks! But I think this will lose millisecond accuracy😂

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gortiz I think that snippet isn't correct because code assumes offset is constant (and actually 0 while local is likely not :) )
@chunxiaozheng Please consider :

  public static void main(String[] args) {
    LocalDateTime dt = LocalDateTime.of(2024, 11, 8, 12, 13, 14, 231000000);

    ZoneId zoneId = ZoneId.of("Europe/Warsaw");// zone should be cached
    ZoneRules rules = zoneId.getRules();
    ZoneOffset offset = rules.getOffset(dt);

    System.out.println("to epoch (UTC) " + dt.toEpochSecond(ZoneOffset.UTC) * 1000);
    System.out.println("to epoch " + dt.toEpochSecond(offset) * 1000);
    System.out.println("to epoch plus nanos " + dt.toEpochSecond(offset) * 1000 + dt.getNano() / 1000000);
  }

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@gortiz I think that snippet isn't correct because code assumes offset is constant (and actually 0 while local is likely not :) ) @chunxiaozheng Please consider :

  public static void main(String[] args) {
    LocalDateTime dt = LocalDateTime.of(2024, 11, 8, 12, 13, 14, 231000000);

    ZoneId zoneId = ZoneId.of("Europe/Warsaw");// zone should be cached
    ZoneRules rules = zoneId.getRules();
    ZoneOffset offset = rules.getOffset(dt);

    System.out.println("to epoch (UTC) " + dt.toEpochSecond(ZoneOffset.UTC) * 1000);
    System.out.println("to epoch " + dt.toEpochSecond(offset) * 1000);
    System.out.println("to epoch plus nanos " + dt.toEpochSecond(offset) * 1000 + dt.getNano() / 1000000);
  }

Thanks! The implementation of TIMESTAMP type without time zone, such as TIMESTAMP in Flink and TIMESTAMP_NTZ type in Spark, will parse according to UTC time to ensure that the time queried in any time zone is consistent. Therefore, for TIMESTAMP types without time zones, I believe it is need to use a constant(UTC time zone) for parsing.

@chunxiaozheng
Copy link
Author

Related to #13004, #9575

Thanks for working on this! A high level design covering the following points can help explain the code and guide the review:

  1. What is the storage type and format for each of them
  2. What is the supported input formats for each of them
  3. What is the output format for each of them
  4. What is the expected behavior when comparing different types, e.g. comparing TIMESTAMP with TIMESTAMP_NTZ, DATE with numeric type etc. We should try to match PostgreSQL behavior.

Thanks! I will write an document for this PR, and then we can discuss it together.

Comment on lines +46 to +55
if (map.containsKey(SqlTypeName.DATE)) {
Set<SqlTypeName> fromTypes = new HashSet<>(map.get(SqlTypeName.DATE));
fromTypes.addAll(TYPES);
map.put(SqlTypeName.DATE, ImmutableSet.copyOf(fromTypes));
}
if (map.containsKey(SqlTypeName.TIME)) {
Set<SqlTypeName> fromTypes = new HashSet<>(map.get(SqlTypeName.TIME));
fromTypes.addAll(TYPES);
map.put(SqlTypeName.TIME, ImmutableSet.copyOf(fromTypes));
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we actually want to support these castings. Instead we should add the functions that use them. For example, in postgres there is a function that adds date + integer. See https://www.postgresql.org/docs/current/functions-datetime.html

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we actually want to support these castings. Instead we should add the functions that use them. For example, in postgres there is a function that adds date + integer. See https://www.postgresql.org/docs/current/functions-datetime.html

I don't know if we need to support the syntax of cast(12345 as DATE) or cast(12345 as TIME(3)). If we need this syntax, we need to add a similar implementation.
Otherwise, when executing the above statement, the mapping rules in org.apache.calcite.sql.type.SqlTypeCoercionRule will be used, which does not support converting Integer type to Date or Time

Copy link
Contributor

@gortiz gortiz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like at runtime we sometimes use longs to implement timestamps, date and time, but some other times we use LocalDate and LocalTime. That seems a bit odd. I don't know if we are doing the same with Timestamp right now in master, we would need to study if this is what we want

@chunxiaozheng
Copy link
Author

It looks like at runtime we sometimes use longs to implement timestamps, date and time, but some other times we use LocalDate and LocalTime. That seems a bit odd. I don't know if we are doing the same with Timestamp right now in master, we would need to study if this is what we want

Thanks! In fact, I added these new data types based on the implementation of TIMESTAMP data type in Pinot.🤔
From my understanding, these time related type use the longtype in storage, and when queried through DQL, they will be converted to the corresponding Java type, such as java.sql.Timestamp, java.time.LocalDateTime, java.time.LocalDate and java.time.LocalTime. Therefore, this will include many conversions between these Java types and long types

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Configuration Config changes (addition/deletion/change in behavior) feature release-notes Referenced by PRs that need attention when compiling the next release notes
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants