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 4439a22 Add map_no_value.parquet (#63)
4439a22 is described below
commit 4439a223a315cf874746d3b5da25e6a6b2a2b16e
Author: Ed Seidl <[email protected]>
AuthorDate: Fri Nov 22 10:21:26 2024 -0800
Add map_no_value.parquet (#63)
* add map_no_value.parquet
* fix wording
* add map with all null values, provide code to generate
* reduce file overhead
* close code block properly
---
data/README.md | 1 +
data/map_no_value.md | 164 ++++++++++++++++++++++++++++++++++++++++++++++
data/map_no_value.parquet | Bin 0 -> 825 bytes
3 files changed, 165 insertions(+)
diff --git a/data/README.md b/data/README.md
index 7875a2f..c96f2b6 100644
--- a/data/README.md
+++ b/data/README.md
@@ -55,6 +55,7 @@
| sort_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) |
| repeated_primitive_no_list.parquet | REPEATED INT32 and BYTE_ARRAY fields
without LIST annotation. See
[note](#REPEATED-primitive-fields-with-no-LIST-annotation) |
+| map_no_value.parquet | MAP with null values, MAP with INT32 keys and no
values, and LIST<INT32> column with same values as the MAP keys. See
[map_no_value.md](map_no_value.md) |
TODO: Document what each file is in the table above.
diff --git a/data/map_no_value.md b/data/map_no_value.md
new file mode 100644
index 0000000..8e6c5a7
--- /dev/null
+++ b/data/map_no_value.md
@@ -0,0 +1,164 @@
+<!--
+ ~ 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.
+ -->
+
+`map_no_value.parquet` is generated with parquet-rs version 53.2.0
+using the following code:
+```
+ fn main() {
+ use crate::data_type::Int32Type;
+ use crate::file::properties::{EnabledStatistics, WriterProperties};
+ use crate::file::writer::SerializedFileWriter;
+ use crate::schema::parser::parse_message_type;
+ use std::sync::Arc;
+
+ let schema = "
+ message schema {
+ REQUIRED group my_map (MAP) {
+ REPEATED group key_value {
+ REQUIRED INT32 key;
+ OPTIONAL INT32 value;
+ }
+ }
+ REQUIRED group my_map_no_v (MAP) {
+ REPEATED group key_value {
+ REQUIRED INT32 key;
+ }
+ }
+ REQUIRED group my_list (LIST) {
+ REPEATED group list {
+ REQUIRED INT32 element;
+ }
+ }
+ }
+ ";
+ let schema = Arc::new(parse_message_type(schema).unwrap());
+
+ // Write Parquet file to buffer
+ let mut file =
std::fs::File::create("/tmp/map_no_value.parquet").unwrap();
+ let props = Arc::new(
+ WriterProperties::builder()
+ .set_statistics_enabled(EnabledStatistics::None)
+ .build(),
+ );
+ let mut file_writer = SerializedFileWriter::new(&mut file, schema,
props).unwrap();
+ let mut row_group_writer = file_writer.next_row_group().unwrap();
+
+ // Write column my_map.key_value.key
+ let mut column_writer =
row_group_writer.next_column().unwrap().unwrap();
+ column_writer
+ .typed::<Int32Type>()
+ .write_batch(
+ &[1, 2, 3, 4, 5, 6, 7, 8, 9],
+ Some(&[1, 1, 1, 1, 1, 1, 1, 1, 1]),
+ Some(&[0, 1, 1, 0, 1, 1, 0, 1, 1]),
+ )
+ .unwrap();
+ column_writer.close().unwrap();
+
+ // Write column my_map.key_value.value
+ let mut column_writer =
row_group_writer.next_column().unwrap().unwrap();
+ column_writer
+ .typed::<Int32Type>()
+ .write_batch(
+ &[],
+ Some(&[1, 1, 1, 1, 1, 1, 1, 1, 1]),
+ Some(&[0, 1, 1, 0, 1, 1, 0, 1, 1]),
+ )
+ .unwrap();
+ column_writer.close().unwrap();
+
+ // Write column my_map_no_v.key_value.key
+ let mut column_writer =
row_group_writer.next_column().unwrap().unwrap();
+ column_writer
+ .typed::<Int32Type>()
+ .write_batch(
+ &[1, 2, 3, 4, 5, 6, 7, 8, 9],
+ Some(&[1, 1, 1, 1, 1, 1, 1, 1, 1]),
+ Some(&[0, 1, 1, 0, 1, 1, 0, 1, 1]),
+ )
+ .unwrap();
+ column_writer.close().unwrap();
+
+ // Write column my_list.list.element
+ let mut column_writer =
row_group_writer.next_column().unwrap().unwrap();
+ column_writer
+ .typed::<Int32Type>()
+ .write_batch(
+ &[1, 2, 3, 4, 5, 6, 7, 8, 9],
+ Some(&[1, 1, 1, 1, 1, 1, 1, 1, 1]),
+ Some(&[0, 1, 1, 0, 1, 1, 0, 1, 1]),
+ )
+ .unwrap();
+ column_writer.close().unwrap();
+
+ // Finalize Parquet file
+ row_group_writer.close().unwrap();
+ file_writer.close().unwrap();
+ }
+```
+
+It contains a MAP with all null values, a second MAP without a `values` field,
and
+an equivalent LIST repeating the MAP keys. The first column is 3 MAP rows:
+```
+{1 -> null, 2 -> null, 3 -> null}
+{4 -> null, 5 -> null, 6 -> null}
+{7 -> null, 8 -> null, 9 -> null}
+```
+
+The last two columns comprise 3 equivalent rows of `list<Integer>`:
+```
+[1, 2, 3]
+[4, 5, 6]
+[7, 8, 9]
+```
+
+Here is the file metadata printed by parquet-cli:
+```
+File path: map_no_value.parquet
+Created by: parquet-rs version 53.2.0
+Properties: (none)
+Schema:
+message schema {
+ required group my_map (MAP) {
+ repeated group key_value {
+ required int32 key;
+ optional int32 value;
+ }
+ }
+ required group my_map_no_v (MAP) {
+ repeated group key_value {
+ required int32 key;
+ }
+ }
+ required group my_list (LIST) {
+ repeated group list {
+ required int32 element;
+ }
+ }
+}
+
+
+Row group 0: count: 3 105.00 B records start: 4 total(compressed): 315 B
total(uncompressed):315 B
+--------------------------------------------------------------------------------
+ type encodings count avg size nulls
min / max
+my_map.key_value.key INT32 _ RR_ 9 10.00 B
+my_map.key_value.value INT32 _ RR_ 9 5.00 B
+my_map_no_v.key_value.key INT32 _ RR_ 9 10.00 B
+my_list.list.element INT32 _ RR_ 9 10.00 B
+```
diff --git a/data/map_no_value.parquet b/data/map_no_value.parquet
new file mode 100644
index 0000000..0cf996e
Binary files /dev/null and b/data/map_no_value.parquet differ