-
Notifications
You must be signed in to change notification settings - Fork 598
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
refactor(source): enhance parquet file source #19221
Changes from 2 commits
2301fae
c86a8ce
a1d1a9b
d804208
01d102c
7c92a3f
61e79a1
69c8320
857d86d
8d033f7
c83f421
99b6846
4006d13
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -572,7 +572,6 @@ pub trait FromArrow { | |
if let Some(type_name) = field.metadata().get("ARROW:extension:name") { | ||
return self.from_extension_array(type_name, array); | ||
} | ||
|
||
match array.data_type() { | ||
Boolean => self.from_bool_array(array.as_any().downcast_ref().unwrap()), | ||
Int16 => self.from_int16_array(array.as_any().downcast_ref().unwrap()), | ||
|
@@ -584,12 +583,30 @@ pub trait FromArrow { | |
Float64 => self.from_float64_array(array.as_any().downcast_ref().unwrap()), | ||
Date32 => self.from_date32_array(array.as_any().downcast_ref().unwrap()), | ||
Time64(Microsecond) => self.from_time64us_array(array.as_any().downcast_ref().unwrap()), | ||
Timestamp(Second, None) => { | ||
self.from_timestampsecond_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Second, Some(_)) => { | ||
self.from_timestampsecond_some_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Millisecond, None) => { | ||
self.from_timestampms_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Millisecond, Some(_)) => { | ||
self.from_timestampms_some_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Microsecond, None) => { | ||
self.from_timestampus_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Microsecond, Some(_)) => { | ||
self.from_timestampus_some_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Nanosecond, None) => { | ||
self.from_timestampns_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Timestamp(Nanosecond, Some(_)) => { | ||
self.from_timestampns_some_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
Interval(MonthDayNano) => { | ||
self.from_interval_array(array.as_any().downcast_ref().unwrap()) | ||
} | ||
|
@@ -692,6 +709,33 @@ pub trait FromArrow { | |
Ok(ArrayImpl::Time(array.into())) | ||
} | ||
|
||
fn from_timestampsecond_array( | ||
&self, | ||
array: &arrow_array::TimestampSecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamp(array.into())) | ||
} | ||
fn from_timestampsecond_some_array( | ||
&self, | ||
array: &arrow_array::TimestampSecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamp(array.into())) | ||
wcy-fdu marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
fn from_timestampms_array( | ||
&self, | ||
array: &arrow_array::TimestampMillisecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamp(array.into())) | ||
} | ||
|
||
fn from_timestampms_some_array( | ||
&self, | ||
array: &arrow_array::TimestampMillisecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamptz(array.into())) | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am curious, we don't need to do any conversion between these types? All of them can be implemented in the same way? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I also don't get why we need to add different methods to convert timestamp in different units. IIUC, the current conversion logic won't lose any precision. What happened if we just add different units of the timestamp type to
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In our previous code, we only handled timestamps in microseconds, so this conversion is only correct for There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The correct approach should be as shown in the PR, where different handling is done based on the time unit. For example, for milliseconds, it should be
So I think specifying the time unit is essential. |
||
|
||
fn from_timestampus_array( | ||
&self, | ||
array: &arrow_array::TimestampMicrosecondArray, | ||
|
@@ -706,6 +750,20 @@ pub trait FromArrow { | |
Ok(ArrayImpl::Timestamptz(array.into())) | ||
} | ||
|
||
fn from_timestampns_array( | ||
&self, | ||
array: &arrow_array::TimestampNanosecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamp(array.into())) | ||
} | ||
|
||
fn from_timestampns_some_array( | ||
&self, | ||
array: &arrow_array::TimestampNanosecondArray, | ||
) -> Result<ArrayImpl, ArrayError> { | ||
Ok(ArrayImpl::Timestamptz(array.into())) | ||
} | ||
|
||
fn from_interval_array( | ||
&self, | ||
array: &arrow_array::IntervalMonthDayNanoArray, | ||
|
@@ -854,8 +912,14 @@ converts!(Utf8Array, arrow_array::StringArray); | |
converts!(Utf8Array, arrow_array::LargeStringArray); | ||
converts!(DateArray, arrow_array::Date32Array, @map); | ||
converts!(TimeArray, arrow_array::Time64MicrosecondArray, @map); | ||
converts!(TimestampArray, arrow_array::TimestampSecondArray, @map); | ||
converts!(TimestampArray, arrow_array::TimestampMillisecondArray, @map); | ||
converts!(TimestampArray, arrow_array::TimestampMicrosecondArray, @map); | ||
converts!(TimestampArray, arrow_array::TimestampNanosecondArray, @map); | ||
converts!(TimestamptzArray, arrow_array::TimestampSecondArray, @map); | ||
converts!(TimestamptzArray, arrow_array::TimestampMillisecondArray, @map); | ||
converts!(TimestamptzArray, arrow_array::TimestampMicrosecondArray, @map); | ||
converts!(TimestamptzArray, arrow_array::TimestampNanosecondArray, @map); | ||
converts!(IntervalArray, arrow_array::IntervalMonthDayNanoArray, @map); | ||
converts!(SerialArray, arrow_array::Int64Array, @map); | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not directly related to this PR but I wonder why we ignore the timezone (
Some(_)
) from arrow timestamp when constructingTimestamptz
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Introduced in #17201
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, we have encountered three issues regarding the conversion of the timestamp type:
timestamp
ortimestampz
based on whether there is a timezone (none or some).timestamp(_, none)
should match with rw’s timestamp, whiletimestamp(_, some)
should match with rw’stimestamptz
. However, previously, apart from microseconds, the other units did not match.#17201 fix 1, and this pr fix 2 and 3.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I understand. Correct me if I am wrong, I think 1 is not fixed completely because we don't use the correct timezone for timestamptz based on the arrow data type.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Discussed offline, indeed this is another bug: during the conversion, the timezone should be handled using Arrow'stimezone()
, as we are currently using the default timezone. I will fix this in the next PR. Thanks to @hzxa21 for identifying this issue.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Double checked, their is no correctness issue:
When the timezone in Arrow is
Some(_)
, the i64 always stores UTC time, which is consistent with PG/RW's timestamptz. Therefore, we only need to distinguish between None and Some(_), as the contents of Some do not affect the actual value; they are merely for display purposes.refer to https://docs.rs/arrow-schema/53.2.0/arrow_schema/enum.DataType.html#timestamps-with-a-non-empty-timezone