1
0

[HUDI-1415] Read Hoodie Table As Spark DataSource Table (#2283)

This commit is contained in:
pengzhiwei
2021-04-21 05:21:38 +08:00
committed by GitHub
parent 3253079507
commit aacb8be521
13 changed files with 382 additions and 48 deletions

View File

@@ -62,8 +62,20 @@ public abstract class AbstractSyncHoodieClient {
this.fs = fs;
}
/**
* Create the table.
* @param tableName The table name.
* @param storageSchema The table schema.
* @param inputFormatClass The input format class of this table.
* @param outputFormatClass The output format class of this table.
* @param serdeClass The serde class of this table.
* @param serdeProperties The serde properites of this table.
* @param tableProperties The table properties for this table.
*/
public abstract void createTable(String tableName, MessageType storageSchema,
String inputFormatClass, String outputFormatClass, String serdeClass);
String inputFormatClass, String outputFormatClass,
String serdeClass, Map<String, String> serdeProperties,
Map<String, String> tableProperties);
public abstract boolean doesTableExist(String tableName);
@@ -75,6 +87,8 @@ public abstract class AbstractSyncHoodieClient {
public abstract void updatePartitionsToTable(String tableName, List<String> changedPartitions);
public void updateTableProperties(String tableName, Map<String, String> tableProperties) {}
public abstract Map<String, String> getTableSchema(String tableName);
public HoodieTableType getTableType() {