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

[BEAM-7554] Add MillisInstant logical type to replace DATETIME #11456

Closed
wants to merge 22 commits into from

Conversation

TheNeuralBit
Copy link
Member

@TheNeuralBit TheNeuralBit commented Apr 17, 2020

This PR adds a MillisInstant logical type, and replaces all usages of the primitive DATETIME type with it.

MillisInstant mirrors the NanosInstant type. It uses org.joda.time.Instant rather than java.time.Instant for consistency with DATETIME, and it is backed by an INT64 representing the number of milliseconds since the epoch.

The majority of the changes in this PR are relatively trivial, but there are a few significant (and some potentially controversial) ones:

  • Updates the datetime getter/setter functions generated in ByteBuddyUtils to produce and consume instances of Long rather than Instant. This is necessary because these functions are expected to produce/consume the base type for logical types.
  • Updates all joda time literals in schema inference tests so they specify a timezone.
  • Throughout the SQL module I've replaced references to getBaseValue with getValue. Previously the only logical types used in SQL were PassThroughLogicalType instances, where the input type and base type were the same.

Changes that are not currently included in this PR but we might consider:

  • Replace all references to DateTime (addDateTimeField, addDateTimeValue, ..) with Instant, and deprecate or remove the DateTime references.

Post-Commit Tests Status (on master branch)

Lang SDK Apex Dataflow Flink Gearpump Samza Spark
Go Build Status --- --- Build Status --- --- Build Status
Java Build Status Build Status Build Status
Build Status
Build Status
Build Status
Build Status
Build Status Build Status Build Status
Build Status
Build Status
Python Build Status
Build Status
Build Status
Build Status
--- Build Status
Build Status
Build Status
Build Status
Build Status
--- --- Build Status
XLang --- --- --- Build Status --- --- Build Status

Pre-Commit Tests Status (on master branch)

--- Java Python Go Website
Non-portable Build Status Build Status
Build Status
Build Status Build Status
Portable --- Build Status --- ---

See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.

@TheNeuralBit
Copy link
Member Author

Run SQL PostCommit

@TheNeuralBit TheNeuralBit force-pushed the deprecate-datetime-2 branch from 9e589b8 to 2f59493 Compare April 20, 2020 20:43
@TheNeuralBit TheNeuralBit changed the title WIP: [BEAM-7554] Add MillisInstant logical type to replace DATETIME [BEAM-7554] Add MillisInstant logical type to replace DATETIME Apr 22, 2020
@TheNeuralBit TheNeuralBit requested a review from reuvenlax April 22, 2020 18:11
@TheNeuralBit TheNeuralBit force-pushed the deprecate-datetime-2 branch from 6f6fa60 to 633710a Compare April 27, 2020 21:56
@@ -419,7 +420,6 @@ public int hashCode() {
FLOAT,
DOUBLE,
STRING, // String.
DATETIME, // Date and time.
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm a little worried about this. Empirically many users are using schemas. Maybe we should start off by leaving DATETIME around and remove it later in another PR?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yeah we could deprecate it for a release or two before removing. Would we also need to keep support for the primitive DATETIME in IOs and SQL? I'd need to think about how that transition would work.

: combined.aggregateFieldBaseValue(
inputs.get(0), combineFn, fieldAggregation.outputField);
? byFields.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField)
: combined.aggregateField(inputs.get(0), combineFn, fieldAggregation.outputField);
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure I agree with this change. I think it's important that SQL work over user logical types by interpreting it as the base value. The user writing the SQL statement usually understands the base type of their logical type, and can write the SQL statement appropriately. This will break that.

Copy link
Member Author

Choose a reason for hiding this comment

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

What about adding the option for SqlTransform to convert unknown logical types to their base type at the input? I think that behavior would be effectively the same

Expressions.constant(index),
Expressions.constant(convertTo)),
convertTo);
Expressions.call(expression, "getValue", Expressions.constant(index)), convertTo);
Copy link
Contributor

Choose a reason for hiding this comment

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

I have the same concern with this change.


/** A timestamp represented as milliseconds since the epoch. */
public class MillisInstant extends MillisType<ReadableInstant> {
public static final String IDENTIFIER = "beam:logical_type:millis_instant:v1";
Copy link
Member Author

Choose a reason for hiding this comment

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

@reuvenlax what do you think about making this type (and maybe NanosInstant as well) parameterized by timezone?

The arrow approach seems useful: un-specified timezone indicates time-zone naive (e.g. joda time Instant), otherwise time zone parameter should reference a value in tzdata (and would map to joda time DateTime).

cc: @alexvanboxel

Copy link
Contributor

Choose a reason for hiding this comment

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

A timestamp type seems like it's by definition time-zone agnostic. If we want a datetime type, that should be a different type.

@robinyqiu
Copy link
Contributor

cc: @robinyqiu @ZijieSong946

@stale
Copy link

stale bot commented Sep 5, 2020

This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@stale stale bot added the stale label Sep 5, 2020
@stale
Copy link

stale bot commented Sep 12, 2020

This pull request has been closed due to lack of activity. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants