Hi Kaka,
Yes, this is expected as the table is non-partitioned. The 0.4.5 release which
happened yesterday has the fix you referenced.
Thanks,Balaji.V
On Tuesday, February 26, 2019, 6:37:59 PM PST, kaka chen
<[email protected]> wrote:
BTW, because it cannot get partition field, after I merged with
https://github.com/uber/hudi/pull/569/files, the job can run successfully.
Thanks,
Frank
kaka chen <[email protected]> 于2019年2月27日周三 上午10:34写道:
>
> I have tried two environments(Hive 2.1.1 and Hive 1.1.0-cdh5.15.1) both
> cannot get the partition field.
>
> And I added simple logs to show the result:
>
> LOG.info("schema: " + schema + " partitioningFields: " + partitioningFields);
>
> 2019-02-26 19:53:47,855 INFO [main]
> com.uber.hoodie.hadoop.realtime.AbstractRealtimeRecordReader: schema:
> {"type":"record","name":"test_record","namespace":"hoodie.test","fields":[{"name":"_hoodie_commit_time","type":["null","string"],"default":null},{"name":"_hoodie_commit_seqno","type":["null","string"],"default":null},{"name":"_hoodie_record_key","type":["null","string"],"default":null},{"name":"_hoodie_partition_path","type":["null","string"],"default":null},{"name":"_hoodie_file_name","type":["null","string"],"default":null},{"name":"id","type":["null","int"],"default":null},{"name":"user_id","type":["null","int"],"default":null},{"name":"dashboard_id","type":["null","int"],"default":null},{"name":"created_at","type":["null","string"],"default":null},{"name":"updated_at","type":["null","string"],"default":null},{"name":"timestamp","type":["null","long"],"default":null},{"name":"eventType","type":["null","string"],"default":null},{"name":"par","type":["null","string"],"default":null}]}
> partitioningFields: []
>
>
>
>
> desc formatted dev.statistics_dashboard_visitor_hudi_rt
>
>
> Hive 2.1.1:
>
>
> col_name, data_type, comment # col_name , data_type , comment , ,
> _hoodie_commit_time, string, _hoodie_commit_seqno, string,
> _hoodie_record_key, string, _hoodie_partition_path, string,
> _hoodie_file_name, string, id, int, user_id, int, dashboard_id, int,
> created_at, string, updated_at, string, timestamp, bigint, eventtype,
> string, , , # Partition Information, , # col_name , data_type , comment , ,
> par, string, , , # Detailed Table Information, , Database: , dev , Owner: ,
> app , CreateTime: , Mon Feb 25 12:07:34 CST 2019, LastAccessTime: , UNKNOWN
> , Retention: , 0 , Location: ,
> hdfs://yz-cluster-qa/user/hive/warehouse/dev.db/statistics_dashboard_visitor_hudi,
> Table Type: , EXTERNAL_TABLE , Table Parameters:, , , EXTERNAL , TRUE ,
> spark.sql.sources.schema.numPartCols, 1 ,
> spark.sql.sources.schema.numParts, 1 , spark.sql.sources.schema.part.0,
> {\"type\":\"struct\",\"fields\":[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"int\"}},{\"name\":\"user_id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"int\"}},{\"name\":\"dashboard_id\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"int\"}},{\"name\":\"created_at\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"updated_at\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"timestamp\",\"type\":\"long\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"bigint\"}},{\"name\":\"eventType\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"comment\":\"\",\"HIVE_TYPE_STRING\":\"string\"}},{\"name\":\"par\",\"type\":\"string\",\"nullable\":true,\"metadata\":{\"HIVE_TYPE_STRING\":\"string\"}}]}
> , spark.sql.sources.schema.partCol.0, par , transient_lastDdlTime,
> 1551074880 , , # Storage Information, , SerDe Library: ,
> com.uber.hoodie.hadoop.realtime.HoodieParquetSerde, InputFormat: ,
> com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat, OutputFormat: ,
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat, Compressed:
> , No , Num Buckets: , -1 , Bucket Columns: , [] , Sort Columns: , [] ,
> Storage Desc Params:, , , serialization.format, 1
>
>
>
> Hive 1.1.0-cdh5.15.1
>
>
> -------------------------------+----------------------------------------------------+-----------------------+--+
>
> | col_name | data_type
> | comment |
>
>
> +-------------------------------+----------------------------------------------------+-----------------------+--+
>
> | # col_name | data_type
> | comment |
>
> | | NULL
> | NULL |
>
> | _hoodie_commit_time | string
> | |
>
> | _hoodie_commit_seqno | string
> | |
>
> | _hoodie_record_key | string
> | |
>
> | _hoodie_partition_path | string
> | |
>
> | _hoodie_file_name | string
> | |
>
> | id | int
> | |
>
> | user_id | int
> | |
>
> | dashboard_id | int
> | |
>
> | created_at | string
> | |
>
> | updated_at | string
> | |
>
> | timestamp | bigint
> | |
>
> | eventtype | string
> | |
>
> | | NULL
> | NULL |
>
> | # Partition Information | NULL
> | NULL |
>
> | # col_name | data_type
> | comment |
>
> | | NULL
> | NULL |
>
> | par | string
> | |
>
> | | NULL
> | NULL |
>
> | # Detailed Table Information | NULL
> | NULL |
>
> | Database: | dev
> | NULL |
>
> | Owner: | hive
> | NULL |
>
> | CreateTime: | Tue Feb 26 00:05:25 CST 2019
> | NULL |
>
> | LastAccessTime: | UNKNOWN
> | NULL |
>
> | Protect Mode: | None
> | NULL |
>
> | Retention: | 0
> | NULL |
>
> | Location: |
> hdfs://qabb-perf-alluxio-hadoop0:8020/user/hive/warehouse/dev.db/statistics_dashboard_visitor_hudi
> | NULL |
>
> | Table Type: | EXTERNAL_TABLE
> | NULL |
>
> | Table Parameters: | NULL
> | NULL |
>
> | | EXTERNAL
> | TRUE |
>
> | | numPartitions
> | 1 |
>
> | | transient_lastDdlTime
> | 1551110725 |
>
> | | NULL
> | NULL |
>
> | # Storage Information | NULL
> | NULL |
>
> | SerDe Library: |
> com.uber.hoodie.hadoop.realtime.HoodieParquetSerde | NULL |
>
> | InputFormat: |
> com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat | NULL
> |
>
> | OutputFormat: |
> org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat | NULL
> |
>
> | Compressed: | No
> | NULL |
>
> | Num Buckets: | -1
> | NULL |
>
> | Bucket Columns: | []
> | NULL |
>
> | Sort Columns: | []
> | NULL |
>
> | Storage Desc Params: | NULL
> | NULL |
>
> | | serialization.format
> | 1 |
>
>
> +-------------------------------+----------------------------------------------------+-----------------------+--+
>
> Thanks,
> Frank
>
> [email protected] <[email protected]> 于2019年2月27日周三 上午6:15写道:
>
>> Hi Frank,
>> As Vinoth mentioned, can you share your environment (especially
>> Hive/Spark version). Also, Can you paste the table definition as seen in
>> Hive metastore ( desc formatted <table_name> )
>>
>> Balaji.V
>> On Tuesday, February 26, 2019, 11:10:16 AM PST, Vinoth Chandar <
>> [email protected]> wrote:
>>
>> Hi,
>>
>> Can you share more details about your environment and the full stack
>> trace?
>>
>> Thanks
>> Vinoth
>>
>> On Mon, Feb 25, 2019 at 11:10 PM kaka chen <[email protected]> wrote:
>>
>> > Hi All,
>> >
>> > AbstractRealtimeRecordReader cannot get the partition field from the
>> > hive partition table by
>> > String partitionFields = jobConf.get("partition_columns", "");
>> >
>> > Thanks,
>> > Frank
>> >
>>
>
>