This is an automated email from the ASF dual-hosted git repository.
maplefu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-testing.git
The following commit(s) were added to refs/heads/master by this push:
new a7f1d28 Add file with nesting two-level list structure (#60)
a7f1d28 is described below
commit a7f1d288e693dbb08e3199851c4eb2140ff8dff2
Author: Gang Wu <[email protected]>
AuthorDate: Thu Oct 24 12:48:00 2024 +0800
Add file with nesting two-level list structure (#60)
* Add file with nesting two-level list structure
* add description to md
---
data/README.md | 2 +
data/old_list_structure.md | 103 ++++++++++++++++++++++++++++++++++++++++
data/old_list_structure.parquet | Bin 0 -> 539 bytes
3 files changed, 105 insertions(+)
diff --git a/data/README.md b/data/README.md
index b5003c2..2c93965 100644
--- a/data/README.md
+++ b/data/README.md
@@ -53,6 +53,7 @@
| incorrect_map_schema.parquet | Contains a Map schema without explicitly
required keys, produced by Presto. See [note](#incorrect-map-schema) |
| column_chunk_key_value_metadata.parquet | two INT32 columns, one with column
chunk key-value metadata {"foo": "bar", "thisiskeywithoutvalue": null} note
that the second key "thisiskeywithoutvalue", does not have a value, but the
value can be mapped to an empty string "" when read depending on the client |
| sorting_columns.parquet | INT64 and BYTE_ARRAY columns with first column
with nulls first and descending, second column with nulls last and ascending.
This file contains two row groups with same data and sorting columns. |
+| old_list_structure.parquet | Single LIST<LIST<INT32>> column with legacy
two-level list structure. See [old_list_structure.md](old_list_structure.md) |
TODO: Document what each file is in the table above.
@@ -143,6 +144,7 @@ The schema for the `*-dict-*-checksum.parquet` test files
is:
[ Column "long_field" [ Dict Page [bad crc] | Uncompressed PLAIN Contents ][
Page 0 [correct crc] | Uncompressed RLE_DICTIONARY Contents ]]
[ Column "binary_field" [ Dict Page [bad crc] | Uncompressed PLAIN Contents
][ Page 0 [correct crc] | Uncompressed RLE_DICTIONARY Contents ]]
```
+
## Bloom Filter Files
Bloom filter examples have been generated by parquet-mr.
diff --git a/data/old_list_structure.md b/data/old_list_structure.md
new file mode 100644
index 0000000..2eb4639
--- /dev/null
+++ b/data/old_list_structure.md
@@ -0,0 +1,103 @@
+<!--
+ ~ 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.
+ -->
+
+`old_list_structure.parquet` is generated with parquet-java version 1.14.3.
+It contains a `LIST<LIST<INT32>>` column with a single value `[[1, 2], [3, 4]]`
+using the legacy two-level structure encoding for list type.
+
+The file is created by the following Java code:
+```java
+package org.example;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.avro.AvroParquetWriter;
+import org.apache.parquet.hadoop.ParquetWriter;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public class TwoLevelList {
+
+ public static void main(String[] args) {
+ Schema schema = new Schema.Parser().parse("{"
+ + "\"type\":\"record\","
+ + "\"name\":\"my_record\","
+ + "\"fields\":["
+ + " {"
+ + " \"name\":\"a\","
+ + " \"type\":{\"type\":\"array\", \"items\":{\"type\":\"array\",
\"items\":\"int\"}}"
+ + " }"
+ + "]"
+ + "}");
+
+ GenericRecord record = new GenericData.Record(schema);
+
+ // Write [[1, 2], [3, 4]] to the avro record
+ record.put("a", Stream.of(Arrays.asList(1, 2), Arrays.asList(3, 4))
+ .map(list -> {
+ Schema innerListType = schema.getField("a").schema().getElementType();
+ GenericData.Array<Integer> innerList = new
GenericData.Array<>(list.size(), innerListType);
+ innerList.addAll(list);
+ return innerList;
+ }).collect(Collectors.toList()));
+
+ Path file = new Path("/tmp/old_list_structure.parquet");
+ Configuration conf = new Configuration();
+ conf.set("parquet.avro.write-old-list-structure", "true"); // this is the
default value
+ try (ParquetWriter<GenericRecord> writer =
AvroParquetWriter.<GenericRecord>builder(file)
+ .withSchema(schema)
+ .withConf(conf)
+ .build()) {
+ writer.write(record);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+}
+```
+
+Here is the file metadata printed by parquet-cli:
+```
+File path: /tmp/old_list_structure.parquet
+Created by: parquet-mr version 1.14.3 (build
b5e376a2caee767a11e75b783512b14cf8ca90ec)
+Properties:
+ parquet.avro.schema:
{"type":"record","name":"my_record","fields":[{"name":"a","type":{"type":"array","items":{"type":"array","items":"int"}}}]}
+ writer.model.name: avro
+Schema:
+message my_record {
+ required group a (LIST) {
+ repeated group array (LIST) {
+ repeated int32 array;
+ }
+ }
+}
+
+
+Row group 0: count: 1 53.00 B records start: 4 total(compressed): 53 B
total(uncompressed):53 B
+--------------------------------------------------------------------------------
+ type encodings count avg size nulls min / max
+a.array.array INT32 _ _ 4 13.25 B 0 "1" / "4"
+```
diff --git a/data/old_list_structure.parquet b/data/old_list_structure.parquet
new file mode 100644
index 0000000..7dcf1bc
Binary files /dev/null and b/data/old_list_structure.parquet differ