d-willis opened a new issue, #2484:
URL: https://github.com/apache/arrow-rs/issues/2484

   **Describe the bug**
   <!--
   A clear and concise description of what the bug is.
   -->
   Given a parquet file with the following nested map content:
   
   ```
    +--------------------------------------------------------+
    |map                                                     |
    +--------------------------------------------------------+
    |null                                                    |
    |null                                                    |
    |{three -> 3, four -> 4, five -> 5, six -> 6, seven -> 7}|
    +--------------------------------------------------------+
   ```
   
   Reading the keys values for the third record gives the wrong data. We expect 
`["three", "four", "five", "six", "seven"]` but get `[null, null, "three", 
"four", "five"]`.
   
   **To Reproduce**
   <!--
   Steps to reproduce the behavior:
   -->
   The following test will reproduce this:
   
   ```
   #[test]
   // This test writes a parquet file with the following data:
   // +--------------------------------------------------------+
   // |map                                                     |
   // +--------------------------------------------------------+
   // |null                                                    |
   // |null                                                    |
   // |{three -> 3, four -> 4, five -> 5, six -> 6, seven -> 7}|
   // +--------------------------------------------------------+
   //
   // It then attempts to read the data back and checks that the third record
   // contains the expected values.
   fn read_map_array_column() {
       // Make sure generated parquet file is removed whether test passes or not
       let _file_remover = FileRemover {
           filename: "read_map_column_with_leading_nulls.parquet".to_string(),
       };
   
       // Schema for single map of string to int32
       let path = Path::new("read_map_column_with_leading_nulls.parquet");
       let schema = Schema::new(vec![Field::new(
           "map",
           DataType::Map(
               Box::new(Field::new(
                   "entries",
                   DataType::Struct(vec![
                       Field::new("keys", DataType::Utf8, false),
                       Field::new("values", DataType::Int32, true),
                   ]),
                   false,
               )),
               false, // Map field not sorted
           ),
           true,
       )]);
   
       // Create builders for map
       let string_builder = StringBuilder::new(5);
       let ints_builder: PrimitiveBuilder<Int32Type> = PrimitiveBuilder::new(1);
       let mut map_builder = MapBuilder::new(None, string_builder, 
ints_builder);
   
       // Add two null records and one record with five entries
       map_builder.append(false).expect("adding null map entry");
       map_builder.append(false).expect("adding null map entry");
       map_builder.keys().append_value("three");
       map_builder.keys().append_value("four");
       map_builder.keys().append_value("five");
       map_builder.keys().append_value("six");
       map_builder.keys().append_value("seven");
   
       map_builder.values().append_value(3);
       map_builder.values().append_value(4);
       map_builder.values().append_value(5);
       map_builder.values().append_value(6);
       map_builder.values().append_value(7);
       map_builder.append(true).expect("adding map entry");
   
       // Create record batch
       let batch =
           RecordBatch::try_new(Arc::new(schema), 
vec![Arc::new(map_builder.finish())])
               .expect("create record batch");
   
       // Write record batch to file
       let props = WriterProperties::builder().build();
       let file = fs::File::create(&path).expect("create file");
       let mut writer = ArrowWriter::try_new(file, batch.schema(), Some(props))
           .expect("creat file writer");
       writer.write(&batch).expect("writing file");
       writer.close().expect("close writer");
   
       // Read file
       let file = 
fs::File::open(&Path::new("read_map_column_with_leading_nulls.parquet"))
           .expect("open file");
       let record_batch_reader = 
ParquetRecordBatchReaderBuilder::try_new(file).expect("Trying to read parquet 
file").build().expect("Getting batch iterator");
       for maybe_record_batch in record_batch_reader {
           let record_batch = maybe_record_batch.expect("Getting current 
batch");
           let col = record_batch.column(0);
           let map_entry = array::as_map_array(col).value(2);
           let struct_col = array::as_struct_array(&map_entry);
           let key_col = array::as_string_array(struct_col.column(0)); // Key 
column
           assert_eq!(key_col.value(0), "three");
           assert_eq!(key_col.value(1), "four");
           assert_eq!(key_col.value(2), "five");
           assert_eq!(key_col.value(3), "six");
           assert_eq!(key_col.value(4), "seven");
       }
       println!("finished reading");
       
fs::remove_file("read_map_column_with_leading_nulls.parquet").expect("Removing 
file");
   }
   ```
   
   
   **Expected behavior**
   <!--
   A clear and concise description of what you expected to happen.
   -->
   Reading the keys values for the third record should give`["three", "four", 
"five", "six", "seven"]`.
   
   **Additional context**
   <!--
   Add any other context about the problem here.
   -->
   Debugging the code shows that the offsets array for the MapArray's key 
column is this `[0, 0, 0, 5, 9, 13]` when I think it should be this `[0, 5, 9, 
13, 16, 21]`. It looks like the issue is in 
`parquet::arrow::array_reader::map_array::MapArrayReaderconsume_batch` where 
the repetition and definition levels are used to determine the data offsets 
(starting at parquet\src\arrow\array_reader\map_array.rs line 123).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to