This is an automated email from the ASF dual-hosted git repository.

danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new b6bad403fa9 [HUDI-8104] Fix hive on spark select mor rt error: 
HoodieCombineRealtimeFileSplit cannot be cast to 
org.apache.hadoop.hive.shims.HadoopShimsSecure (#11799)
b6bad403fa9 is described below

commit b6bad403fa97479c44112b741007a08102617ab3
Author: FelixZh <[email protected]>
AuthorDate: Wed Aug 21 08:55:59 2024 +0800

    [HUDI-8104] Fix hive on spark select mor rt error: 
HoodieCombineRealtimeFileSplit cannot be cast to 
org.apache.hadoop.hive.shims.HadoopShimsSecure (#11799)
---
 .../java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java   | 1 +
 1 file changed, 1 insertion(+)

diff --git 
a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
 
b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
index a122f59d03b..c5ab69cde9e 100644
--- 
a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
+++ 
b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/hive/HoodieCombineHiveInputFormat.java
@@ -554,6 +554,7 @@ public class HoodieCombineHiveInputFormat<K extends 
WritableComparable, V extend
     if 
(inputFormatClass.getName().equals(getParquetRealtimeInputFormatClassName())) {
       HoodieCombineFileInputFormatShim shims = createInputFormatShim();
       IOContextMap.get(job).setInputPath(((CombineHiveInputSplit) 
split).getPath(0));
+      job.set("hudi.hive.realtime","true");
       return shims.getRecordReader(job, ((CombineHiveInputSplit) 
split).getInputSplitShim(),
           reporter, CombineHiveRecordReader.class);
     } else {

Reply via email to