[HUDI-1415] Read Hoodie Table As Spark DataSource Table (#2283)
This commit is contained in:
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.dla;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import java.util.HashMap;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
@@ -149,14 +150,14 @@ public class DLASyncTool extends AbstractSyncTool {
|
||||
if (!useRealTimeInputFormat) {
|
||||
String inputFormatClassName = HoodieParquetInputFormat.class.getName();
|
||||
hoodieDLAClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(),
|
||||
ParquetHiveSerDe.class.getName());
|
||||
ParquetHiveSerDe.class.getName(), new HashMap<>(), new HashMap<>());
|
||||
} else {
|
||||
// Custom serde will not work with ALTER TABLE REPLACE COLUMNS
|
||||
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive
|
||||
// /ql/exec/DDLTask.java#L3488
|
||||
String inputFormatClassName = HoodieParquetRealtimeInputFormat.class.getName();
|
||||
hoodieDLAClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(),
|
||||
ParquetHiveSerDe.class.getName());
|
||||
ParquetHiveSerDe.class.getName(), new HashMap<>(), new HashMap<>());
|
||||
}
|
||||
} else {
|
||||
// Check if the table schema has evolved
|
||||
|
||||
@@ -101,9 +101,12 @@ public class HoodieDLAClient extends AbstractSyncHoodieClient {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass) {
|
||||
public void createTable(String tableName, MessageType storageSchema, String inputFormatClass,
|
||||
String outputFormatClass, String serdeClass,
|
||||
Map<String, String> serdeProperties, Map<String, String> tableProperties) {
|
||||
try {
|
||||
String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, toHiveSyncConfig(), inputFormatClass, outputFormatClass, serdeClass);
|
||||
String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, toHiveSyncConfig(),
|
||||
inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties);
|
||||
LOG.info("Creating table with " + createSQLQuery);
|
||||
updateDLASQL(createSQLQuery);
|
||||
} catch (IOException e) {
|
||||
|
||||
Reference in New Issue
Block a user