Skip to content

Hudi表通过spark sql添加字段后,用Hive Sql查询原有数据发生列错位问题 #12771

@liucongjy

Description

@liucongjy

Tips before filing an issue

  • Have you gone through our FAQs?

  • Join the mailing list to engage in conversations and get faster support at dev-subscribe@hudi.apache.org.

  • If you have triaged this as a bug, then file an issue directly.

Describe the problem you faced

A clear and concise description of the problem.

To Reproduce

Steps to reproduce the behavior:

  1. create table dll
    CREATE TABLE zdyj_ehr.ehr_etbj_csyxzm (
    wdsyh STRING ,
    hzjcsyh STRING ,
    jgdm STRING ,
    jgmc STRING ,
    csyxzmbh STRING ,
    xsexm STRING ,
    jlsj TIMESTAMP)
    using hudi location '/user/hive/warehouse/ws.db/ehr_etbj_csyxzm' TBLPROPERTIES(type='cow' ,primaryKey='wdsyh','hoodie.datasource.write.recordkey.field'='wdsyh' ,preCombineField='gxsj' ,'hoodie.table.partition.fields'='jlsj',
    'hoodie.table.keygenerator.class' = 'org.apache.hudi.keygen.TimestampBasedKeyGenerator',
    'hoodie.datasource.write.keygenerator.class' = 'org.apache.hudi.keygen.TimestampBasedKeyGenerator',
    'hoodie.keygen.timebased.timestamp.type' = 'DATE_STRING',
    'hoodie.keygen.timebased.timezone' = 'GMT+8:00',
    'hoodie.keygen.timebased.input.dateformat' = 'yyyy-MM-dd hh:mm:ss',
    'hoodie.keygen.timebased.output.dateformat' = 'yyyyMM',
    'hoodie.schema.on.read.enable' = 'true',
    'hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled' = 'true')
    partitioned by(jlsj);

2.using spark sql add data to ehr_etbj_csyxzm
insert into ehr_etbj_csyxzm values('t001','h001','j01','zhongyiyuan','ttt','tom',current_timestamp());

3.query data using hive shell
select * from ehr_etbj_csyxzm
jlsj field value is '2018-08-14 00:00:00'

4.using spark sql alter table add new column
alter table ehr_etbj_csyxzm add columns(ext1 string comment '扩展字段1');

5.query data using hive shell
select * from ehr_etbj_csyxzm
新加字段ext1 值为'2018-08-14 00:00:00',原有字段jlsj的值为空
正确结果应该是 字段ext1的值为空,jlsj值为'2018-08-14 00:00:00'

Image

Expected behavior

A clear and concise description of what you expected to happen.

Environment Description

  • Hudi version : 0.15

  • Spark version : 3.3.0

  • Hive version : 2.3.9

  • Hadoop version : 3.0

  • Storage (HDFS/S3/GCS..) :hdfs

  • Running on Docker? (yes/no) :no

Additional context

Add any other context about the problem here.

Stacktrace

Add the stacktrace of the error.

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:schemaSchema evolution and data typesengine:hiveHive integration

    Type

    No type

    Projects

    Status

    ✅ Done

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions