[HUDI-2371] Improvement flink streaming reader (#3552)
- Support reading empty table - Fix filtering by partition path - Support reading from earliest commit
This commit is contained in:
@@ -196,6 +196,7 @@ public class FlinkOptions extends HoodieConfig {
|
|||||||
.defaultValue(60)// default 1 minute
|
.defaultValue(60)// default 1 minute
|
||||||
.withDescription("Check interval for streaming read of SECOND, default 1 minute");
|
.withDescription("Check interval for streaming read of SECOND, default 1 minute");
|
||||||
|
|
||||||
|
public static final String START_COMMIT_EARLIEST = "earliest";
|
||||||
public static final ConfigOption<String> READ_STREAMING_START_COMMIT = ConfigOptions
|
public static final ConfigOption<String> READ_STREAMING_START_COMMIT = ConfigOptions
|
||||||
.key("read.streaming.start-commit")
|
.key("read.streaming.start-commit")
|
||||||
.stringType()
|
.stringType()
|
||||||
|
|||||||
@@ -48,6 +48,8 @@ import org.apache.hadoop.fs.FileStatus;
|
|||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@@ -105,20 +107,23 @@ public class StreamReadMonitoringFunction
|
|||||||
|
|
||||||
private transient org.apache.hadoop.conf.Configuration hadoopConf;
|
private transient org.apache.hadoop.conf.Configuration hadoopConf;
|
||||||
|
|
||||||
private final HoodieTableMetaClient metaClient;
|
private HoodieTableMetaClient metaClient;
|
||||||
|
|
||||||
private final long maxCompactionMemoryInBytes;
|
private final long maxCompactionMemoryInBytes;
|
||||||
|
|
||||||
|
// for partition pruning
|
||||||
|
private final Set<String> requiredPartitionPaths;
|
||||||
|
|
||||||
public StreamReadMonitoringFunction(
|
public StreamReadMonitoringFunction(
|
||||||
Configuration conf,
|
Configuration conf,
|
||||||
Path path,
|
Path path,
|
||||||
HoodieTableMetaClient metaClient,
|
long maxCompactionMemoryInBytes,
|
||||||
long maxCompactionMemoryInBytes) {
|
Set<String> requiredPartitionPaths) {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.path = path;
|
this.path = path;
|
||||||
this.metaClient = metaClient;
|
|
||||||
this.interval = conf.getInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL);
|
this.interval = conf.getInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL);
|
||||||
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
|
this.maxCompactionMemoryInBytes = maxCompactionMemoryInBytes;
|
||||||
|
this.requiredPartitionPaths = requiredPartitionPaths;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -180,8 +185,26 @@ public class StreamReadMonitoringFunction
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
private HoodieTableMetaClient getOrCreateMetaClient() {
|
||||||
|
if (this.metaClient != null) {
|
||||||
|
return this.metaClient;
|
||||||
|
}
|
||||||
|
if (StreamerUtil.tableExists(this.path.toString(), hadoopConf)) {
|
||||||
|
this.metaClient = StreamerUtil.createMetaClient(this.path.toString(), hadoopConf);
|
||||||
|
return this.metaClient;
|
||||||
|
}
|
||||||
|
// fallback
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public void monitorDirAndForwardSplits(SourceContext<MergeOnReadInputSplit> context) {
|
public void monitorDirAndForwardSplits(SourceContext<MergeOnReadInputSplit> context) {
|
||||||
|
HoodieTableMetaClient metaClient = getOrCreateMetaClient();
|
||||||
|
if (metaClient == null) {
|
||||||
|
// table does not exist
|
||||||
|
return;
|
||||||
|
}
|
||||||
metaClient.reloadActiveTimeline();
|
metaClient.reloadActiveTimeline();
|
||||||
HoodieTimeline commitTimeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants();
|
HoodieTimeline commitTimeline = metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants();
|
||||||
if (commitTimeline.empty()) {
|
if (commitTimeline.empty()) {
|
||||||
@@ -200,8 +223,9 @@ public class StreamReadMonitoringFunction
|
|||||||
} else if (this.conf.getOptional(FlinkOptions.READ_STREAMING_START_COMMIT).isPresent()) {
|
} else if (this.conf.getOptional(FlinkOptions.READ_STREAMING_START_COMMIT).isPresent()) {
|
||||||
// first time consume and has a start commit
|
// first time consume and has a start commit
|
||||||
final String specifiedStart = this.conf.getString(FlinkOptions.READ_STREAMING_START_COMMIT);
|
final String specifiedStart = this.conf.getString(FlinkOptions.READ_STREAMING_START_COMMIT);
|
||||||
instantRange = InstantRange.getInstance(specifiedStart, instantToIssue.getTimestamp(),
|
instantRange = specifiedStart.equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)
|
||||||
InstantRange.RangeType.CLOSE_CLOSE);
|
? null
|
||||||
|
: InstantRange.getInstance(specifiedStart, instantToIssue.getTimestamp(), InstantRange.RangeType.CLOSE_CLOSE);
|
||||||
} else {
|
} else {
|
||||||
// first time consume and no start commit, consumes the latest incremental data set.
|
// first time consume and no start commit, consumes the latest incremental data set.
|
||||||
HoodieInstant latestCommitInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
HoodieInstant latestCommitInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get();
|
||||||
@@ -222,6 +246,11 @@ public class StreamReadMonitoringFunction
|
|||||||
List<HoodieCommitMetadata> metadataList = instants.stream()
|
List<HoodieCommitMetadata> metadataList = instants.stream()
|
||||||
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
.map(instant -> WriteProfiles.getCommitMetadata(tableName, path, instant, commitTimeline)).collect(Collectors.toList());
|
||||||
Set<String> writePartitions = getWritePartitionPaths(metadataList);
|
Set<String> writePartitions = getWritePartitionPaths(metadataList);
|
||||||
|
// apply partition push down
|
||||||
|
if (this.requiredPartitionPaths.size() > 0) {
|
||||||
|
writePartitions = writePartitions.stream()
|
||||||
|
.filter(this.requiredPartitionPaths::contains).collect(Collectors.toSet());
|
||||||
|
}
|
||||||
FileStatus[] fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList);
|
FileStatus[] fileStatuses = WriteProfiles.getWritePathsOfInstants(path, hadoopConf, metadataList);
|
||||||
if (fileStatuses.length == 0) {
|
if (fileStatuses.length == 0) {
|
||||||
LOG.warn("No files found for reading in user provided path.");
|
LOG.warn("No files found for reading in user provided path.");
|
||||||
@@ -310,7 +339,8 @@ public class StreamReadMonitoringFunction
|
|||||||
return commitTimeline.getInstants()
|
return commitTimeline.getInstants()
|
||||||
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, issuedInstant))
|
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN, issuedInstant))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
} else if (this.conf.getOptional(FlinkOptions.READ_STREAMING_START_COMMIT).isPresent()) {
|
} else if (this.conf.getOptional(FlinkOptions.READ_STREAMING_START_COMMIT).isPresent()
|
||||||
|
&& !this.conf.get(FlinkOptions.READ_STREAMING_START_COMMIT).equalsIgnoreCase(FlinkOptions.START_COMMIT_EARLIEST)) {
|
||||||
String definedStartCommit = this.conf.get(FlinkOptions.READ_STREAMING_START_COMMIT);
|
String definedStartCommit = this.conf.get(FlinkOptions.READ_STREAMING_START_COMMIT);
|
||||||
return commitTimeline.getInstants()
|
return commitTimeline.getInstants()
|
||||||
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, definedStartCommit))
|
.filter(s -> HoodieTimeline.compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, definedStartCommit))
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.table;
|
package org.apache.hudi.table;
|
||||||
|
|
||||||
|
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
import org.apache.hudi.common.fs.FSUtils;
|
||||||
import org.apache.hudi.common.model.BaseFile;
|
import org.apache.hudi.common.model.BaseFile;
|
||||||
import org.apache.hudi.common.model.HoodieLogFile;
|
import org.apache.hudi.common.model.HoodieLogFile;
|
||||||
@@ -85,6 +86,7 @@ import java.util.Collections;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
@@ -151,9 +153,8 @@ public class HoodieTableSource implements
|
|||||||
: requiredPos;
|
: requiredPos;
|
||||||
this.limit = limit == null ? NO_LIMIT_CONSTANT : limit;
|
this.limit = limit == null ? NO_LIMIT_CONSTANT : limit;
|
||||||
this.filters = filters == null ? Collections.emptyList() : filters;
|
this.filters = filters == null ? Collections.emptyList() : filters;
|
||||||
final String basePath = this.conf.getString(FlinkOptions.PATH);
|
|
||||||
this.hadoopConf = StreamerUtil.getHadoopConf();
|
this.hadoopConf = StreamerUtil.getHadoopConf();
|
||||||
this.metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
|
this.metaClient = StreamerUtil.metaClientForReader(conf, hadoopConf);
|
||||||
this.maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new JobConf(this.hadoopConf));
|
this.maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(new JobConf(this.hadoopConf));
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -173,11 +174,8 @@ public class HoodieTableSource implements
|
|||||||
(TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(getProducedDataType());
|
(TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(getProducedDataType());
|
||||||
if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING)) {
|
if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING)) {
|
||||||
StreamReadMonitoringFunction monitoringFunction = new StreamReadMonitoringFunction(
|
StreamReadMonitoringFunction monitoringFunction = new StreamReadMonitoringFunction(
|
||||||
conf, FilePathUtils.toFlinkPath(path), metaClient, maxCompactionMemoryInBytes);
|
conf, FilePathUtils.toFlinkPath(path), maxCompactionMemoryInBytes, getRequiredPartitionPaths());
|
||||||
InputFormat<RowData, ?> inputFormat = getInputFormat(true);
|
InputFormat<RowData, ?> inputFormat = getInputFormat(true);
|
||||||
if (!(inputFormat instanceof MergeOnReadInputFormat)) {
|
|
||||||
throw new HoodieException("No successful commits under path " + path);
|
|
||||||
}
|
|
||||||
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat);
|
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory((MergeOnReadInputFormat) inputFormat);
|
||||||
SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, "streaming_source")
|
SingleOutputStreamOperator<RowData> source = execEnv.addSource(monitoringFunction, "streaming_source")
|
||||||
.uid("uid_streaming_source_" + conf.getString(FlinkOptions.TABLE_NAME))
|
.uid("uid_streaming_source_" + conf.getString(FlinkOptions.TABLE_NAME))
|
||||||
@@ -218,7 +216,8 @@ public class HoodieTableSource implements
|
|||||||
@Override
|
@Override
|
||||||
public Result applyFilters(List<ResolvedExpression> filters) {
|
public Result applyFilters(List<ResolvedExpression> filters) {
|
||||||
this.filters = new ArrayList<>(filters);
|
this.filters = new ArrayList<>(filters);
|
||||||
return Result.of(new ArrayList<>(filters), new ArrayList<>(filters));
|
// refuse all the filters now
|
||||||
|
return Result.of(Collections.emptyList(), new ArrayList<>(filters));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@@ -266,7 +265,18 @@ public class HoodieTableSource implements
|
|||||||
return requiredPartitions;
|
return requiredPartitions;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Set<String> getRequiredPartitionPaths() {
|
||||||
|
if (this.requiredPartitions == null) {
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
return FilePathUtils.toRelativePartitionPaths(this.partitionKeys, this.requiredPartitions,
|
||||||
|
conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING));
|
||||||
|
}
|
||||||
|
|
||||||
private List<MergeOnReadInputSplit> buildFileIndex(Path[] paths) {
|
private List<MergeOnReadInputSplit> buildFileIndex(Path[] paths) {
|
||||||
|
if (paths.length == 0) {
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
FileStatus[] fileStatuses = Arrays.stream(paths)
|
FileStatus[] fileStatuses = Arrays.stream(paths)
|
||||||
.flatMap(path ->
|
.flatMap(path ->
|
||||||
Arrays.stream(FilePathUtils.getFileStatusRecursively(path, 1, hadoopConf)))
|
Arrays.stream(FilePathUtils.getFileStatusRecursively(path, 1, hadoopConf)))
|
||||||
@@ -304,6 +314,10 @@ public class HoodieTableSource implements
|
|||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public InputFormat<RowData, ?> getInputFormat(boolean isStreaming) {
|
public InputFormat<RowData, ?> getInputFormat(boolean isStreaming) {
|
||||||
|
return isStreaming ? getStreamInputFormat() : getBatchInputFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
private InputFormat<RowData, ?> getBatchInputFormat() {
|
||||||
// When this table has no partition, just return an empty source.
|
// When this table has no partition, just return an empty source.
|
||||||
if (!partitionKeys.isEmpty() && getOrFetchPartitions().isEmpty()) {
|
if (!partitionKeys.isEmpty() && getOrFetchPartitions().isEmpty()) {
|
||||||
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
||||||
@@ -314,13 +328,7 @@ public class HoodieTableSource implements
|
|||||||
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
||||||
}
|
}
|
||||||
|
|
||||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
final Schema tableAvroSchema = getTableAvroSchema();
|
||||||
final Schema tableAvroSchema;
|
|
||||||
try {
|
|
||||||
tableAvroSchema = schemaUtil.getTableAvroSchema();
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new HoodieException("Get table avro schema error", e);
|
|
||||||
}
|
|
||||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||||
final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType();
|
final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType();
|
||||||
@@ -331,18 +339,12 @@ public class HoodieTableSource implements
|
|||||||
final HoodieTableType tableType = HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE));
|
final HoodieTableType tableType = HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||||
switch (tableType) {
|
switch (tableType) {
|
||||||
case MERGE_ON_READ:
|
case MERGE_ON_READ:
|
||||||
final List<MergeOnReadInputSplit> inputSplits;
|
final List<MergeOnReadInputSplit> inputSplits = buildFileIndex(paths);
|
||||||
if (!isStreaming) {
|
|
||||||
inputSplits = buildFileIndex(paths);
|
|
||||||
if (inputSplits.size() == 0) {
|
if (inputSplits.size() == 0) {
|
||||||
// When there is no input splits, just return an empty source.
|
// When there is no input splits, just return an empty source.
|
||||||
LOG.warn("No input splits generate for MERGE_ON_READ input format, returns empty collection instead");
|
LOG.warn("No input splits generate for MERGE_ON_READ input format, returns empty collection instead");
|
||||||
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
return new CollectionInputFormat<>(Collections.emptyList(), null);
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
// streaming reader would build the splits automatically.
|
|
||||||
inputSplits = Collections.emptyList();
|
|
||||||
}
|
|
||||||
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
||||||
rowType,
|
rowType,
|
||||||
requiredRowType,
|
requiredRowType,
|
||||||
@@ -359,28 +361,9 @@ public class HoodieTableSource implements
|
|||||||
.fieldTypes(rowDataType.getChildren())
|
.fieldTypes(rowDataType.getChildren())
|
||||||
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
||||||
.limit(this.limit)
|
.limit(this.limit)
|
||||||
.emitDelete(isStreaming)
|
.emitDelete(false)
|
||||||
.build();
|
.build();
|
||||||
case COPY_ON_WRITE:
|
case COPY_ON_WRITE:
|
||||||
if (isStreaming) {
|
|
||||||
final MergeOnReadTableState hoodieTableState2 = new MergeOnReadTableState(
|
|
||||||
rowType,
|
|
||||||
requiredRowType,
|
|
||||||
tableAvroSchema.toString(),
|
|
||||||
AvroSchemaConverter.convertToSchema(requiredRowType).toString(),
|
|
||||||
Collections.emptyList(),
|
|
||||||
conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","));
|
|
||||||
return MergeOnReadInputFormat.builder()
|
|
||||||
.config(this.conf)
|
|
||||||
.paths(FilePathUtils.toFlinkPaths(paths))
|
|
||||||
.tableState(hoodieTableState2)
|
|
||||||
// use the explicit fields data type because the AvroSchemaConverter
|
|
||||||
// is not very stable.
|
|
||||||
.fieldTypes(rowDataType.getChildren())
|
|
||||||
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
|
||||||
.limit(this.limit)
|
|
||||||
.build();
|
|
||||||
}
|
|
||||||
FileInputFormat<RowData> format = new CopyOnWriteInputFormat(
|
FileInputFormat<RowData> format = new CopyOnWriteInputFormat(
|
||||||
FilePathUtils.toFlinkPaths(paths),
|
FilePathUtils.toFlinkPaths(paths),
|
||||||
this.schema.getColumnNames().toArray(new String[0]),
|
this.schema.getColumnNames().toArray(new String[0]),
|
||||||
@@ -416,6 +399,86 @@ public class HoodieTableSource implements
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private InputFormat<RowData, ?> getStreamInputFormat() {
|
||||||
|
// if table does not exist, use schema from the DDL
|
||||||
|
Schema tableAvroSchema = this.metaClient == null ? inferSchemaFromDdl() : getTableAvroSchema();
|
||||||
|
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||||
|
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||||
|
final RowType requiredRowType = (RowType) getProducedDataType().notNull().getLogicalType();
|
||||||
|
|
||||||
|
final String queryType = this.conf.getString(FlinkOptions.QUERY_TYPE);
|
||||||
|
org.apache.flink.core.fs.Path[] paths = new org.apache.flink.core.fs.Path[0];
|
||||||
|
if (FlinkOptions.QUERY_TYPE_SNAPSHOT.equals(queryType)) {
|
||||||
|
final HoodieTableType tableType = HoodieTableType.valueOf(this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||||
|
switch (tableType) {
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
||||||
|
rowType,
|
||||||
|
requiredRowType,
|
||||||
|
tableAvroSchema.toString(),
|
||||||
|
AvroSchemaConverter.convertToSchema(requiredRowType).toString(),
|
||||||
|
Collections.emptyList(),
|
||||||
|
conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","));
|
||||||
|
return MergeOnReadInputFormat.builder()
|
||||||
|
.config(this.conf)
|
||||||
|
.paths(paths)
|
||||||
|
.tableState(hoodieTableState)
|
||||||
|
// use the explicit fields data type because the AvroSchemaConverter
|
||||||
|
// is not very stable.
|
||||||
|
.fieldTypes(rowDataType.getChildren())
|
||||||
|
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
||||||
|
.limit(this.limit)
|
||||||
|
.emitDelete(true)
|
||||||
|
.build();
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
final MergeOnReadTableState hoodieTableState2 = new MergeOnReadTableState(
|
||||||
|
rowType,
|
||||||
|
requiredRowType,
|
||||||
|
tableAvroSchema.toString(),
|
||||||
|
AvroSchemaConverter.convertToSchema(requiredRowType).toString(),
|
||||||
|
Collections.emptyList(),
|
||||||
|
conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","));
|
||||||
|
return MergeOnReadInputFormat.builder()
|
||||||
|
.config(this.conf)
|
||||||
|
.paths(paths)
|
||||||
|
.tableState(hoodieTableState2)
|
||||||
|
// use the explicit fields data type because the AvroSchemaConverter
|
||||||
|
// is not very stable.
|
||||||
|
.fieldTypes(rowDataType.getChildren())
|
||||||
|
.defaultPartName(conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME))
|
||||||
|
.limit(this.limit)
|
||||||
|
.build();
|
||||||
|
default:
|
||||||
|
throw new HoodieException("Unexpected table type: " + this.conf.getString(FlinkOptions.TABLE_TYPE));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
String errMsg = String.format("Invalid query type : '%s', options ['%s'] are supported now", queryType,
|
||||||
|
FlinkOptions.QUERY_TYPE_SNAPSHOT);
|
||||||
|
throw new HoodieException(errMsg);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Schema inferSchemaFromDdl() {
|
||||||
|
Schema schema = AvroSchemaConverter.convertToSchema(this.schema.toSourceRowDataType().getLogicalType());
|
||||||
|
return HoodieAvroUtils.addMetadataFields(schema, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public Schema getTableAvroSchema() {
|
||||||
|
try {
|
||||||
|
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
|
||||||
|
return schemaUtil.getTableAvroSchema();
|
||||||
|
} catch (Throwable e) {
|
||||||
|
// table exists but has no written data
|
||||||
|
LOG.warn("Get table avro schema error, use schema from the DDL instead", e);
|
||||||
|
return inferSchemaFromDdl();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public HoodieTableMetaClient getMetaClient() {
|
||||||
|
return this.metaClient;
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public Configuration getConf() {
|
public Configuration getConf() {
|
||||||
return this.conf;
|
return this.conf;
|
||||||
|
|||||||
@@ -36,6 +36,7 @@ import java.util.BitSet;
|
|||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
@@ -82,11 +83,13 @@ public class FilePathUtils {
|
|||||||
* @param partitionKVs The partition key value mapping
|
* @param partitionKVs The partition key value mapping
|
||||||
* @param hivePartition Whether the partition path is with Hive style,
|
* @param hivePartition Whether the partition path is with Hive style,
|
||||||
* e.g. {partition key} = {partition value}
|
* e.g. {partition key} = {partition value}
|
||||||
|
* @param sepSuffix Whether to append the file separator as suffix
|
||||||
* @return an escaped, valid partition name
|
* @return an escaped, valid partition name
|
||||||
*/
|
*/
|
||||||
public static String generatePartitionPath(
|
public static String generatePartitionPath(
|
||||||
LinkedHashMap<String, String> partitionKVs,
|
LinkedHashMap<String, String> partitionKVs,
|
||||||
boolean hivePartition) {
|
boolean hivePartition,
|
||||||
|
boolean sepSuffix) {
|
||||||
if (partitionKVs.isEmpty()) {
|
if (partitionKVs.isEmpty()) {
|
||||||
return "";
|
return "";
|
||||||
}
|
}
|
||||||
@@ -103,7 +106,9 @@ public class FilePathUtils {
|
|||||||
suffixBuf.append(escapePathName(e.getValue()));
|
suffixBuf.append(escapePathName(e.getValue()));
|
||||||
i++;
|
i++;
|
||||||
}
|
}
|
||||||
|
if (sepSuffix) {
|
||||||
suffixBuf.append(File.separator);
|
suffixBuf.append(File.separator);
|
||||||
|
}
|
||||||
return suffixBuf.toString();
|
return suffixBuf.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -371,11 +376,30 @@ public class FilePathUtils {
|
|||||||
boolean hivePartition) {
|
boolean hivePartition) {
|
||||||
return partitionPaths.stream()
|
return partitionPaths.stream()
|
||||||
.map(m -> validateAndReorderPartitions(m, partitionKeys))
|
.map(m -> validateAndReorderPartitions(m, partitionKeys))
|
||||||
.map(kvs -> FilePathUtils.generatePartitionPath(kvs, hivePartition))
|
.map(kvs -> FilePathUtils.generatePartitionPath(kvs, hivePartition, true))
|
||||||
.map(n -> new Path(path, n))
|
.map(n -> new Path(path, n))
|
||||||
.toArray(Path[]::new);
|
.toArray(Path[]::new);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Transforms the given partition key value mapping to relative partition paths.
|
||||||
|
*
|
||||||
|
* @param partitionKeys The partition key list
|
||||||
|
* @param partitionPaths The partition key value mapping
|
||||||
|
* @param hivePartition Whether the partition path is in Hive style
|
||||||
|
*
|
||||||
|
* @see #getReadPaths
|
||||||
|
*/
|
||||||
|
public static Set<String> toRelativePartitionPaths(
|
||||||
|
List<String> partitionKeys,
|
||||||
|
List<Map<String, String>> partitionPaths,
|
||||||
|
boolean hivePartition) {
|
||||||
|
return partitionPaths.stream()
|
||||||
|
.map(m -> validateAndReorderPartitions(m, partitionKeys))
|
||||||
|
.map(kvs -> FilePathUtils.generatePartitionPath(kvs, hivePartition, false))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transforms the array of Hadoop paths to Flink paths.
|
* Transforms the array of Hadoop paths to Flink paths.
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -229,9 +229,7 @@ public class StreamerUtil {
|
|||||||
public static void initTableIfNotExists(Configuration conf) throws IOException {
|
public static void initTableIfNotExists(Configuration conf) throws IOException {
|
||||||
final String basePath = conf.getString(FlinkOptions.PATH);
|
final String basePath = conf.getString(FlinkOptions.PATH);
|
||||||
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||||
// Hadoop FileSystem
|
if (!tableExists(basePath, hadoopConf)) {
|
||||||
FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
|
|
||||||
if (!fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))) {
|
|
||||||
HoodieTableMetaClient.withPropertyBuilder()
|
HoodieTableMetaClient.withPropertyBuilder()
|
||||||
.setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
|
.setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
|
||||||
.setTableName(conf.getString(FlinkOptions.TABLE_NAME))
|
.setTableName(conf.getString(FlinkOptions.TABLE_NAME))
|
||||||
@@ -251,6 +249,19 @@ public class StreamerUtil {
|
|||||||
// some of the filesystems release the handles in #close method.
|
// some of the filesystems release the handles in #close method.
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns whether the hoodie table exists under given path {@code basePath}.
|
||||||
|
*/
|
||||||
|
public static boolean tableExists(String basePath, org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||||
|
// Hadoop FileSystem
|
||||||
|
FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
|
||||||
|
try {
|
||||||
|
return fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME));
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieException("Error while checking whether table exists under path:" + basePath, e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Generates the bucket ID using format {partition path}_{fileID}.
|
* Generates the bucket ID using format {partition path}_{fileID}.
|
||||||
*/
|
*/
|
||||||
@@ -282,11 +293,37 @@ public class StreamerUtil {
|
|||||||
&& conf.getBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED);
|
&& conf.getBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the meta client for reader.
|
||||||
|
*
|
||||||
|
* <p>The streaming pipeline process is long running, so empty table path is allowed,
|
||||||
|
* the reader would then check and refresh the meta client.
|
||||||
|
*
|
||||||
|
* @see org.apache.hudi.source.StreamReadMonitoringFunction
|
||||||
|
*/
|
||||||
|
public static HoodieTableMetaClient metaClientForReader(
|
||||||
|
Configuration conf,
|
||||||
|
org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||||
|
final String basePath = conf.getString(FlinkOptions.PATH);
|
||||||
|
if (conf.getBoolean(FlinkOptions.READ_AS_STREAMING) && !tableExists(basePath, hadoopConf)) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
return createMetaClient(basePath, hadoopConf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the meta client.
|
||||||
|
*/
|
||||||
|
public static HoodieTableMetaClient createMetaClient(String basePath, org.apache.hadoop.conf.Configuration hadoopConf) {
|
||||||
|
return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(hadoopConf).build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates the meta client.
|
* Creates the meta client.
|
||||||
*/
|
*/
|
||||||
public static HoodieTableMetaClient createMetaClient(String basePath) {
|
public static HoodieTableMetaClient createMetaClient(String basePath) {
|
||||||
return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(FlinkClientUtil.getHadoopConf()).build();
|
return createMetaClient(basePath, FlinkClientUtil.getHadoopConf());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|||||||
@@ -168,6 +168,37 @@ public class TestStreamReadMonitoringFunction {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConsumeFromEarliestCommit() throws Exception {
|
||||||
|
// write 2 commits first, then specify the start commit as 'earliest',
|
||||||
|
// all the splits should come from the earliest commit.
|
||||||
|
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||||
|
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
|
||||||
|
String specifiedCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
|
||||||
|
conf.setString(FlinkOptions.READ_STREAMING_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
|
||||||
|
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||||
|
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||||
|
harness.setup();
|
||||||
|
harness.open();
|
||||||
|
|
||||||
|
CountDownLatch latch = new CountDownLatch(4);
|
||||||
|
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||||
|
|
||||||
|
runAsync(sourceContext, function);
|
||||||
|
|
||||||
|
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||||
|
assertThat("Should produce the expected splits",
|
||||||
|
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||||
|
assertTrue(sourceContext.splits.stream().noneMatch(split -> split.getInstantRange().isPresent()),
|
||||||
|
"No instants should have range limit");
|
||||||
|
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getLatestCommit().equals(specifiedCommit)),
|
||||||
|
"All the splits should be with specified instant time");
|
||||||
|
|
||||||
|
// Stop the stream task.
|
||||||
|
function.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCheckpointRestore() throws Exception {
|
public void testCheckpointRestore() throws Exception {
|
||||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||||
|
|||||||
@@ -21,7 +21,6 @@ package org.apache.hudi.table;
|
|||||||
import org.apache.hudi.common.model.HoodieTableType;
|
import org.apache.hudi.common.model.HoodieTableType;
|
||||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||||
import org.apache.hudi.configuration.FlinkOptions;
|
import org.apache.hudi.configuration.FlinkOptions;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
|
||||||
import org.apache.hudi.util.StreamerUtil;
|
import org.apache.hudi.util.StreamerUtil;
|
||||||
import org.apache.hudi.utils.TestConfigurations;
|
import org.apache.hudi.utils.TestConfigurations;
|
||||||
import org.apache.hudi.utils.TestData;
|
import org.apache.hudi.utils.TestData;
|
||||||
@@ -124,6 +123,13 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
|||||||
execInsertSql(streamTableEnv, insertInto);
|
execInsertSql(streamTableEnv, insertInto);
|
||||||
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||||
assertRowsEquals(rows2, TestData.DATA_SET_SOURCE_INSERT);
|
assertRowsEquals(rows2, TestData.DATA_SET_SOURCE_INSERT);
|
||||||
|
|
||||||
|
streamTableEnv.getConfig().getConfiguration()
|
||||||
|
.setBoolean("table.dynamic-table-options.enabled", true);
|
||||||
|
// specify the start commit as earliest
|
||||||
|
List<Row> rows3 = execSelectSql(streamTableEnv,
|
||||||
|
"select * from t1/*+options('read.streaming.start-commit'='earliest')*/", 10);
|
||||||
|
assertRowsEquals(rows3, TestData.DATA_SET_SOURCE_INSERT);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@@ -300,6 +306,34 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
|||||||
assertRowsEquals(result, expected, true);
|
assertRowsEquals(result, expected, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("tableTypeAndPartitioningParams")
|
||||||
|
void testStreamReadFilterByPartition(HoodieTableType tableType, boolean hiveStylePartitioning) throws Exception {
|
||||||
|
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||||
|
conf.setString(FlinkOptions.TABLE_NAME, "t1");
|
||||||
|
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
|
||||||
|
conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning);
|
||||||
|
|
||||||
|
// write one commit
|
||||||
|
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||||
|
|
||||||
|
String hoodieTableDDL = sql("t1")
|
||||||
|
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||||
|
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
|
||||||
|
.option(FlinkOptions.READ_AS_STREAMING, "true")
|
||||||
|
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, "2")
|
||||||
|
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
|
||||||
|
.end();
|
||||||
|
streamTableEnv.executeSql(hoodieTableDDL);
|
||||||
|
|
||||||
|
List<Row> result = execSelectSql(streamTableEnv,
|
||||||
|
"select * from t1 where `partition`='par1'", 10);
|
||||||
|
final String expected = "["
|
||||||
|
+ "+I(+I[id1, Danny, 23, 1970-01-01T00:00:00.001, par1]), "
|
||||||
|
+ "+I(+I[id2, Stephen, 33, 1970-01-01T00:00:00.002, par1])]";
|
||||||
|
assertRowsEquals(result, expected, true);
|
||||||
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource("executionModeAndPartitioningParams")
|
@MethodSource("executionModeAndPartitioningParams")
|
||||||
void testWriteAndRead(ExecMode execMode, boolean hiveStylePartitioning) {
|
void testWriteAndRead(ExecMode execMode, boolean hiveStylePartitioning) {
|
||||||
@@ -568,10 +602,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
void testStreamReadEmptyTablePath() throws Exception {
|
void testStreamReadEmptyTablePath() throws Exception {
|
||||||
// create an empty table
|
// case1: table metadata path does not exists
|
||||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
|
||||||
StreamerUtil.initTableIfNotExists(conf);
|
|
||||||
|
|
||||||
// create a flink source table
|
// create a flink source table
|
||||||
String createHoodieTable = sql("t1")
|
String createHoodieTable = sql("t1")
|
||||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||||
@@ -580,10 +611,39 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
|||||||
.end();
|
.end();
|
||||||
streamTableEnv.executeSql(createHoodieTable);
|
streamTableEnv.executeSql(createHoodieTable);
|
||||||
|
|
||||||
// execute query and assert throws exception
|
// no exception expects to be thrown
|
||||||
assertThrows(HoodieException.class, () -> execSelectSql(streamTableEnv, "select * from t1", 10),
|
List<Row> rows1 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||||
"No successful commits under path " + tempFile.getAbsolutePath());
|
assertRowsEquals(rows1, "[]");
|
||||||
|
|
||||||
|
// case2: empty table without data files
|
||||||
|
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||||
|
StreamerUtil.initTableIfNotExists(conf);
|
||||||
|
|
||||||
|
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||||
|
assertRowsEquals(rows2, "[]");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testBatchReadEmptyTablePath() throws Exception {
|
||||||
|
// case1: table metadata path does not exists
|
||||||
|
// create a flink source table
|
||||||
|
String createHoodieTable = sql("t1")
|
||||||
|
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||||
|
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
|
||||||
|
.end();
|
||||||
|
batchTableEnv.executeSql(createHoodieTable);
|
||||||
|
|
||||||
|
// no exception expects to be thrown
|
||||||
|
assertThrows(Exception.class,
|
||||||
|
() -> execSelectSql(batchTableEnv, "select * from t1", 10),
|
||||||
|
"Exception should throw when querying non-exists table in batch mode");
|
||||||
|
|
||||||
|
// case2: empty table without data files
|
||||||
|
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||||
|
StreamerUtil.initTableIfNotExists(conf);
|
||||||
|
|
||||||
|
List<Row> rows2 = CollectionUtil.iteratorToList(batchTableEnv.executeSql("select * from t1").collect());
|
||||||
|
assertRowsEquals(rows2, "[]");
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@@ -781,6 +841,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
|||||||
|
|
||||||
private List<Row> execSelectSql(TableEnvironment tEnv, String select, String sinkDDL, long timeout)
|
private List<Row> execSelectSql(TableEnvironment tEnv, String select, String sinkDDL, long timeout)
|
||||||
throws InterruptedException {
|
throws InterruptedException {
|
||||||
|
tEnv.executeSql("DROP TABLE IF EXISTS sink");
|
||||||
tEnv.executeSql(sinkDDL);
|
tEnv.executeSql(sinkDDL);
|
||||||
TableResult tableResult = tEnv.executeSql("insert into sink " + select);
|
TableResult tableResult = tEnv.executeSql("insert into sink " + select);
|
||||||
// wait for the timeout then cancels the job
|
// wait for the timeout then cancels the job
|
||||||
|
|||||||
@@ -25,12 +25,12 @@ import org.apache.hudi.util.StreamerUtil;
|
|||||||
import org.apache.hudi.utils.TestConfigurations;
|
import org.apache.hudi.utils.TestConfigurations;
|
||||||
import org.apache.hudi.utils.TestData;
|
import org.apache.hudi.utils.TestData;
|
||||||
|
|
||||||
|
import org.apache.avro.Schema;
|
||||||
import org.apache.flink.api.common.io.FileInputFormat;
|
import org.apache.flink.api.common.io.FileInputFormat;
|
||||||
import org.apache.flink.api.common.io.InputFormat;
|
import org.apache.flink.api.common.io.InputFormat;
|
||||||
import org.apache.flink.configuration.Configuration;
|
import org.apache.flink.configuration.Configuration;
|
||||||
import org.apache.flink.table.data.RowData;
|
import org.apache.flink.table.data.RowData;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.io.TempDir;
|
import org.junit.jupiter.api.io.TempDir;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@@ -43,12 +43,14 @@ import java.util.Collections;
|
|||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
import static org.hamcrest.MatcherAssert.assertThat;
|
||||||
import static org.hamcrest.core.Is.is;
|
import static org.hamcrest.core.Is.is;
|
||||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -62,17 +64,17 @@ public class TestHoodieTableSource {
|
|||||||
@TempDir
|
@TempDir
|
||||||
File tempFile;
|
File tempFile;
|
||||||
|
|
||||||
@BeforeEach
|
|
||||||
void beforeEach() throws IOException {
|
void beforeEach() throws IOException {
|
||||||
final String path = tempFile.getAbsolutePath();
|
final String path = tempFile.getAbsolutePath();
|
||||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
conf = TestConfigurations.getDefaultConf(path);
|
||||||
StreamerUtil.initTableIfNotExists(conf);
|
StreamerUtil.initTableIfNotExists(conf);
|
||||||
IntStream.range(1, 5)
|
IntStream.range(1, 5)
|
||||||
.forEach(i -> new File(path + File.separator + "par" + i).mkdirs());
|
.forEach(i -> new File(path + File.separator + "par" + i).mkdirs());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
void testGetReadPaths() {
|
void testGetReadPaths() throws Exception {
|
||||||
|
beforeEach();
|
||||||
HoodieTableSource tableSource = new HoodieTableSource(
|
HoodieTableSource tableSource = new HoodieTableSource(
|
||||||
TestConfigurations.TABLE_SCHEMA,
|
TestConfigurations.TABLE_SCHEMA,
|
||||||
new Path(tempFile.getPath()),
|
new Path(tempFile.getPath()),
|
||||||
@@ -99,6 +101,7 @@ public class TestHoodieTableSource {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
void testGetInputFormat() throws Exception {
|
void testGetInputFormat() throws Exception {
|
||||||
|
beforeEach();
|
||||||
// write some data to let the TableSchemaResolver get the right instant
|
// write some data to let the TableSchemaResolver get the right instant
|
||||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||||
|
|
||||||
@@ -118,4 +121,29 @@ public class TestHoodieTableSource {
|
|||||||
() -> tableSource.getInputFormat(),
|
() -> tableSource.getInputFormat(),
|
||||||
"Invalid query type : 'incremental'. Only 'snapshot' is supported now");
|
"Invalid query type : 'incremental'. Only 'snapshot' is supported now");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
void testGetTableAvroSchema() {
|
||||||
|
final String path = tempFile.getAbsolutePath();
|
||||||
|
conf = TestConfigurations.getDefaultConf(path);
|
||||||
|
conf.setBoolean(FlinkOptions.READ_AS_STREAMING, true);
|
||||||
|
|
||||||
|
HoodieTableSource tableSource = new HoodieTableSource(
|
||||||
|
TestConfigurations.TABLE_SCHEMA,
|
||||||
|
new Path(tempFile.getPath()),
|
||||||
|
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
|
||||||
|
"default-par",
|
||||||
|
conf);
|
||||||
|
assertNull(tableSource.getMetaClient(), "Streaming source with empty table path is allowed");
|
||||||
|
final String schemaFields = tableSource.getTableAvroSchema().getFields().stream()
|
||||||
|
.map(Schema.Field::name)
|
||||||
|
.collect(Collectors.joining(","));
|
||||||
|
final String expected = "_hoodie_commit_time,"
|
||||||
|
+ "_hoodie_commit_seqno,"
|
||||||
|
+ "_hoodie_record_key,"
|
||||||
|
+ "_hoodie_partition_path,"
|
||||||
|
+ "_hoodie_file_name,"
|
||||||
|
+ "uuid,name,age,ts,partition";
|
||||||
|
assertThat(schemaFields, is(expected));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -28,6 +28,7 @@ import org.apache.flink.configuration.Configuration;
|
|||||||
import org.apache.flink.core.fs.Path;
|
import org.apache.flink.core.fs.Path;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
@@ -57,8 +58,6 @@ public class TestUtils {
|
|||||||
|
|
||||||
public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) {
|
public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) {
|
||||||
final String basePath = conf.getString(FlinkOptions.PATH);
|
final String basePath = conf.getString(FlinkOptions.PATH);
|
||||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
return new StreamReadMonitoringFunction(conf, new Path(basePath), 1024 * 1024L, Collections.emptySet());
|
||||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
|
||||||
return new StreamReadMonitoringFunction(conf, new Path(basePath), metaClient, 1024 * 1024L);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
Reference in New Issue
Block a user