2424import org .apache .hudi .exception .HoodieException ;
2525import org .apache .hudi .common .table .TableSchemaResolver ;
2626import org .apache .hudi .hadoop .utils .HoodieRealtimeRecordReaderUtils ;
27+ import org .apache .hudi .hadoop .utils .HiveAvroSerializer ;
2728
2829import org .apache .avro .Schema ;
2930import org .apache .avro .Schema .Field ;
3031import org .apache .hadoop .hive .metastore .api .hive_metastoreConstants ;
3132import org .apache .hadoop .hive .serde2 .ColumnProjectionUtils ;
33+ import org .apache .hadoop .hive .ql .io .parquet .serde .ArrayWritableObjectInspector ;
34+ import org .apache .hadoop .hive .serde .serdeConstants ;
35+ import org .apache .hadoop .hive .serde2 .typeinfo .StructTypeInfo ;
36+ import org .apache .hadoop .hive .serde2 .typeinfo .TypeInfo ;
37+ import org .apache .hadoop .hive .serde2 .typeinfo .TypeInfoFactory ;
38+ import org .apache .hadoop .hive .serde2 .typeinfo .TypeInfoUtils ;
3239import org .apache .hadoop .mapred .JobConf ;
3340import org .apache .log4j .LogManager ;
3441import org .apache .log4j .Logger ;
@@ -55,13 +62,19 @@ public abstract class AbstractRealtimeRecordReader {
5562 private Schema writerSchema ;
5663 private Schema hiveSchema ;
5764 private HoodieTableMetaClient metaClient ;
65+ // support merge operation
66+ protected boolean supportPayload = true ;
67+ // handle hive type to avro record
68+ protected HiveAvroSerializer serializer ;
5869
5970 public AbstractRealtimeRecordReader (RealtimeSplit split , JobConf job ) {
6071 this .split = split ;
6172 this .jobConf = job ;
6273 LOG .info ("cfg ==> " + job .get (ColumnProjectionUtils .READ_COLUMN_NAMES_CONF_STR ));
6374 LOG .info ("columnIds ==> " + job .get (ColumnProjectionUtils .READ_COLUMN_IDS_CONF_STR ));
6475 LOG .info ("partitioningColumns ==> " + job .get (hive_metastoreConstants .META_TABLE_PARTITION_COLUMNS , "" ));
76+ this .supportPayload = Boolean .parseBoolean (job .get ("hoodie.support.payload" , "true" ));
77+ prepareHiveAvroSerializer ();
6578 try {
6679 metaClient = HoodieTableMetaClient .builder ().setConf (jobConf ).setBasePath (split .getBasePath ()).build ();
6780 if (metaClient .getTableConfig ().getPreCombineField () != null ) {
@@ -80,6 +93,26 @@ private boolean usesCustomPayload(HoodieTableMetaClient metaClient) {
8093 || metaClient .getTableConfig ().getPayloadClass ().contains ("org.apache.hudi.OverwriteWithLatestAvroPayload" ));
8194 }
8295
96+ private void prepareHiveAvroSerializer () {
97+ try {
98+ List <String > hiveInternalColumns = Arrays .asList (new String [] {"BLOCK__OFFSET__INSIDE__FILE" , "INPUT__FILE__NAME" , "ROW__ID" });
99+ List <String > columnNameList = Arrays .stream (jobConf .get (serdeConstants .LIST_COLUMNS ).split ("," )).collect (Collectors .toList ());
100+ int dropNum = columnNameList .stream ().filter (f -> hiveInternalColumns .contains (f )).map (f -> columnNameList .indexOf (f )).collect (Collectors .toList ()).size ();
101+ List <TypeInfo > columnTypeList = TypeInfoUtils .getTypeInfosFromTypeString (jobConf .get (serdeConstants .LIST_COLUMN_TYPES ));
102+ for (int i = 0 ; i < dropNum ; i ++) {
103+ columnNameList .remove (columnNameList .size () - 1 );
104+ columnTypeList .remove (columnTypeList .size () - 1 );
105+ }
106+ StructTypeInfo rowTypeInfo = (StructTypeInfo ) TypeInfoFactory .getStructTypeInfo (columnNameList , columnTypeList );
107+ this .serializer = new HiveAvroSerializer (new ArrayWritableObjectInspector (rowTypeInfo ), columnNameList , columnTypeList );
108+ } catch (Exception e ) {
109+ // fallback to origin logical
110+ LOG .warn ("fall to init HiveAvroSerializer to support payload merge" , e );
111+ this .supportPayload = false ;
112+ }
113+
114+ }
115+
83116 /**
84117 * Gets schema from HoodieTableMetaClient. If not, falls
85118 * back to the schema from the latest parquet file. Finally, sets the partition column and projection fields into the
0 commit comments