[
https://issues.apache.org/jira/browse/PARQUET-1647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17779323#comment-17779323
]
ASF GitHub Bot commented on PARQUET-1647:
-----------------------------------------
zhangjiashen commented on code in PR #1142:
URL: https://github.com/apache/parquet-mr/pull/1142#discussion_r1371132290
##########
parquet-column/src/main/java/org/apache/parquet/column/statistics/Statistics.java:
##########
@@ -150,26 +150,26 @@ public Float16Builder(PrimitiveType type) {
@Override
public Statistics<?> build() {
- Float16Statistics stats = (Float16Statistics) super.build();
+ BinaryStatistics stats = (BinaryStatistics) super.build();
if (stats.hasNonNullValue()) {
Binary bMin = stats.genericGetMin();
Binary bMax = stats.genericGetMax();
short min = bMin.get2BytesLittleEndian();
short max = bMax.get2BytesLittleEndian();
// Drop min/max values in case of NaN as the sorting order of values
is undefined for this case
if (Float16.isNaN(min) || Float16.isNaN(max)) {
- bMin =
Binary.fromConstantByteArray(Float16.POSITIVE_ZERO_BYTES_LITTLE_ENDIAN);
- bMax =
Binary.fromConstantByteArray(Float16.POSITIVE_ZERO_BYTES_LITTLE_ENDIAN);
+ bMin = Binary.fromConstantByteArray(new byte[] {0x00, 0x00});
+ bMax = Binary.fromConstantByteArray(new byte[] {0x00, (byte) 0x80});
Review Comment:
updated, thanks!
> [Java] support for Arrow's float16
> ----------------------------------
>
> Key: PARQUET-1647
> URL: https://issues.apache.org/jira/browse/PARQUET-1647
> Project: Parquet
> Issue Type: Improvement
> Components: parquet-format, parquet-thrift
> Reporter: The Alchemist
> Priority: Minor
>
> h2. DESCRIPTION
>
> I'm wondering if there's any interest in supporting Arrow's {{float16}} type
> in Parquet.
> There seem to be one or two {{float16}} / {{halffloat}} tickets here (e.g.,
> PARQUET-1403) but nothing that speaks to adding half-float support to Parquet
> in-general.
>
> h2. PLANS
> I'm able to spend some time on this, if someone points me in the right
> direction.
>
> # Add the {{HALFFLOAT}} or {{FLOAT16}} enum (any preferred naming
> convention?) to
> [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L32]
> # Add {{HALFFLOAT}} to {{org.apache.parquet.schema.PrimitiveType}}
> # Add {{HALFFLOAT}} support to
> {{org.apache.parquet.arrow.schema.SchemaConverter}}
> # Add encoding for new type at {{org.apache.parquet.column.Encoding}}
> # ??
> If anyone has any interest in this, pointers, or comments, they would be
> greatly appreciated!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)