hudi icon indicating copy to clipboard operation
hudi copied to clipboard

[SUPPORT] using Flink to write to Hudi in upsert mode and syncing to Hive, querying the external table in Hive gives an error:Caused by: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0.

Open Toroidals opened this issue 11 months ago • 22 comments

Tips before filing an issue

  • Have you gone through our FAQs? yes

  • Join the mailing list to engage in conversations and get faster support at [email protected].

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

Describe the problem you faced using Flink to write to Hudi in upsert mode and syncing to Hive, querying the external table in Hive gives an error: ], TaskAttempt 2 failed, info=[Error: Error while running task ( failure ) : attempt_1706375932152_272353_2_00_000201_2:java.lang.RuntimeException: java.lang.RuntimeException: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131) at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:74) at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.RuntimeException: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:199) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:68) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:419) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:267) ... 16 more Caused by: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57) at org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:420) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:196) ... 21 more Caused by: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hudi.org.apache.avro.Schema$RecordSchema.setFields(Schema.java:651) at org.apache.hudi.avro.HoodieAvroUtils.generateProjectionSchema(HoodieAvroUtils.java:534) at org.apache.hudi.hadoop.avro.HoodieAvroParquetReader.(HoodieAvroParquetReader.java:63) at org.apache.hudi.hadoop.avro.HoodieTimestampAwareParquetInputFormat.createRecordReader(HoodieTimestampAwareParquetInputFormat.java:42) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:94) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60) at org.apache.hudi.hadoop.HoodieParquetInputFormat.getRecordReaderInternal(HoodieParquetInputFormat.java:129) at org.apache.hudi.hadoop.HoodieParquetInputFormat.getRecordReader(HoodieParquetInputFormat.java:121) at org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:417) ... 22 more ], TaskAttempt 3 failed, info=[Error: Error while running task ( failure ) : attempt_1706375932152_272353_2_00_000201_3:java.lang.RuntimeException: java.lang.RuntimeException: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250) at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:381) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:82) at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:69) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1878) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:69) at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:39) at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36) at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131) at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:74) at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.RuntimeException: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:199) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(TezGroupedSplitsInputFormat.java:145) at org.apache.tez.mapreduce.lib.MRReaderMapred.next(MRReaderMapred.java:116) at org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:68) at org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:419) at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:267) ... 16 more Caused by: java.io.IOException: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97) at org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57) at org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:420) at org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:196) ... 21 more Caused by: org.apache.hudi.org.apache.avro.AvroRuntimeException: Duplicate field _hoodie_commit_time in record flink_schema: _hoodie_commit_time type:UNION pos:13 and _hoodie_commit_time type:UNION pos:0. at org.apache.hudi.org.apache.avro.Schema$RecordSchema.setFields(Schema.java:651) at org.apache.hudi.avro.HoodieAvroUtils.generateProjectionSchema(HoodieAvroUtils.java:534) at org.apache.hudi.hadoop.avro.HoodieAvroParquetReader.(HoodieAvroParquetReader.java:63) at org.apache.hudi.hadoop.avro.HoodieTimestampAwareParquetInputFormat.createRecordReader(HoodieTimestampAwareParquetInputFormat.java:42) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:94) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.(ParquetRecordReaderWrapper.java:60) at org.apache.hudi.hadoop.HoodieParquetInputFormat.getRecordReaderInternal(HoodieParquetInputFormat.java:129) at org.apache.hudi.hadoop.HoodieParquetInputFormat.getRecordReader(HoodieParquetInputFormat.java:121) at org.apache.hadoop.hive.ql.io.HiveInputFormat.getRecordReader(HiveInputFormat.java:417) ... 22 more ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:93, Vertex vertex_1706375932152_272353_2_00 [Map 1] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 (state=08S01,code=2) 1: jdbc:hive2://crpprd10hd01:21181,crpprd6hd0> Duplicate field _hoodie_commit_time in record flink_schema

To Reproduce

Steps to reproduce the behavior:

package com.hand.sink;

import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; import lombok.extern.slf4j.Slf4j; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.util.HoodiePipeline;

import java.util.ArrayList; import java.util.HashMap; import java.util.Locale; import java.util.Map;

