cshuo commented on code in PR #18406:
URL: https://github.com/apache/hudi/pull/18406#discussion_r3008098721
##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/reader/function/HoodieSplitReaderFunction.java:
##########
@@ -71,28 +62,26 @@ public HoodieSplitReaderFunction(
String mergeType,
List<ExpressionPredicates.Predicate> predicates,
boolean emitDelete) {
-
+ super(configuration, predicates, internalSchemaManager, emitDelete);
ValidationUtils.checkArgument(tableSchema != null, "tableSchema can't be
null");
ValidationUtils.checkArgument(requiredSchema != null, "requiredSchema
can't be null");
ValidationUtils.checkArgument(internalSchemaManager != null,
"internalSchemaManager can't be null");
this.tableSchema = tableSchema;
this.requiredSchema = requiredSchema;
- this.internalSchemaManager = internalSchemaManager;
- this.configuration = configuration;
- this.writeConfig = FlinkWriteClients.getHoodieClientConfig(configuration);
- this.predicates = predicates;
this.mergeType = mergeType;
- this.emitDelete = emitDelete;
}
@Override
public RecordsWithSplitIds<HoodieRecordWithPosition<RowData>>
read(HoodieSourceSplit split) {
final String splitId = split.splitId();
- HoodieTableMetaClient metaClient =
StreamerUtil.metaClientForReader(configuration, getHadoopConf());
+ HoodieTableMetaClient metaClient = StreamerUtil.metaClientForReader(conf,
getHadoopConf());
try {
+ if (fileGroupReader != null) {
Review Comment:
We do not need to close file group reader here, since the underlying file
group reader will be closed when `RecordsWithSplitIds` is finished and
`RecordsWithSplitIds#recycle()` is called.
Besides, it's not safe as well, because the split fetcher runs in a separate
thread, and may fetch more than one `RecordsWithSplitIds` into a queue, and
it's possible that the second `RecordsWithSplitIds` is fetched before the first
`RecordsWithSplitIds` is finished.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]