I could be mistaken, but I seem to remember ANSI SQL doesn't have timezones, and therefore as far as it is concerned days can only have 24 hours. Whether this assumption should be preserved in databases that support timezones though...
On Thu, 3 Jul 2025, 18:32 Micah Kornfield, <emkornfi...@gmail.com> wrote: > > > > Putting this to one side, I think the proposed DayTimeInterval still > > needs a separate day component in order to be SQL compliant, as days may > > not have 24 hours due to daylight savings time. I think what is being > > suggested might be better described as a duration type. > > > This has come up a few times. It seems the SQL spec defines 1 day as > exactly 24 hours (86400) seconds for the purpose of intervals. So yes it > is in fact a duration. Postgres and BigQuery seem to be the two SQL > systems I'm aware of that treat day components in the "calendar" sense > instead of the duration sense. > > I wonder if people would be more comfortable not calling it DayTime > Interval and instead calling it a duration. From the perspective of > arrow<->parquet interop since there is already a Nanosecond duration, > perhaps we should jump to Picoseconds and use int128 which I think would > satisfy the SQL requirements as well or allow for the fact that > Parquet's duration nanoseconds could potentially overflow Arrow's duration > seconds? > > Thanks, > Micah > > On Thu, Jul 3, 2025 at 9:22 AM Raphael Taylor-Davies > <r.taylordav...@googlemail.com.invalid> wrote: > > > Hi, > > > > I may be misunderstanding something, but it appears that the motivation > > for this effort is that the current interval type represents a superset > > of the functionality required by ANSI SQL, and therefore can cause > > compatibility problems for some databases that only support the minimum > > required by ANSI SQL? > > > > In particular the parquet Interval (along with Arrow's MonthDayNano and > > postgres' interval) encode a number of months, days and some seconds in > > a single unified interval type, whereas the SQL standard instead defines > > two interval types - one consisting of a combination of years and > > months, and another days and some seconds quantity. > > > > Whilst I can see that bifurcating the parquet interval types to match > > the minimum required by the SQL standard might help those systems with > > such bifurcated interval types, it will be a net negative for those > > systems that instead have a unified interval type. I therefore suspect > > this will end up being a repeat of the ongoing Int96 saga, especially as > > IMO having a unified interval representation is a significant UX > > improvement whilst also providing strictly more functionality. I wonder > > if instead it might be better to define some mapping that can be used by > > systems with bifurcated interval types - perhaps as a struct column > > consisting of the two components? This would avoid an interminable > > migration period, whilst also not representing a loss of functionality > > or ergonomics for existing workloads. > > > > Putting this to one side, I think the proposed DayTimeInterval still > > needs a separate day component in order to be SQL compliant, as days may > > not have 24 hours due to daylight savings time. I think what is being > > suggested might be better described as a duration type. > > > > Kind Regards, > > > > Raphael > > > > On 03/07/2025 16:01, Felipe Oliveira Carvalho wrote: > > > Hi, > > > > > > I recently took the time to understand MonthDayNano better and it's not > > as > > > crazy as my first impression of it. > > > > > > I think we could solve many of the problems you're trying to solve by > > > embracing it. Replies inline: > > > > > > On Tue, Jul 1, 2025 at 2:56 AM yun zou <yunzou.colost...@gmail.com> > > wrote: > > > > > >> Hi Team, > > >> > > >> Resending the previous email about the Interval Type discussion from > the > > >> last Parquet community sync, with some formatting adjustments. > > >> > > >> The primary focus of the conversation is the proposed INTERVAL type's > > >> *compatibility > > >> with Apache Arrow*. Several key issues have been raised: > > >> > > >> 1. *Is there a more descriptive name for DayTimeInterval?* > > >> While the name DayTimeInterval closely follows the SQL standard and > > matches > > >> naming conventions used by most engines, some suggest that a name > > >> emphasizing precision—such as *DayNanoInterval*—might provide better > > >> clarity. > > >> > > >> 2. *Should we consider representing DayTimeInterval using Arrow's > > >> MonthDayNano?* > > >> Mapping DayTimeInterval to Arrow's MonthDayNano type is problematic > due > > to > > >> semantic differences: > > >> > > >> - MonthDayNano combines both calendar-based and duration-based > > >> components, whereas DayTimeInterval represents a pure duration. > > >> > > > Month and Day components actually allow the SQL engine to delay caring > > > about calendars when parsing interval literals: > > > > > > INTERVAL '2-11 28' YEAR TO DAY > > > > > > month = 2 * 12 + 11. (there are no leap months, so converting from > years > > to > > > month is easy) > > > day = 28 > > > nanos = 0 > > > > > > If you get an interval/duration that is a single integer (as in the > > number > > > of nanoseconds), you can leave month and day set to 0. The annoying > > > limitation is that a 64-bit nanos field can't represent 10K years. How > > else > > > would you turn that weird SQL interval literal into a single integer > > > duration? Would you assume a year is 365 days? > > > > > > This is the internal representation of intervals in PostgreSQL. It > > follows > > > the same structure as MonthDayNano: a 64-bit offset and 32-bit days and > > > months. This has the nice feature of aligning nicely to 8 bytes without > > > padding and still not requiring 128-bit math for operations. > > > > > > /* > > > * Storage format for type interval. > > > */ > > > typedef struct { > > > TimeOffset time; /* all time units other than days, months and > > years */ > > > > > > int32 day; /* days, after time for alignment */ > > > int32 month; /* months and years, after time for alignment */ > > > } Interval; > > > > > > - MonthDayNano allows mixed signs across components (e.g., positive > > >> months and negative days), which complicates comparison and > > evaluation. > > >> > > > Yes, but adding more types to Arrow also complicates engines and > > > implementations. > > > > > > Adding month and days to a timestamp wouldn't be so hard if timestamps > > also > > > stored the number of months and days since an epoch, but they are > > usually a > > > single value (millis, micros, nanos) since an epoch. Couldn't an > argument > > > be made that any SQL system already needs to convert between > > single-integer > > > timestamps and YYYY-MM-DD HH:MM:SS literals, so this operation is not > > that > > > hard to support? > > > > > > Complicated expressions involving timestamps and intervals could be > > > evaluated by converting timestamps to MonthDayNano durations since > epoch > > > making every value become a duration, then at the end convert the > > duration > > > since epoch to a single-integer timestamp. Given how SQL literals are, > I > > > don't see how operations on these values can be correct without these > > > concerns being taken into account. > > > > > > > > >> Given these differences, MonthDayNano is not a suitable candidate for > > >> representing DayTimeInterval and *we recommend not mapping > > DayTimeInterval > > >> to Arrow's MonthDayNano*. > > >> > > >> 3.* Memory Footprint: Is 16 bytes necessary for DayTimeInterval? * > > >> > > >> - Some engines (e.g., Spark, Trino) represent DayTimeInterval > using > > only > > >> 8 bytes, while others (like Oracle and Snowflake) support a wider > > range, > > >> potentially requiring more than 8 bytes. Additionally, there is > > >> interest in > > >> future support for higher precision, such as picoseconds, which > > would > > >> also > > >> demand a larger footprint. > > >> - One proposal is to parameterize the size or precision, allowing > > >> engines to define their own representations. However, this > approach > > >> introduces complexity and makes standardization difficult. A > > fixed-size > > >> format that provides enough range for most use cases is considered > > more > > >> robust. > > >> - Several alternative strategies have been proposed: > > >> 1. Use a 10-byte array, which is likely sufficient for all current > > >> engine requirements. > > >> 2. Use a 16-byte array now, with the option to evolve it into a > > >> standardized int128 in the future. > > >> 3. Start with an int64 representation, and plan for a future > > transition > > >> to int128, updating related types such as timestamps and intervals > > in > > >> parallel. > > >> > > >> Looking forward to hearing your thoughts on the above questions! > > >> > > >> Link to the proposal: > > >> > > >> > > > https://docs.google.com/document/d/12ghQxWxyAhSQeZyy0IWiwJ02gTqFOgfYm8x851HZFLk/edit?tab=t.0 > > >> > > >> Link to the PR: > https://github.com/apache/parquet-format/pull/496/files > > >> > > >> Best Regards, > > >> Yun > > >> > > > So, can you expand on why MonthDayNano wouldn't work well? Yes, it's > > > complicated, but operating on Timestamps and Durations in SQL engines > is > > a > > > complicated problem and after thinking about these types in the context > > of > > > a SQL engine I think the complexity of MonthDayNano is essential, not > > > accidental. > > > > > > -- > > > Felipe > > > > > >