bhasudha edited a comment on issue #1325: presto - querying nested object in 
parquet file created by hudi
URL: https://github.com/apache/incubator-hudi/issues/1325#issuecomment-592997350
 
 
   @adamjoneill I tried to reproduce this in local docker (little modification 
to Hudi docker demo setup) and could not.  I am able to select * on both the 
schemas ( with and without the simple identifier in the array element). 
   
   I wanted to verify that presto query select * from table against hudi 
parquet file is NOT failing currently.  That is ensured. Looks like there is 
some environment issues between this setup and your setup. Lets see if we can 
dig into that more somehow.
   
   Here is the setup and  reproducing steps.
   
   I am using the hoodie docker demo setup on latest master.
   Hudi version : 0.5.2-incubating
   Spark version : 2.4.4
   Hive version : Hive 2.3.3,
   Presto 0.217
   Hadoop version : 2.8.4
   Storage (HDFS/S3/GCS..) : HDFS
   Running on Docker? (yes/no) : yes
   
   I tried for both simple and wothout simple identifier inside the array item. 
Here is the set up I used:
   
   cat nested-table-kafka-source.properties
        include=base.properties
        # Key fields, for kafka example
        hoodie.datasource.write.recordkey.field=id
        hoodie.datasource.write.partitionpath.field=date
        # Schema provider props (change to absolute path based on your 
installation)
   
hoodie.deltastreamer.schemaprovider.source.schema.file=/var/demo/config/nested_table_schema.avsc
   
hoodie.deltastreamer.schemaprovider.target.schema.file=/var/demo/config/nested_table_schema.avsc
        # Kafka Source
        hoodie.deltastreamer.source.kafka.topic=nested_table
        #Kafka props
        bootstrap.servers=kafkabroker:9092
        auto.offset.reset=earliest
   
   
   
   cat nested_table_schema.avsc
        {
          "name": "nested_table",
          "type": "record",
          "fields": [
            {
              "name": "id",
              "type": "int"
            },
            {
              "name": "date",
              "type":"string"
            },
            {
              "name": "foos",
              "type": {
                "type": "array",
                "items": {
                  "name": "foos_record",
                  "type": "record",
                  "fields": [
                    {  // remove this for without simple
                      "name": "id",
                      "type": "int"
                    },
                    {
                      "name": "bar",
                      "type": {
                        "name": "bar",
                        "type": "record",
                        "fields": [
                          {
                            "name": "id",
                            "type": "int"
                          },
                          {
                            "name": "name",
                            "type": "string"
                          }
                        ]
                      }
                    }
                  ]
                }
              }
            }
          ]
        }
   
   cat nested_table.json //remove the id entry inside foos[{elem}] for trying  
without simple.
        {"id":1,"date": 
"2020/02/15","foos":[{"id":11,"bar":{"id":1,"name":"OneBar"}},{"id":12,"bar":{"id":2,"name":"TwoBar"}},{"id":13,"bar":{"id":3,"name":"ThreeBar"}}]}
        {"id":2,"date": 
"2020/02/16","foos":[{"id":21,"bar":{"id":2,"name":"OneBar"}},{"id":22,"bar":{"id":2,"name":"TwoBar"}},{"id":23,"bar":{"id":3,"name":"ThreeBar"}}]}
        {"id":3,"date": 
"2020/02/17","foos":[{"id":31,"bar":{"id":3,"name":"OneBar"}},{"id":32,"bar":{"id":2,"name":"TwoBar"}},{"id":33,"bar":{"id":3,"name":"ThreeBar"}}]}
        {"id":4,"date": 
"2020/02/18","foos":[{"id":41,"bar":{"id":4,"name":"OneBar"}},{"id":42,"bar":{"id":2,"name":"TwoBar"}},{"id":43,"bar":{"id":3,"name":"ThreeBar"}}]}
   
   
   Steps to publish to kafka, ingest from kafka and sync with hive. these are 
in the website - https://hudi.apache.org/docs/docker_demo.html. I modified it a 
bit for this example.
   // publich data from local to kafka. cd into hudi repo dir.
   cat docker/demo/data/nested_table.json | kafkacat -b kafkabroker -t 
nested_table -P
   
   //verify kafka publish using
   kafkacat -b kafkabroker -L -J | jq .
   
   // Now hop onto one of the adhoc containers
   docker exec -it adhoc-2 /bin/bash
   
   // using deltastreamer ingest data into hdfs
   spark-submit --class 
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer 
$HUDI_UTILITIES_BUNDLE --table-type COPY_ON_WRITE --source-class 
org.apache.hudi.utilities.sources.JsonKafkaSource --source-ordering-field id  
--target-base-path /user/hive/warehouse/nested_table_cow --target-table 
nested_table_cow --props /var/demo/config/nested-table-kafka-source.properties 
--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider
   
   // verify ingest by viewing this page in web hdfs
   http://namenode:50070/explorer.html#/user/hive/warehouse/nested_table_cow
   
   // sync to hive to make it available as a hive table.
   /var/hoodie/ws/hudi-hive/run_sync_tool.sh  --jdbc-url 
jdbc:hive2://hiveserver:10000 --user hive --pass hive --partitioned-by dt 
--base-path /user/hive/warehouse/nested_table_cow --database default --table 
nested_table_cow
   
   // Now we can query from presto as below
   docker exec -it presto-worker-1 presto --server presto-coordinator-1:8090
   
   // presto query results when with simple id inside array element
   presto:default> use hive.default;
   presto:default> show columns from  nested_table_cow;
            Column         |                           Type                     
       |     Extra     | Comment
   
