Skip to content

Commit

Permalink
Removed converted type for INT32 and INT64 since they do not convert (#…
Browse files Browse the repository at this point in the history
…11627)

As brought up in #11626, converted type being written on INT32 and INT64 columns is not out of spec, but abnormal for parquet writers. This change brings cudf's writer in line with pandas and spark by not including converted type information on these types.

closes #11626

Authors:
  - Mike Wilson (https://github.com/hyperbolic2346)

Approvers:
  - Robert (Bobby) Evans (https://github.com/revans2)
  - Jim Brennan (https://github.com/jbrennan333)
  - Yunsong Wang (https://github.com/PointKernel)

URL: #11627
  • Loading branch information
hyperbolic2346 committed Aug 31, 2022
1 parent cc15765 commit 4e45256
Showing 1 changed file with 4 additions and 6 deletions.
10 changes: 4 additions & 6 deletions cpp/src/io/parquet/writer_impl.cu
Original file line number Diff line number Diff line change
Expand Up @@ -248,17 +248,15 @@ struct leaf_schema_fn {
template <typename T>
std::enable_if_t<std::is_same_v<T, int32_t>, void> operator()()
{
col_schema.type = Type::INT32;
col_schema.converted_type = ConvertedType::INT_32;
col_schema.stats_dtype = statistics_dtype::dtype_int32;
col_schema.type = Type::INT32;
col_schema.stats_dtype = statistics_dtype::dtype_int32;
}

template <typename T>
std::enable_if_t<std::is_same_v<T, int64_t>, void> operator()()
{
col_schema.type = Type::INT64;
col_schema.converted_type = ConvertedType::INT_64;
col_schema.stats_dtype = statistics_dtype::dtype_int64;
col_schema.type = Type::INT64;
col_schema.stats_dtype = statistics_dtype::dtype_int64;
}

template <typename T>
Expand Down

0 comments on commit 4e45256

Please sign in to comment.