[HUDI-1961] Add a debezium json integration test case for flink (#3030)
This commit is contained in:
@@ -144,6 +144,9 @@ public class FilePathUtils {
|
||||
boolean hivePartition,
|
||||
String[] partitionKeys) {
|
||||
LinkedHashMap<String, String> fullPartSpec = new LinkedHashMap<>();
|
||||
if (partitionKeys.length == 0) {
|
||||
return fullPartSpec;
|
||||
}
|
||||
List<String[]> kvs = new ArrayList<>();
|
||||
int curDepth = 0;
|
||||
do {
|
||||
|
||||
@@ -273,7 +273,7 @@ public class MergeOnReadInputFormat
|
||||
LinkedHashMap<String, String> partSpec = FilePathUtils.extractPartitionKeyValues(
|
||||
new org.apache.hadoop.fs.Path(path).getParent(),
|
||||
this.conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION),
|
||||
this.conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(","));
|
||||
extractPartitionKeys());
|
||||
LinkedHashMap<String, Object> partObjects = new LinkedHashMap<>();
|
||||
partSpec.forEach((k, v) -> partObjects.put(k, restorePartValueFromType(
|
||||
defaultPartName.equals(v) ? null : v,
|
||||
@@ -293,6 +293,13 @@ public class MergeOnReadInputFormat
|
||||
Long.MAX_VALUE); // read the whole file
|
||||
}
|
||||
|
||||
private String[] extractPartitionKeys() {
|
||||
if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.PARTITION_PATH_FIELD)) {
|
||||
return new String[0];
|
||||
}
|
||||
return this.conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",");
|
||||
}
|
||||
|
||||
private Iterator<RowData> getLogFileIterator(MergeOnReadInputSplit split) {
|
||||
final Schema tableSchema = new Schema.Parser().parse(tableState.getAvroSchema());
|
||||
final Schema requiredSchema = new Schema.Parser().parse(tableState.getRequiredAvroSchema());
|
||||
|
||||
Reference in New Issue
Block a user