I believe I've addressed all outstanding feedback on the PR, are there
other thoughts on this that should be discussed before we move forward
towards an implementation / voting plan?

On Tue, Apr 27, 2021 at 6:00 PM Wes McKinney <wesmck...@gmail.com> wrote:

> Thanks Micah — I commented in the PR. Once we've settled on the details we
> can come up with an implementation / vote plan
>
> On Tue, Apr 27, 2021 at 1:12 PM Micah Kornfield <emkornfi...@gmail.com>
> wrote:
>
>> To nudge this along I opend up https://github.com/apache/arrow/pull/10177
>>
>>
>> Comments welcome.
>>
>> On Sun, Apr 11, 2021 at 9:38 PM Micah Kornfield <emkornfi...@gmail.com>
>> wrote:
>>
>>> If there are no more comments on this maybe we should update the
>>> original RFC PR and ensure we are OK with it in principle (Dmitry do you
>>> want to do this or should we start a new PR)?  I can try to work on the
>>> C++/Python and Java code in the next few weeks.
>>>
>>>
>>> On Sun, Apr 4, 2021 at 1:35 PM Micah Kornfield <emkornfi...@gmail.com>
>>> wrote:
>>>
>>>> Looking more at the Postgres spec and storage details, I'd be
>>>>> supportive of having a COMPLEX interval type which could be a packed
>>>>> type (possibly using the same 16-byte storage layout as Postgres --
>>>>> depending on whether this complex interval needs granularity smaller
>>>>> than seconds, more analysis needed)
>>>>
>>>>
>>>> I agree we seem to be coalescing towards this representation which IIUC
>>>> is (32 bit months, 32 bit days, 64 bit for Seconds + fractional seconds).
>>>> I think the main questions here are:
>>>>
>>>> 1.  Configurable sub-second granularity (I would again lean no and just
>>>> say nanoseconds, since this is the highest granularity other Arrow time
>>>> types support).
>>>> 2.  Range compatibility with other interval types (if we fix at
>>>> nanoseconds IIUC there are some extreme values that couldn't be converted
>>>> to Arrow).  I would guess that these extrema are rare enough that this
>>>> should not be an issue.
>>>>
>>>>  I think that adding an entry to
>>>>> the IntervalUnit enum doesn't pose any forward compatibility problems
>>>>> (because implementations _should_ be able to recognized an unsupported
>>>>> unit?
>>>>
>>>>
>>>> I also agree with this, I can do an audit before an official proposal
>>>> of Java and C++.
>>>>
>>>> -Micah
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Sun, Apr 4, 2021 at 11:10 AM Wes McKinney <wesmck...@gmail.com>
>>>> wrote:
>>>>
>>>>> Looking more at the Postgres spec and storage details, I'd be
>>>>> supportive of having a COMPLEX interval type which could be a packed
>>>>> type (possibly using the same 16-byte storage layout as Postgres --
>>>>> depending on whether this complex interval needs granularity smaller
>>>>> than seconds, more analysis needed). I think that adding an entry to
>>>>> the IntervalUnit enum doesn't pose any forward compatibility problems
>>>>> (because implementations _should_ be able to recognized an unsupported
>>>>> unit?). It isn't wonderful to have 3 different types of intervals
>>>>> (ideally just SIMPLE and COMPLEX), but it seems like something we can
>>>>> live with. If we're going to do this, we should try to do it properly
>>>>> now since I would probably object at adding a 4th interval type if the
>>>>> need for it came up in the future.
>>>>>
>>>>> On Fri, Apr 2, 2021 at 5:37 PM Micah Kornfield <emkornfi...@gmail.com>
>>>>> wrote:
>>>>> >
>>>>> > >
>>>>> > > However it seems a little unfortunate that there is now way to
>>>>> represent a
>>>>> > > "common" interval like "1 week and 1 hour" with native arrow types
>>>>> >
>>>>> >
>>>>> > I might have misunderstood,but at least in postgres, I thought this
>>>>> boils
>>>>> > down to "0 months, 7 days, 3600 seconds".  Since months is 0, this
>>>>> seems
>>>>> > like it fits squarely in the existing interval type Days_Mills.
>>>>> >
>>>>> > I thought what can't be represented today is "1 Year 1 Hour".  It
>>>>> seems
>>>>> > like none of the proposals so far cover weeks as an explicit type?
>>>>> >
>>>>> > On Fri, Apr 2, 2021 at 2:42 PM Andrew Lamb <al...@influxdata.com>
>>>>> wrote:
>>>>> >
>>>>> > > I think it is plausible that we use Arrow structs to create a
>>>>> synthetic
>>>>> > > interval type for DataFusion (I don't have a compelling usecase to
>>>>> store
>>>>> > > the intervals themselves, or to expose them outside of DataFusion).
>>>>> > >
>>>>> > > However it seems a little unfortunate that there is now way to
>>>>> represent a
>>>>> > > "common" interval like "1 week and 1 hour" with native arrow types
>>>>> > >
>>>>> > >
>>>>> > >
>>>>> > > On Fri, Apr 2, 2021 at 4:38 PM Micah Kornfield <
>>>>> emkornfi...@gmail.com>
>>>>> > > wrote:
>>>>> > >
>>>>> > >> The real usecase I have is "postgres compatibility"
>>>>> > >>
>>>>> > >>
>>>>> > >> Yeah, I'm a little conflicted on this.  A broader analysis might
>>>>> be
>>>>> > >> necessary and I'd welcome others thoughts, but at what point
>>>>> should we
>>>>> > >> mostly consider the type system closed?  Should we be aiming for
>>>>> full
>>>>> > >> parity with ANSI SQL/Postgres SQL or something else?
>>>>> > >>
>>>>> > >>
>>>>> > >>> I have no known need for the actual postgres timestamp internal
>>>>> > >>> representation.
>>>>> > >>
>>>>> > >>
>>>>> > >> I suppose there is an edge case that the seconds range is larger
>>>>> for
>>>>> > >> microseconds compared to nanoseconds with the simple
>>>>> representation.  But
>>>>> > >> that seems minor.
>>>>> > >>
>>>>> > >> On Fri, Apr 2, 2021 at 1:25 PM Andrew Lamb <al...@influxdata.com>
>>>>> wrote:
>>>>> > >>
>>>>> > >>> The real usecase I have is "postgres compatibility" - in the
>>>>> sense that
>>>>> > >>> we can write SQL queries / expressions that use postgres
>>>>> interval type [1]
>>>>> > >>> and corresponding expressions with the full postgres interval
>>>>> range. I have
>>>>> > >>> no known need for the actual postgres timestamp internal
>>>>> representation.
>>>>> > >>>
>>>>> > >>> A more vague usecase I would like (but do not strictly need) is
>>>>> for
>>>>> > >>> expressing intervals up to the same precision as IOx's underlying
>>>>> > >>> timestamps (which are stored as nanosecond precision).
>>>>> > >>>
>>>>> > >>> [1]
>>>>> > >>>
>>>>> https://www.postgresql.org/docs/current/datatype-datetime.html#DATATYPE-INTERVAL-INPUT
>>>>> > >>>
>>>>> > >>> "Internally interval values are stored as months, days, and
>>>>> seconds.
>>>>> > >>> This is done because the number of days in a month varies, and a
>>>>> day can
>>>>> > >>> have 23 or 25 hours if a daylight savings time adjustment is
>>>>> involved. The
>>>>> > >>> months and days fields are integers while the seconds field can
>>>>> store
>>>>> > >>> fractions. Because intervals are usually created from constant
>>>>> strings or
>>>>> > >>>  timestamp subtraction, this storage method works well in most
>>>>> cases,
>>>>> > >>> but can cause unexpected results:
>>>>> > >>> "
>>>>> > >>>
>>>>> > >>> On Fri, Apr 2, 2021 at 3:53 PM Micah Kornfield <
>>>>> emkornfi...@gmail.com>
>>>>> > >>> wrote:
>>>>> > >>>
>>>>> > >>>> Andrew is the use-case you have simply postgres compatibility
>>>>> or is it
>>>>> > >>>> more
>>>>> > >>>> extensive?
>>>>> > >>>>
>>>>> > >>>> One potential problem with combining Month and Day fields, is
>>>>> that the
>>>>> > >>>> type
>>>>> > >>>> no longer has a defined sort order (the existing
>>>>> Day-Millisecond type
>>>>> > >>>> without assumptions, in particular because I don't think today
>>>>> there is
>>>>> > >>>> an
>>>>> > >>>> explicit constraint on the bounds for the millisecond
>>>>> component).
>>>>> > >>>>
>>>>> > >>>> -Micah
>>>>> > >>>>
>>>>> > >>>>
>>>>> > >>>>
>>>>> > >>>> On Wed, Mar 31, 2021 at 9:03 AM Antoine Pitrou <
>>>>> anto...@python.org>
>>>>> > >>>> wrote:
>>>>> > >>>>
>>>>> > >>>> >
>>>>> > >>>> > Le 31/03/2021 à 17:55, Micah Kornfield a écrit :
>>>>> > >>>> > > Thanks for the feedback.  A couple of points here and some
>>>>> responses
>>>>> > >>>> > below.
>>>>> > >>>> > >
>>>>> > >>>> > > * One other question is whether the Nanoseconds should
>>>>> actually be
>>>>> > >>>> > > configurable (i.e. use milliseconds or microseconds).  I
>>>>> would lean
>>>>> > >>>> > towards
>>>>> > >>>> > > no.
>>>>> > >>>> >
>>>>> > >>>> > Same for me.
>>>>> > >>>> >
>>>>> > >>>> > > * I'm also still not 100% convinced we need this as a first
>>>>> class
>>>>> > >>>> type in
>>>>> > >>>> > > arrow or if we should be looking more closely at the Struct
>>>>> (in the
>>>>> > >>>> Arrow
>>>>> > >>>> > > sense) based implementation.  In the future where
>>>>> alternative
>>>>> > >>>> encodings
>>>>> > >>>> > are
>>>>> > >>>> > > supported, this could allow for much smaller footprints for
>>>>> this
>>>>> > >>>> type.
>>>>> > >>>> >
>>>>> > >>>> > Having a "packed" first class type allows for better locality
>>>>> when
>>>>> > >>>> > accessing data.  It doesn't sound very likely that you'd
>>>>> access only
>>>>> > >>>> one
>>>>> > >>>> > component of the interval.
>>>>> > >>>> >
>>>>> > >>>> > But I have no idea how important this is, and temporal
>>>>> datetypes are
>>>>> > >>>> > generally cumbersome to add support for (conversions,
>>>>> arithmetic,
>>>>> > >>>> etc.),
>>>>> > >>>> > so it would be nice to avoid adding too many of them :-)
>>>>> > >>>> >
>>>>> > >>>> > Regards
>>>>> > >>>> >
>>>>> > >>>> > Antoine.
>>>>> > >>>> >
>>>>> > >>>> >
>>>>> > >>>> >
>>>>> > >>>> > >
>>>>> > >>>> > > The 3
>>>>> > >>>> > >> field implementation doesn't seem to have any way to
>>>>> represent
>>>>> > >>>> integral
>>>>> > >>>> > >> days, so I am also not sure about that one.
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > > Sorry this was an email gaffe.  I intended Month (32 bit
>>>>> int), Day
>>>>> > >>>> (32
>>>>> > >>>> > bit
>>>>> > >>>> > > int), Nanosecond (64 bit int).
>>>>> > >>>> > >
>>>>> > >>>> > > OTOH I don't really understand the point of supporting "the
>>>>> most
>>>>> > >>>> > >> reasonable ranges for Year, Month and Nanoseconds
>>>>> independently".
>>>>> > >>>> What
>>>>> > >>>> > >> does it bring to encode more than one month in the
>>>>> nanoseconds
>>>>> > >>>> field?
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > > I'm happy with simplicity.   In the past there has been some
>>>>> > >>>> reference to
>>>>> > >>>> > > people wanting to store very large timestamps (fall out of
>>>>> > >>>> Nanoseconds
>>>>> > >>>> > max
>>>>> > >>>> > > representable value) but we've concluded that this wasn't
>>>>> something
>>>>> > >>>> that
>>>>> > >>>> > we
>>>>> > >>>> > > wanted to really support.
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > >
>>>>> > >>>> > > On Wed, Mar 31, 2021 at 4:49 AM Antoine Pitrou <
>>>>> anto...@python.org>
>>>>> > >>>> > wrote:
>>>>> > >>>> > >
>>>>> > >>>> > >>
>>>>> > >>>> > >> I would favour the following characteristics :
>>>>> > >>>> > >> - support for nanoseconds (especially as other Arrow
>>>>> temporal types
>>>>> > >>>> > >> support it)
>>>>> > >>>> > >> - easy to handle (which excludes the ZetaSQL
>>>>> representtaion IMHO)
>>>>> > >>>> > >>
>>>>> > >>>> > >> OTOH I don't really understand the point of supporting
>>>>> "the most
>>>>> > >>>> > >> reasonable ranges for Year, Month and Nanoseconds
>>>>> independently".
>>>>> > >>>> What
>>>>> > >>>> > >> does it bring to encode more than one month in the
>>>>> nanoseconds
>>>>> > >>>> field?
>>>>> > >>>> > >> You can already use the Duration type for that.
>>>>> > >>>> > >>
>>>>> > >>>> > >> Regards
>>>>> > >>>> > >>
>>>>> > >>>> > >> Antoine.
>>>>> > >>>> > >>
>>>>> > >>>> > >>
>>>>> > >>>> > >> Le 31/03/2021 à 05:48, Micah Kornfield a écrit :
>>>>> > >>>> > >>> To follow-up on this conversation I did some analysis on
>>>>> interval
>>>>> > >>>> > types:
>>>>> > >>>> > >>>
>>>>> > >>>> > >>>
>>>>> > >>>> > >>
>>>>> > >>>> >
>>>>> > >>>>
>>>>> https://docs.google.com/document/d/1i1E_fdQ_xODZcAhsV11Pfq27O50k679OYHXFJpm9NS0/edit
>>>>> > >>>> > >> Please feel free to add more details/systems I missed.
>>>>> > >>>> > >>>
>>>>> > >>>> > >>> Given the disparate requirements of different systems I
>>>>> think the
>>>>> > >>>> > >> following might make sense for official types (if there
>>>>> isn't
>>>>> > >>>> > consensus, I
>>>>> > >>>> > >> might try to contributation extension Array
>>>>> implementations for
>>>>> > >>>> them to
>>>>> > >>>> > >> Java and C++/Python separately).
>>>>> > >>>> > >>>
>>>>> > >>>> > >>> 1.  3 fields: Year (32 bit), Month (32 bit), Nanoseconds
>>>>> (64 bit)
>>>>> > >>>> all
>>>>> > >>>> > >> signed.
>>>>> > >>>> > >>> 2.  Postgres representation (Downside is it doesn't
>>>>> support
>>>>> > >>>> > Nanoseconds,
>>>>> > >>>> > >> only microseconds).
>>>>> > >>>> > >>> 3.  ZetaSQL implementation (Requires some bit
>>>>> manipulation) but
>>>>> > >>>> > supports
>>>>> > >>>> > >> the most reasonable ranges for Year, Month and Nanoseconds
>>>>> > >>>> > independently.
>>>>> > >>>> > >>>
>>>>> > >>>> > >>> Thoughts?
>>>>> > >>>> > >>>
>>>>> > >>>> > >>> Micah
>>>>> > >>>> > >>>
>>>>> > >>>> > >>> On 2021/02/18 04:30:55 Micah Kornfield wrote:
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> I didn’t find any page/documentation on how to do RFC
>>>>> in Arrow
>>>>> > >>>> > >> protocol,
>>>>> > >>>> > >>>>> so can anyone point me to it or PR with email will be
>>>>> enough?
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> That is enough to start discussion.  Before formal
>>>>> acceptance and
>>>>> > >>>> > >> merging
>>>>> > >>>> > >>>> of the PR there needs to be a Java and C++
>>>>> implementations for
>>>>> > >>>> the
>>>>> > >>>> > type
>>>>> > >>>> > >>>> that pass integration tests.  At the time this guideline
>>>>> was
>>>>> > >>>> > instituted
>>>>> > >>>> > >>>> Java and C++ were considered the "reference"
>>>>> implementations (I
>>>>> > >>>> think
>>>>> > >>>> > >> they
>>>>> > >>>> > >>>> still have the most complete integration test coverage).
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> My understanding is that the current modelling of
>>>>> intervals
>>>>> > >>>> mimics SQL
>>>>> > >>>> > >>>> standards (e.g. SQL Server [1]).  So it would also be
>>>>> good to
>>>>> > >>>> step
>>>>> > >>>> > back
>>>>> > >>>> > >> and
>>>>> > >>>> > >>>> understand what problem DF is trying to solve and how it
>>>>> differs
>>>>> > >>>> from
>>>>> > >>>> > >> other
>>>>> > >>>> > >>>> SQL implementations.  I'd be hesitant to accept COMPLEX
>>>>> as a new
>>>>> > >>>> type
>>>>> > >>>> > >>>> without a much deeper analysis into calendar
>>>>> representations
>>>>> > >>>> within
>>>>> > >>>> > >> Arrow
>>>>> > >>>> > >>>> and how they relate to other existing systems (e.g. Hive
>>>>> and some
>>>>> > >>>> > >>>> assortment of existing SQL databases).  For instance the
>>>>> current
>>>>> > >>>> > >> modelling
>>>>> > >>>> > >>>> of timestamps does not lend itself to constructing a
>>>>> COMPLEX
>>>>> > >>>> interval
>>>>> > >>>> > >> type
>>>>> > >>>> > >>>> particularly well. (Duration was introduced for this
>>>>> reason).
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> I think both Wes's suggestion of FixedSizeBinary and
>>>>> Andrew's of
>>>>> > >>>> > >> composing
>>>>> > >>>> > >>>> the with a struct are good stop-gaps.  These obviously
>>>>> have
>>>>> > >>>> different
>>>>> > >>>> > >>>> trade-offs.  Ultimately, it would be good to define
>>>>> common
>>>>> > >>>> extension
>>>>> > >>>> > >> types
>>>>> > >>>> > >>>> that can represent this use-case if there really is
>>>>> demand for
>>>>> > >>>> it (if
>>>>> > >>>> > it
>>>>> > >>>> > >>>> doesn't become a top level type).
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> [1]
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>
>>>>> > >>>> >
>>>>> > >>>>
>>>>> https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/interval-data-types?view=sql-server-ver15
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> -Micah
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>> On Wed, Feb 17, 2021 at 2:05 PM Andrew Lamb <
>>>>> > >>>> al...@influxdata.com>
>>>>> > >>>> > >> wrote:
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>>>> That is a great suggestion Wes, thank you.
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> I wonder if we could get away with a 128 bit
>>>>> representation
>>>>> > >>>> that is
>>>>> > >>>> > the
>>>>> > >>>> > >>>>> concatenation of the two existing interval types
>>>>> > >>>> > (YearMonth)(DayTime).
>>>>> > >>>> > >> Or
>>>>> > >>>> > >>>>> maybe even define a `struct` type with those fields
>>>>> that is
>>>>> > >>>> used by
>>>>> > >>>> > >>>>> DataFusion.
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> Basically, given our reading of the Arrow spec[1], it is
>>>>> > >>>> currently
>>>>> > >>>> > not
>>>>> > >>>> > >>>>> possible to precisely represent an interval that has
>>>>> both
>>>>> > >>>> monthly and
>>>>> > >>>> > >>>>> sub-montly granularity.
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> As Dmtry says, if you have an interval seemingly simple
>>>>> like  1
>>>>> > >>>> > month,
>>>>> > >>>> > >> 1
>>>>> > >>>> > >>>>> day
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> Using IntervalUnit(YEAR_MONTH) can't represent the 1 day
>>>>> > >>>> > >>>>> Using IntervalUnit(DAY_TIME) can't represent the month
>>>>> as
>>>>> > >>>> different
>>>>> > >>>> > >> months
>>>>> > >>>> > >>>>> have different numbers of days
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> [1]
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>
>>>>> > >>>>
>>>>> https://github.com/apache/arrow/blob/master/format/Schema.fbs#L249-L260
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>> On Wed, Feb 17, 2021 at 5:01 PM Wes McKinney <
>>>>> > >>>> wesmck...@gmail.com>
>>>>> > >>>> > >> wrote:
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>>> On Wed, Feb 17, 2021 at 3:46 PM <t...@dmtry.me> wrote:
>>>>> > >>>> > >>>>>>>
>>>>> > >>>> > >>>>>>>> It's unclear to me that this needs to be introduced
>>>>> into the
>>>>> > >>>> > >>>>> top-level
>>>>> > >>>> > >>>>>>>
>>>>> > >>>> > >>>>>>> Similar thing to columnar format, How to store
>>>>> interval like 1
>>>>> > >>>> > month
>>>>> > >>>> > >> 1
>>>>> > >>>> > >>>>>> day 1 hour? It’s not possible to do it without
>>>>> converting 1
>>>>> > >>>> month to
>>>>> > >>>> > >> 30
>>>>> > >>>> > >>>>>> days, which is a bad way.
>>>>> > >>>> > >>>>>>>
>>>>> > >>>> > >>>>>>
>>>>> > >>>> > >>>>>> Presumably you can represent a complex interval in a
>>>>> fixed
>>>>> > >>>> number of
>>>>> > >>>> > >>>>>> bytes, and then embed the data in a FixedSizeBinary
>>>>> type. You
>>>>> > >>>> can
>>>>> > >>>> > >>>>>> adorn this type with extension type metadata so that
>>>>> > >>>> DataFusion can
>>>>> > >>>> > >>>>>> then apply Interval semantics to it. This could also
>>>>> serve as
>>>>> > >>>> an
>>>>> > >>>> > >>>>>> interim strategy for you to proceed with
>>>>> implementation while
>>>>> > >>>> > >>>>>> proposing a top-level type to the Arrow format (which
>>>>> may or
>>>>> > >>>> may not
>>>>> > >>>> > >>>>>> be accepting) so you aren't blocked on acceptance of
>>>>> changes
>>>>> > >>>> into
>>>>> > >>>> > >>>>>> Schema.fbs.
>>>>> > >>>> > >>>>>>
>>>>> > >>>> > >>>>>>>> On 17 Feb 2021, at 21:02, Wes McKinney <
>>>>> wesmck...@gmail.com>
>>>>> > >>>> > wrote:
>>>>> > >>>> > >>>>>>>>
>>>>> > >>>> > >>>>>>>> It's unclear to me that this needs to be introduced
>>>>> into the
>>>>> > >>>> > >>>>> top-level
>>>>> > >>>> > >>>>>>>> columnar format without more analysis — have you
>>>>> considered
>>>>> > >>>> > >>>>>>>> implementing this for DataFusion as an extension
>>>>> type for
>>>>> > >>>> the time
>>>>> > >>>> > >>>>>>>> being?
>>>>> > >>>> > >>>>>>>>
>>>>> > >>>> > >>>>>>>> On Wed, Feb 17, 2021 at 11:59 AM t...@dmtry.me
>>>>> <mailto:
>>>>> > >>>> > >> t...@dmtry.me
>>>>> > >>>> > >>>>>>
>>>>> > >>>> > >>>>>> <t...@dmtry.me <mailto:t...@dmtry.me>> wrote:
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> Hi,
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> For now, There are only two types of IntervalUnit
>>>>> inside
>>>>> > >>>> Arrow:
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> - YearMonth - month stored as int32
>>>>> > >>>> > >>>>>>>>> - DayTime - days as int32 and time in milliseconds
>>>>> as in32.
>>>>> > >>>> > Total
>>>>> > >>>> > >>>>>> (64 bites)
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> Since DF is using Arrow, It’s not possible to store
>>>>> > >>>> “Complex”
>>>>> > >>>> > >>>>>> intervals such 1 MONTH 1 DAY 1 HOUR.
>>>>> > >>>> > >>>>>>>>> I think, the best way to understand the problem
>>>>> will be to
>>>>> > >>>> read a
>>>>> > >>>> > >>>>>> comment from DF codebase:
>>>>> > >>>> > >>>>>>
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>
>>>>> > >>>> >
>>>>> > >>>>
>>>>> https://github.com/apache/arrow/blob/bca7d2fe84ccd8fc1129cb4d85448eb0779c52c3/rust/datafusion/src/sql/planner.rs#L1148
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>>          // Interval is tricky thing
>>>>> > >>>> > >>>>>>>>>          // 1 day is not 24 hours because
>>>>> timezones, 1 year
>>>>> > >>>> !=
>>>>> > >>>> > >>>>> 365/364!
>>>>> > >>>> > >>>>>> 30 days != 1 month
>>>>> > >>>> > >>>>>>>>>          // The true way to store and calculate
>>>>> intervals
>>>>> > >>>> is to
>>>>> > >>>> > >> store
>>>>> > >>>> > >>>>>> it as it defined
>>>>> > >>>> > >>>>>>>>>          // Due the fact that Arrow supports only
>>>>> two types
>>>>> > >>>> > >> YearMonth
>>>>> > >>>> > >>>>>> (month) and DayTime (day, time)
>>>>> > >>>> > >>>>>>>>>          // It's not possible to store complex
>>>>> intervals
>>>>> > >>>> > >>>>>>>>>          // It's possible to do select (NOW() +
>>>>> INTERVAL '1
>>>>> > >>>> > year') +
>>>>> > >>>> > >>>>>> INTERVAL '1 day'; as workaround
>>>>> > >>>> > >>>>>>>>>          if result_month != 0 && (result_days != 0
>>>>> ||
>>>>> > >>>> > result_millis
>>>>> > >>>> > >> !=
>>>>> > >>>> > >>>>>> 0) {
>>>>> > >>>> > >>>>>>>>>              return
>>>>> > >>>> Err(DataFusionError::NotImplemented(format!(
>>>>> > >>>> > >>>>>>>>>                  "DF does not support intervals
>>>>> that have
>>>>> > >>>> both a
>>>>> > >>>> > >>>>>> Year/Month part as well as Days/Hours/Mins/Seconds:
>>>>> {:?}.
>>>>> > >>>> Hint: try
>>>>> > >>>> > >>>>>> breaking the interval into two parts, one with
>>>>> Year/Month and
>>>>> > >>>> the
>>>>> > >>>> > >> other
>>>>> > >>>> > >>>>>> with Days/Hours/Mins/Seconds - e.g. (NOW() + INTERVAL
>>>>> '1
>>>>> > >>>> year') +
>>>>> > >>>> > >>>>> INTERVAL
>>>>> > >>>> > >>>>>> '1 day'",
>>>>> > >>>> > >>>>>>>>>                  value
>>>>> > >>>> > >>>>>>>>>              )));
>>>>> > >>>> > >>>>>>>>>          }
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> I prepared a PR
>>>>> > >>>> https://github.com/apache/arrow/pull/9516/files
>>>>> > >>>> > <
>>>>> > >>>> > >>>>>> https://github.com/apache/arrow/pull/9516/files> <
>>>>> > >>>> > >>>>>> https://github.com/apache/arrow/pull/9516/files <
>>>>> > >>>> > >>>>>> https://github.com/apache/arrow/pull/9516/files>> that
>>>>> > >>>> introduce a
>>>>> > >>>> > >> new
>>>>> > >>>> > >>>>>> type for IntervalUnit called Complex, that store both
>>>>> > >>>> YearMonth and
>>>>> > >>>> > >>>>> DayTime
>>>>> > >>>> > >>>>>> to support complex interval.
>>>>> > >>>> > >>>>>>>>> I didn’t find any page/documentation on how to do
>>>>> RFC in
>>>>> > >>>> Arrow
>>>>> > >>>> > >>>>>> protocol, so can anyone point me to it or PR with
>>>>> email will be
>>>>> > >>>> > >> enough?
>>>>> > >>>> > >>>>>>>>>
>>>>> > >>>> > >>>>>>>>> Thanks.
>>>>> > >>>> > >>>>>>>
>>>>> > >>>> > >>>>>>
>>>>> > >>>> > >>>>>
>>>>> > >>>> > >>>>
>>>>> > >>>> > >>
>>>>> > >>>> > >
>>>>> > >>>> >
>>>>> > >>>>
>>>>> > >>>
>>>>>
>>>>

Reply via email to