Also made a JIRA: https://issues.apache.org/jira/projects/BEAM/issues/BEAM-8738
On Mon, Nov 18, 2019 at 2:32 PM Sam Rohde <[email protected]> wrote: > Cool I wrote up https://github.com/apache/beam/pull/10146 > > On Mon, Nov 18, 2019 at 2:09 PM Luke Cwik <[email protected]> wrote: > >> Sam, I think doing that makes the most sense right now as we haven't yet >> had a strong enough consensus to change it so to support all of Beam's >> timestamps/durations it makes sense to still use the format but work around >> the limitation that is imposed. >> >> On Mon, Nov 18, 2019 at 11:25 AM Sam Rohde <[email protected]> wrote: >> >>> Timestamp related question: I want to modify Python's utils/timestamp.py >>> <https://github.com/apache/beam/blob/master/sdks/python/apache_beam/utils/timestamp.py> >>> module to include google.protobuf.timestamp to/from translation methods. >>> What do you guys think? Now that we know the timestamp.proto is implicitly >>> RFC3339 compliant, is it right to include translation methods that could >>> potentially break that compliance (a la min/max watermarks)? We already use >>> the timestamp.proto in: windows definitions >>> <https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/standard_window_fns.proto#L44>, >>> pubsub messages >>> <https://github.com/apache/beam/blob/master/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/pubsub.proto#L32>, >>> bundle applications >>> <https://github.com/apache/beam/blob/master/model/fn-execution/src/main/proto/beam_fn_api.proto#L173>, >>> metrics >>> <https://github.com/apache/beam/blob/master/model/fn-execution/src/main/proto/beam_fn_api.proto#L173>, >>> and logs >>> <https://github.com/apache/beam/blob/master/model/fn-execution/src/main/proto/beam_fn_api.proto#L804>. >>> Is my change okay? >>> >>> On Thu, Nov 14, 2019 at 3:40 PM Luke Cwik <[email protected]> wrote: >>> >>>> The timestamps flow both ways since: >>>> * IO authors are responsible for saying what the watermark timestamp is >>>> and stateful DoFns also allow for users to set timers in relative and >>>> processing time domains. >>>> * Runner authors need to understand and merge these timestamps together >>>> to compute what the global watermark is for a PCollection. >>>> >>>> On Thu, Nov 14, 2019 at 3:15 PM Sam Rohde <[email protected]> wrote: >>>> >>>>> My two cents are we just need a proto representation for timestamps >>>>> and durations that includes units. The underlying library can then >>>>> determine what to do with it. Then further, we can have a standard across >>>>> Beam SDKs and Runners of how to interpret the proto. Using a raw int64 for >>>>> timestamps and durations is confusing and *very very *bug prone (as >>>>> we have seen in the past). >>>>> >>>>> I don't know if this is relevant, but does Apache Beam have any >>>>> standards surrounding leap years or seconds? If we were to make our own >>>>> timestamp format, would we have to worry about that? Or is the timestamp >>>>> supplied to Beam a property of the underlying system giving Beam the >>>>> timestamp? If it is, then there may be some interop problems between >>>>> sources. >>>>> >>>>> On Wed, Nov 13, 2019 at 10:35 AM Luke Cwik <[email protected]> wrote: >>>>> >>>>>> I do agree that Apache Beam can represent dates and times with >>>>>> arbitrary precision and can do it many different ways. >>>>>> >>>>>> My argument has always been should around whether we restrict this >>>>>> range to a common standard to increase interoperability across other >>>>>> systems. For example, SQL database servers have varying degrees as to >>>>>> what >>>>>> ranges they support: >>>>>> * Oracle 10[1]: 0001-01-01 to 9999-12-31 >>>>>> * Oracle 11g[2]: Julian era, ranging from January 1, 4712 BCE through >>>>>> December 31, 9999 CE (Common Era, or 'AD'). Unless BCE ('BC' in the >>>>>> format >>>>>> mask) >>>>>> * MySQL[3]: '1000-01-01 00:00:00' to '9999-12-31 23:59:59' >>>>>> * Microsoft SQL: January 1, 1753, through December 31, 9999 for >>>>>> datetime[4] and January 1,1 CE through December 31, 9999 CE for >>>>>> datetime2[5] >>>>>> >>>>>> The common case of the global window containing timestamps that are >>>>>> before and after all of these supported ranges above means that our users >>>>>> can't represent a global window within a database using its common data >>>>>> types. >>>>>> >>>>>> 1: https://docs.oracle.com/javadb/10.8.3.0/ref/rrefdttlimits.html >>>>>> 2: >>>>>> https://docs.oracle.com/cd/B28359_01/server.111/b28318/datatype.htm#CNCPT413 >>>>>> 3: https://dev.mysql.com/doc/refman/8.0/en/datetime.html >>>>>> 4: >>>>>> https://docs.microsoft.com/en-us/sql/t-sql/data-types/datetime-transact-sql?view=sql-server-ver15 >>>>>> 5: >>>>>> https://docs.microsoft.com/en-us/sql/t-sql/data-types/datetime2-transact-sql?view=sql-server-ver15 >>>>>> >>>>>> On Wed, Nov 13, 2019 at 3:28 AM Jan Lukavský <[email protected]> wrote: >>>>>> >>>>>>> Hi, >>>>>>> >>>>>>> just an idea on these related topics that appear these days - it >>>>>>> might help to realize, that what we actually don't need a full >>>>>>> arithmetic >>>>>>> on timestamps (Beam model IMHO doesn't need to know exactly what is the >>>>>>> exact difference of two events). What we actually need is a slightly >>>>>>> simplified algebra. Given two timestamps T1 and T2 and a "duration" (a >>>>>>> different type from timestamp), we need operations (not 100% sure that >>>>>>> this >>>>>>> is exhaustive, but seems to be): >>>>>>> >>>>>>> - is_preceding(T1, T2): bool >>>>>>> >>>>>>> - important !is_preceding(T1, T2) does NOT imply that >>>>>>> is_preceding(T2, T1) - !is_preceding(T1, T2) && !is_preceding(T2, T1) >>>>>>> would >>>>>>> mean events are _concurrent_ >>>>>>> >>>>>>> - this relation has to be also antisymmetric >>>>>>> >>>>>>> - given this function we can construct a comparator, where >>>>>>> multiple distinct timestamps can be "equal" (or with no particular >>>>>>> ordering, which is natural property of time) >>>>>>> >>>>>>> - min_timestamp_following(T1, duration): T2 >>>>>>> >>>>>>> - that would return a timestamp for which is_preceding(T1 + >>>>>>> duration, T2) would return true and no other timestamp X would exist for >>>>>>> which is_preceding(T1 + duration, X) && is_preceding(X, T2) would be >>>>>>> true >>>>>>> >>>>>>> - actually, this function would serve as the definition for the >>>>>>> duration object >>>>>>> >>>>>>> If we can supply this algebra, it seems that we can use any >>>>>>> representation of timestamps and intervals. It might be (probably) even >>>>>>> possible to let user specify his own type used as timestamps and >>>>>>> durations, >>>>>>> which could solve the issues of not currently being able to correctly >>>>>>> represent timestamps lower than Long.MIN_VALUE (although we can get data >>>>>>> for that low timestamps - cosmic microwave background being one example >>>>>>> :)). Specifying this algebra actually probably boils down to proposal >>>>>>> (3) >>>>>>> in Robert's thread [1]. >>>>>>> >>>>>>> Just my 2 cents. >>>>>>> >>>>>>> Jan >>>>>>> >>>>>>> [1] >>>>>>> https://lists.apache.org/thread.html/1672898393cb0d54a77a879be0fb5725902289a3e5063d0f9ec36fe1@%3Cdev.beam.apache.org%3E >>>>>>> On 11/13/19 10:11 AM, jincheng sun wrote: >>>>>>> >>>>>>> Thanks for bringing up this discussion @Luke. >>>>>>> >>>>>>> As @Kenn mentioned, in Beam we have defined the constants value for >>>>>>> the min/max/end of global window. I noticed that >>>>>>> google.protobuf.Timestamp/Duration is only used in window >>>>>>> definitions, such as FixedWindowsPayload, SlidingWindowsPayload, >>>>>>> SessionsPayload, etc. >>>>>>> >>>>>>> I think that both RFC 3339 and Beam's current implementation are big >>>>>>> enough to express a common window definitions. But users can really >>>>>>> define a window size that outside the scope of the RFC 3339. >>>>>>> Conceptually, we should not limit the time range for >>>>>>> window(although I think the range of RPC 3339 is big enough in most >>>>>>> cases). >>>>>>> >>>>>>> To ensure that people well know the background of the discussion, >>>>>>> hope you don't mind that I put the original conversion thread[1] here. >>>>>>> >>>>>>> Best, >>>>>>> Jincheng >>>>>>> >>>>>>> [1] https://github.com/apache/beam/pull/10041#discussion_r344380809 >>>>>>> >>>>>>> Robert Bradshaw <[email protected]> 于2019年11月12日周二 下午4:09写道: >>>>>>> >>>>>>>> I agree about it being a tagged union in the model (together with >>>>>>>> actual_time(...) - epsilon). It's not just a performance hack >>>>>>>> though, >>>>>>>> it's also (as discussed elsewhere) a question of being able to find >>>>>>>> an >>>>>>>> embedding into existing datetime libraries. The real question here >>>>>>>> is >>>>>>>> whether we should limit ourselves to just these 10000 years AD, or >>>>>>>> find value in being able to process events for the lifetime of the >>>>>>>> universe (or, at least, recorded human history). Artificially >>>>>>>> limiting >>>>>>>> in this way would seem surprising to me at least. >>>>>>>> >>>>>>>> On Mon, Nov 11, 2019 at 11:58 PM Kenneth Knowles <[email protected]> >>>>>>>> wrote: >>>>>>>> > >>>>>>>> > The max timestamp, min timestamp, and end of the global window >>>>>>>> are all performance hacks in my view. Timestamps in beam are really a >>>>>>>> tagged union: >>>>>>>> > >>>>>>>> > timestamp ::= min | max | end_of_global | actual_time(... >>>>>>>> some quantitative timestamp ...) >>>>>>>> > >>>>>>>> > with the ordering >>>>>>>> > >>>>>>>> > min < actual_time(...) < end_of_global < max >>>>>>>> > >>>>>>>> > We chose arbitrary numbers so that we could do simple numeric >>>>>>>> comparisons and arithmetic. >>>>>>>> > >>>>>>>> > Kenn >>>>>>>> > >>>>>>>> > On Mon, Nov 11, 2019 at 2:03 PM Luke Cwik <[email protected]> >>>>>>>> wrote: >>>>>>>> >> >>>>>>>> >> While crites@ was investigating using protobuf to represent >>>>>>>> Apache Beam timestamps within the TestStreamEvents, he found out that >>>>>>>> the >>>>>>>> well known type google.protobuf.Timestamp doesn't support certain >>>>>>>> timestamps we were using in our tests (specifically the max timestamp >>>>>>>> that >>>>>>>> Apache Beam supports). >>>>>>>> >> >>>>>>>> >> This lead me to investigate and the well known type >>>>>>>> google.protobuf.Timestamp supports dates/times from >>>>>>>> 0001-01-01T00:00:00Z to >>>>>>>> 9999-12-31T23:59:59.999999999Z which is much smaller than the timestamp >>>>>>>> range that Apache Beam currently supports -9223372036854775ms to >>>>>>>> 9223372036854775ms which is about 292277BC to 294247AD (it was >>>>>>>> difficult to >>>>>>>> find a time range that represented this). >>>>>>>> >> >>>>>>>> >> Similarly the google.protobuf.Duration represents any time range >>>>>>>> over those ~10000 years. Google decided to limit their range to be >>>>>>>> compatible with the RFC 3339[2] standard to which does simplify many >>>>>>>> things >>>>>>>> since it guarantees that all RFC 3339 time parsing/manipulation >>>>>>>> libraries >>>>>>>> are supported. >>>>>>>> >> >>>>>>>> >> Should we: >>>>>>>> >> A) define our own timestamp/duration types to be able to >>>>>>>> represent the full time range that Apache Beam can express? >>>>>>>> >> B) limit the valid timestamps in Apache Beam to some standard >>>>>>>> such as RFC 3339? >>>>>>>> >> >>>>>>>> >> This discussion is somewhat related to the efforts to support >>>>>>>> nano timestamps[2]. >>>>>>>> >> >>>>>>>> >> 1: https://tools.ietf.org/html/rfc3339 >>>>>>>> >> 2: >>>>>>>> https://lists.apache.org/thread.html/86a4dcabdaa1dd93c9a55d16ee51edcff6266eda05221acbf9cf666d@%3Cdev.beam.apache.org%3E >>>>>>>> >>>>>>>
