1
0

HUDI-243 Rename HoodieInputFormat and HoodieRealtimeInputFormat to HoodieParquetInputFormat and HoodieParquetRealtimeInputFormat

This commit is contained in:
Balaji Varadarajan
2019-09-11 11:31:07 -07:00
committed by vinoth chandar
parent d0b9b56b7d
commit 93bc5e2153
10 changed files with 31 additions and 27 deletions

View File

@@ -31,8 +31,8 @@ import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.hadoop.HoodieInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import org.apache.hudi.hive.util.SchemaUtil;
@@ -128,7 +128,7 @@ public class HiveSyncTool {
// for now)
String inputFormatClassName =
cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.HoodieInputFormat.class.getName()
: HoodieInputFormat.class.getName();
: HoodieParquetInputFormat.class.getName();
hoodieHiveClient.createTable(schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
} else {
@@ -137,7 +137,7 @@ public class HiveSyncTool {
// /ql/exec/DDLTask.java#L3488
String inputFormatClassName =
cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName()
: HoodieRealtimeInputFormat.class.getName();
: HoodieParquetRealtimeInputFormat.class.getName();
hoodieHiveClient.createTable(schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
}