HuaHuaY opened a new issue, #49114:
URL: https://github.com/apache/arrow/issues/49114

   ### Describe the bug, including details regarding any error messages, 
version, and platform.
   
   I provide the method for generating parquet files later. We can also add 
this code to the unit test `TestConvertParquetSchema.ParquetLists` to reproduce:
   
   ```cpp
       // Three-level encoding List<List<List<Integer>>>:
       // optional group my_list (LIST) {
       //   repeated group array (LIST) {
       //     repeated group array (LIST) {
       //       repeated int32 array;
       //     }
       //   }
       // }
       {
         auto inner_array =
             PrimitiveNode::Make("array", Repetition::REPEATED, 
ParquetType::INT32);
         auto middle_array = GroupNode::Make("array", Repetition::REPEATED, 
{inner_array},
                                             ConvertedType::LIST);
         auto outer_array = GroupNode::Make("array", Repetition::REPEATED, 
{middle_array},
                                            ConvertedType::LIST);
         parquet_fields.push_back(GroupNode::Make("my_list", 
Repetition::OPTIONAL,
                                                  {outer_array}, 
ConvertedType::LIST));
         auto arrow_inner_array = ::arrow::field("array", INT32, 
/*nullable=*/false);
         auto arrow_middle_array = ::arrow::field(
             "array", list_case.type_factory(arrow_inner_array), 
/*nullable=*/false);
         auto arrow_outer_array = ::arrow::field(
             "array", list_case.type_factory(arrow_middle_array), 
/*nullable=*/false);
         auto arrow_list = list_case.type_factory(arrow_outer_array);
         arrow_fields.push_back(::arrow::field("my_list", arrow_list, true));
       }
   ```
   
   Schema of parquet:
   
   ```thrift
   message complex_record {
     required group list_of_lists_of_lists (LIST) {
       repeated group array (LIST) {
         repeated group array (LIST) {
           repeated binary array (STRING);
         }
       }
     }
   }
   ```
   
   The parquet file can be producted by this java program:
   
   ```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.parquet.avro.AvroParquetWriter;
   import org.apache.parquet.hadoop.ParquetFileWriter;
   import org.apache.parquet.hadoop.ParquetWriter;
   import org.apache.parquet.io.LocalOutputFile;
   
   import java.io.File;
   import java.io.IOException;
   import java.nio.file.Path;
   import java.util.ArrayList;
   import java.util.HashMap;
   import java.util.List;
   import java.util.Map;
   
   public class ComplexParquetWriter {
   
     public static void main(String[] args) {
       Schema schema = new Schema.Parser().parse("{"
           + "\"type\":\"record\","
           + "\"name\":\"complex_record\","
           + "\"fields\":["
           + "  {\"name\":\"list_of_lists_of_lists\", 
\"type\":{\"type\":\"array\", \"items\":{\"type\":\"array\", 
\"items\":{\"type\":\"array\", \"items\":\"string\"}}}}"
           + "]"
           + "}");
   
       GenericRecord record = new GenericData.Record(schema);
   
       Schema listOfListsOfLists = 
schema.getField("list_of_lists_of_lists").schema();
       record.put("list_of_lists_of_lists", new 
GenericData.Array<List<List<String>>>(2, listOfListsOfLists) {
         {
           add(new ArrayList<List<String>>() {
             {
               add(new ArrayList<String>() {
                 {
                   add("deep1");
                 }
               });
             }
           });
           add(new ArrayList<List<String>>() {
             {
               add(new ArrayList<String>() {
                 {
                   add("deep2");
                   add("deep3");
                 }
               });
               add(new ArrayList<String>() {
                 {
                   add("deep4");
                 }
               });
             }
           });
         }
       });
   
       Configuration conf = new Configuration();
       Path file = new File("./complex_structure.parquet").toPath();
       LocalOutputFile outputFile = new LocalOutputFile(file);
       try {
         ParquetWriter<GenericRecord> writer = 
AvroParquetWriter.<GenericRecord>builder(outputFile)
             .withSchema(schema)
             .withConf(conf)
             .withWriteMode(ParquetFileWriter.Mode.OVERWRITE)
             .build();
         writer.write(record);
         writer.close();
         System.out.println("Successfully written to " + file.toString());
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
     }
   }
   ```
   
   ### Component(s)
   
   C++, Parquet


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