/**

  • @Author Toroidal

  • @Date 2024/2/5 15:23

  • @Version 1.0 */ @Slf4j public class CustomHudiStreamSink {

    public static HoodiePipeline.Builder getHoodieBuilder(HashMap<String, String> infoMap, HashMap<String, String> connectInfo) {

     HoodiePipeline.Builder builder = HoodiePipeline.builder(infoMap.get("hudi_table_name"));
    
     String hudiFieldMap = infoMap.get("hudi_field_map").toLowerCase(Locale.ROOT);
     ArrayList<ArrayList<String>> fieldList = JSON.parseObject(hudiFieldMap, new TypeReference<ArrayList<ArrayList<String>>>() {
     });
     for (ArrayList<String> columnList : fieldList) {
         builder.column("`" + columnList.get(0) + "` " + columnList.get(1));
     }
     builder.column("_flink_cdc_connector string");
     builder.column("_flink_cdc_db string");
     builder.column("_flink_cdc_table string");
     builder.column("_flink_cdc_op string");
     builder.column("_flink_cdc_ts_ms timestamp");
    
     String[] hudiPrimaryKeys = infoMap.get("hudi_primary_key").split(",");
     builder.pk(hudiPrimaryKeys);
    
     options.put(FlinkOptions.TABLE_TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
    
     options.put(FlinkOptions.DATABASE_NAME.key(), infoMap.get("hudi_database_name"));
     options.put(FlinkOptions.TABLE_NAME.key(), infoMap.get("hudi_table_name"));
    
     options.put(FlinkOptions.PRE_COMBINE.key(), "true");
     options.put(FlinkOptions.PRECOMBINE_FIELD.key(), infoMap.get("hudi_precombine_field"));
    
     options.put(FlinkOptions.INDEX_TYPE.key(), HoodieIndex.IndexType.BUCKET.name());
    
     options.put(FlinkOptions.WRITE_TASKS.key(), writeTasks);
    
     options.put(FlinkOptions.BUCKET_ASSIGN_TASKS.key(), bucketAssignTasks);
    
     options.put(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.key(), infoMap.get("hudi_bucket_index_num_buckets"));
     options.put(FlinkOptions.BUCKET_INDEX_ENGINE_TYPE.key(), infoMap.get("hudi_bucket_index_engine_type"));
    
     options.put(FlinkOptions.COMPACTION_TASKS.key(), compactionTasks);
    
     options.put(FlinkOptions.COMPACTION_TRIGGER_STRATEGY.key(), infoMap.get("hudi_compaction_trigger_strategy"));
     options.put(FlinkOptions.COMPACTION_DELTA_COMMITS.key(), infoMap.get("hudi_compaction_delta_commits"));
     options.put(FlinkOptions.COMPACTION_DELTA_SECONDS.key(), infoMap.get("hudi_compaction_delta_seconds"));
    
     options.put(FlinkOptions.HIVE_SYNC_ENABLED.key(), infoMap.get("hudi_hive_sync_enabled"));
     options.put(FlinkOptions.HIVE_SYNC_MODE.key(), infoMap.get("hudi_hive_sync_mode"));
     options.put(FlinkOptions.HIVE_SYNC_DB.key(), infoMap.get("hudi_hive_sync_db"));
     options.put(FlinkOptions.HIVE_SYNC_TABLE.key(), infoMap.get("hudi_hive_sync_table"));
    
     options.put(FlinkOptions.HIVE_SYNC_CONF_DIR.key(), "/etc/hive/conf");
     options.put(FlinkOptions.HIVE_SYNC_METASTORE_URIS.key(), connectInfo.get("hive_metastore_url"));
     options.put(FlinkOptions.HIVE_SYNC_JDBC_URL.key(), connectInfo.get("conn_url"));
     options.put(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP.key(), "true");
     options.put(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX.key(), "true");
    
     options.put(FlinkOptions.PARTITION_PATH_FIELD.key(), infoMap.get("hudi_hive_sync_partition_fields"));
     options.put(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS.key(), infoMap.get("hudi_hive_sync_partition_fields"));
    
     options.put(FlinkOptions.WRI TE_CLIENT_ID.key(), infoMap.get("hudi_table_name") + System.currentTimeMillis());
    
     options.put(FlinkOptions.WRITE_RATE_LIMIT.key(), infoMap.get("hudi_write_rate_limit"));
    
     options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value());
    
     builder.options(options);
     return builder;
    

    } }

2.querying the external table in Hive

Expected behavior

Normal query synced to external table in hive

Environment Description

  • Hudi version : 0.14.0

  • Hive version :3.1.3

  • Hive version :1.15.2

  • Hadoop version :3.3.4

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

  • Running on Docker? (yes/no) :

Additional context

Add any other context about the problem here.

Stacktrace

Add the stacktrace of the error.

Toroidals avatar Feb 29 '24 03:02 Toroidals