------------------------+-----------------------------------------------------------+---------------+---------
    _hoodie_commit_time    | varchar                                            
       |               |
    _hoodie_commit_seqno   | varchar                                            
       |               |
    _hoodie_record_key     | varchar                                            
       |               |
    _hoodie_partition_path | varchar                                            
       |               |
    _hoodie_file_name      | varchar                                            
       |               |
    id                     | integer                                            
       |               |
    date                   | varchar                                            
       |               |
    foos                   | array(row(id integer, bar row(id integer, name 
varchar))) |               |
    dt                     | varchar                                            
       | partition key |
   (9 rows)
   
   Query 20200229_203415_00003_ay9hw, FINISHED, 2 nodes
   Splits: 19 total, 19 done (100.00%)
   0:01 [9 rows, 792B] [12 rows/s, 1.11KB/s]
   
   presto:default>
   
   presto:default> select * from nested_table_cow;
    _hoodie_commit_time | _hoodie_commit_seqno | _hoodie_record_key | 
_hoodie_partition_path |                           _hoodie_file_name            
               | id |    date    |                                              
     foos
   
---------------------+----------------------+--------------------+------------------------+-----------------------------------------------------------------------+----+------------+-------------------------------------------------------
    20200229201846      | 20200229201846_0_3   | 3                  | 
2020/02/17             | 
08524ae2-9874-4f52-8b8e-97a509696ee6-0_0-21-24_20200229201846.parquet |  3 | 
2020/02/17 | [{id=31, bar={id=3, name=OneBar}}, {id=32, bar={id=2,
    20200229201846      | 20200229201846_3_4   | 2                  | 
2020/02/16             | 
3ed41232-08a9-41ff-b97c-7b964c8d7978-0_3-21-27_20200229201846.parquet |  2 | 
2020/02/16 | [{id=21, bar={id=2, name=OneBar}}, {id=22, bar={id=2,
    20200229201846      | 20200229201846_1_1   | 4                  | 
2020/02/18             | 
85fa663a-c86f-43f7-94a5-a813a0a67314-0_1-21-25_20200229201846.parquet |  4 | 
2020/02/18 | [{id=41, bar={id=4, name=OneBar}}, {id=42, bar={id=2,
    20200229201846      | 20200229201846_2_2   | 1                  | 
2020/02/15             | 
916d9d23-da62-4d6b-bb9d-85889e8c6ab8-0_2-21-26_20200229201846.parquet |  1 | 
2020/02/15 | [{id=11, bar={id=1, name=OneBar}}, {id=12, bar={id=2,
   (4 rows)
   
   Query 20200229_204101_00005_ay9hw, FINISHED, 1 node
   Splits: 20 total, 20 done (100.00%)
   0:05 [4 rows, 5KB] [0 rows/s, 1.02KB/s]
   
   presto:default>
   
   
   // presto query results when removing the simple identifier
   presto:default> show columns from  nested_table_cow;
            Column         |                     Type                      |    
 Extra     | Comment
   
------------------------+-----------------------------------------------+---------------+---------
    _hoodie_commit_time    | varchar                                       |    
           |
    _hoodie_commit_seqno   | varchar                                       |    
           |
    _hoodie_record_key     | varchar                                       |    
           |
    _hoodie_partition_path | varchar                                       |    
           |
    _hoodie_file_name      | varchar                                       |    
           |
    id                     | integer                                       |    
           |
    date                   | varchar                                       |    
           |
    foos                   | array(row(bar row(id integer, name varchar))) |    
           |
    dt                     | varchar                                       | 
partition key |
   (9 rows)
   
   Query 20200229_205313_00002_xjxz7, FINISHED, 2 nodes
   Splits: 19 total, 19 done (100.00%)
   0:02 [9 rows, 780B] [4 rows/s, 387B/s]
   
   presto:default> select * from nested_table_cow;
    _hoodie_commit_time | _hoodie_commit_seqno | _hoodie_record_key | 
_hoodie_partition_path |                           _hoodie_file_name            
               | id |    date    |                                        foos
   
---------------------+----------------------+--------------------+------------------------+-----------------------------------------------------------------------+----+------------+-------------------------------------------------------
    20200229204957      | 20200229204957_2_1   | 1                  | 
2020/02/15             | 
fb63e6c8-bd12-442b-bdc3-26748f33a4a8-0_2-21-26_20200229204957.parquet |  1 | 
2020/02/15 | [{bar={id=1, name=OneBar}}, {bar={id=2, name=TwoBar}},
    20200229204957      | 20200229204957_0_2   | 3                  | 
2020/02/17             | 
3722865d-7559-4b1c-94e2-6bf993c4d4a0-0_0-21-24_20200229204957.parquet |  3 | 
2020/02/17 | [{bar={id=3, name=OneBar}}, {bar={id=2, name=TwoBar}},
    20200229204957      | 20200229204957_1_3   | 4                  | 
2020/02/18             | 
6e70cd2b-d922-425d-b94b-1f35691758b2-0_1-21-25_20200229204957.parquet |  4 | 
2020/02/18 | [{bar={id=4, name=OneBar}}, {bar={id=2, name=TwoBar}},
    20200229204957      | 20200229204957_3_4   | 2                  | 
2020/02/16             | 
f2a71bf6-ee01-4310-a8f3-2c4b1b1ab34e-0_3-21-27_20200229204957.parquet |  2 | 
2020/02/16 | [{bar={id=2, name=OneBar}}, {bar={id=2, name=TwoBar}},
   (4 rows)
   
   Query 20200229_205341_00003_xjxz7, FINISHED, 1 node
   Splits: 20 total, 20 done (100.00%)
   0:23 [4 rows, 4.68KB] [0 rows/s, 212B/s]
   
   presto:default>
   
   
   
   Can you confirm if the Presto "show column from table" schema matches mine 
especially for the case where simple id is not present in the array element 
when you are querying from hudi parquet file?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to