[ 
https://issues.apache.org/jira/browse/PARQUET-1647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17773071#comment-17773071
 ] 

ASF GitHub Bot commented on PARQUET-1647:
-----------------------------------------

wgtmac commented on code in PR #1142:
URL: https://github.com/apache/parquet-mr/pull/1142#discussion_r1349814639


##########
parquet-hadoop/src/test/java/org/apache/parquet/statistics/TestFloat16Statistics.java:
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.statistics;
+
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.GroupFactory;
+import org.apache.parquet.example.data.simple.SimpleGroupFactory;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.Types;
+import org.apache.parquet.type.Float16;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupWriteSupport;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+
+import static org.apache.parquet.schema.LogicalTypeAnnotation.float16Type;
+import static 
org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.junit.Assert.assertEquals;
+
+public class TestFloat16Statistics {
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private short[] valuesInAscendingOrder = {
+    (short) 0xfc00, // -Infinity
+    (short) 0xc000, // -2.0
+    -Float16.MAX_VALUE, // -6.109476E-5
+    Float16.NEGATIVE_ZERO, // -0
+    Float16.POSITIVE_ZERO, // +0
+    Float16.MIN_VALUE, // 5.9604645E-8
+    Float16.MAX_VALUE, // 65504.0
+    (short) 0x7c00}; // Infinity
+
+  @Test
+  public void testFloat16ColumnIndex() throws IOException
+  {

Review Comment:
   ```suggestion
     public void testFloat16ColumnIndex() throws IOException {
   ```





> [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)

Reply via email to