xushiyan commented on code in PR #18074: URL: https://github.com/apache/hudi/pull/18074#discussion_r2766602424
########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/SplitUtils.java: ########## @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.source.FileIndex; +import org.apache.hudi.source.split.HoodieSourceSplit; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static org.apache.hudi.util.StreamerUtil.EMPTY_PARTITION_PATH; + +/** + * Utils for creating splits from file index. + */ +public class SplitUtils { + + /** + * Get the reader paths with partition path expanded. + */ + public static List<FileSlice> getBaseFileOnlyFileSlices(HoodieTableMetaClient metaClient, FileIndex fileIndex) { + List<String> relPartitionPaths = fileIndex.getOrBuildPartitionPaths(); + if (relPartitionPaths.isEmpty()) { + return Collections.emptyList(); + } + List<StoragePathInfo> pathInfoList = fileIndex.getFilesInPartitions(); + try (HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), pathInfoList)) { + + List<FileSlice> allFileSlices = relPartitionPaths.stream() + .flatMap(par -> fsView.getLatestBaseFiles(par) + .map(baseFile -> new FileSlice(new HoodieFileGroupId(par, baseFile.getFileId()), baseFile.getCommitTime(), baseFile, Collections.emptyList()))) + .collect(Collectors.toList()); + return fileIndex.filterFileSlices(allFileSlices); + } + } + + public static List<HoodieSourceSplit> baseFileOnlyHoodieSourceSplit(HoodieTableMetaClient metaClient, StoragePath path, FileIndex fileIndex, String mergeType) { + final List<FileSlice> fileSlices = getBaseFileOnlyFileSlices(metaClient, fileIndex); + if (fileSlices.isEmpty()) { + return Collections.emptyList(); + } + + List<HoodieSourceSplit> hoodieSourceSplits = fileSlices.stream().filter(fileSlice -> fileSlice.getBaseFile().isPresent()).map(fileSlice -> + new HoodieSourceSplit( + HoodieSourceSplit.SPLIT_COUNTER.incrementAndGet(), + fileSlice.getBaseFile().get().getPath(), + Option.empty(), FilePathUtils.toFlinkPath(path).getPath(), + fileSlice.getPartitionPath(), mergeType, + fileSlice.getLatestInstantTime(), + fileSlice.getFileId()) + ).collect(Collectors.toList()); + + return hoodieSourceSplits; + } + + public static List<HoodieSourceSplit> buildHoodieSplits(HoodieTableMetaClient metaClient, Configuration conf, FileIndex fileIndex) { + List<MergeOnReadInputSplit> splits = buildInputSplits(metaClient, conf, fileIndex); + List<HoodieSourceSplit> hoodieSourceSplits = splits.stream().map(split -> + new HoodieSourceSplit( + HoodieSourceSplit.SPLIT_COUNTER.incrementAndGet(), + split.getBasePath().orElse(null), + split.getLogPaths(), split.getTablePath(), + EMPTY_PARTITION_PATH, split.getMergeType(), Review Comment: always pass `EMPTY_PARTITION_PATH`? this is only meant for non-partitioned table ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/HoodieSource.java: ########## @@ -66,24 +73,28 @@ public class HoodieSource<T> implements Source<T, HoodieSourceSplit, HoodieSplitEnumeratorState> { private static final Logger LOG = LoggerFactory.getLogger(HoodieSource.class); + private final StoragePath storagePath; Review Comment: `HoodieScanContext` uses flink `Path`, while here it's hudi's own StoragePath. let's make it consistent. `StoragePath` is the standard abstraction in Hudi, so we should stick to it. @danny0405 WDYT? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/SplitUtils.java: ########## @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.util; + +import org.apache.flink.configuration.Configuration; +import org.apache.hudi.common.model.BaseFile; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.source.FileIndex; +import org.apache.hudi.source.split.HoodieSourceSplit; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; +import org.apache.hudi.table.format.FilePathUtils; +import org.apache.hudi.table.format.mor.MergeOnReadInputSplit; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static org.apache.hudi.util.StreamerUtil.EMPTY_PARTITION_PATH; + +/** + * Utils for creating splits from file index. + */ +public class SplitUtils { + + /** + * Get the reader paths with partition path expanded. + */ + public static List<FileSlice> getBaseFileOnlyFileSlices(HoodieTableMetaClient metaClient, FileIndex fileIndex) { + List<String> relPartitionPaths = fileIndex.getOrBuildPartitionPaths(); + if (relPartitionPaths.isEmpty()) { + return Collections.emptyList(); + } + List<StoragePathInfo> pathInfoList = fileIndex.getFilesInPartitions(); + try (HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, + metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), pathInfoList)) { + + List<FileSlice> allFileSlices = relPartitionPaths.stream() + .flatMap(par -> fsView.getLatestBaseFiles(par) + .map(baseFile -> new FileSlice(new HoodieFileGroupId(par, baseFile.getFileId()), baseFile.getCommitTime(), baseFile, Collections.emptyList()))) + .collect(Collectors.toList()); + return fileIndex.filterFileSlices(allFileSlices); + } + } + + public static List<HoodieSourceSplit> baseFileOnlyHoodieSourceSplit(HoodieTableMetaClient metaClient, StoragePath path, FileIndex fileIndex, String mergeType) { + final List<FileSlice> fileSlices = getBaseFileOnlyFileSlices(metaClient, fileIndex); + if (fileSlices.isEmpty()) { + return Collections.emptyList(); + } + + List<HoodieSourceSplit> hoodieSourceSplits = fileSlices.stream().filter(fileSlice -> fileSlice.getBaseFile().isPresent()).map(fileSlice -> + new HoodieSourceSplit( + HoodieSourceSplit.SPLIT_COUNTER.incrementAndGet(), + fileSlice.getBaseFile().get().getPath(), + Option.empty(), FilePathUtils.toFlinkPath(path).getPath(), + fileSlice.getPartitionPath(), mergeType, + fileSlice.getLatestInstantTime(), + fileSlice.getFileId()) + ).collect(Collectors.toList()); + + return hoodieSourceSplits; + } + + public static List<HoodieSourceSplit> buildHoodieSplits(HoodieTableMetaClient metaClient, Configuration conf, FileIndex fileIndex) { + List<MergeOnReadInputSplit> splits = buildInputSplits(metaClient, conf, fileIndex); + List<HoodieSourceSplit> hoodieSourceSplits = splits.stream().map(split -> + new HoodieSourceSplit( + HoodieSourceSplit.SPLIT_COUNTER.incrementAndGet(), + split.getBasePath().orElse(null), + split.getLogPaths(), split.getTablePath(), + EMPTY_PARTITION_PATH, split.getMergeType(), + split.getLatestCommit(), + split.getFileId() + ) + ).collect(Collectors.toList()); + + return hoodieSourceSplits; + } + + public static List<MergeOnReadInputSplit> buildInputSplits(HoodieTableMetaClient metaClient, Configuration conf, FileIndex fileIndex) { + List<String> relPartitionPaths = fileIndex.getOrBuildPartitionPaths(); + if (relPartitionPaths.isEmpty()) { + return Collections.emptyList(); + } + List<StoragePathInfo> pathInfoList = fileIndex.getFilesInPartitions(); + if (pathInfoList.isEmpty()) { + throw new HoodieException("No files found for reading in user provided path."); + } + + String latestCommit; + List<FileSlice> allFileSlices; + // file-slice after pending compaction-requested instant-time is also considered valid + try (HoodieTableFileSystemView fsView = new HoodieTableFileSystemView( + metaClient, metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(), pathInfoList)) { + if (!fsView.getLastInstant().isPresent()) { + return Collections.emptyList(); + } + latestCommit = fsView.getLastInstant().get().requestedTime(); + allFileSlices = relPartitionPaths.stream() + .flatMap(par -> fsView.getLatestMergedFileSlicesBeforeOrOn(par, latestCommit)).collect(Collectors.toList()); + } + List<FileSlice> fileSlices = fileIndex.filterFileSlices(allFileSlices); + + final String mergeType = conf.get(FlinkOptions.MERGE_TYPE); + final AtomicInteger cnt = new AtomicInteger(0); + // generates one input split for each file group + return fileSlices.stream().map(fileSlice -> { + String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null); + Option<List<String>> logPaths = Option.ofNullable(fileSlice.getLogFiles() + .sorted(HoodieLogFile.getLogFileComparator()) + .map(logFile -> logFile.getPath().toString()) + .collect(Collectors.toList())); + return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit, + metaClient.getBasePath().toString(), StreamerUtil.getMaxCompactionMemoryInBytes(conf), mergeType, null, fileSlice.getFileId()); Review Comment: maxCompactionMemoryInBytes from HoodieScanContext should take precedence over conf ? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/HoodieSource.java: ########## @@ -130,37 +141,76 @@ private SplitEnumerator<HoodieSourceSplit, HoodieSplitEnumeratorState> createEnu splitProvider = new DefaultHoodieSplitProvider(new HoodieSourceSplitComparator()); } else { LOG.info( - "Hoodie source restored {} splits from state for table {}", - enumeratorState.getPendingSplitStates().size(), - metaClient.getTableConfig().getTableName()); + "Hoodie source restored {} splits from state for table {}", + enumeratorState.getPendingSplitStates().size(), + metaClient.getTableConfig().getTableName()); List<HoodieSourceSplit> pendingSplits = - enumeratorState.getPendingSplitStates().stream().map(HoodieSourceSplitState::split).collect(Collectors.toList()); + enumeratorState.getPendingSplitStates().stream().map(HoodieSourceSplitState::split).collect(Collectors.toList()); splitProvider = new DefaultHoodieSplitProvider(); splitProvider.onDiscoveredSplits(pendingSplits); } if (scanContext.isStreaming()) { HoodieContinuousSplitDiscover discover = new DefaultHoodieSplitDiscover( - scanContext, metaClient); + scanContext, metaClient); return new HoodieContinuousSplitEnumerator(enumContext, splitProvider, discover, scanContext, enumeratorState == null ? Option.empty() : Option.of(enumeratorState)); } else { if (enumeratorState == null) { + List<HoodieSourceSplit> splits = createBatchHoodieSplits(); + splitProvider.onDiscoveredSplits(splits); + } + return new HoodieStaticSplitEnumerator(enumContext, splitProvider); + } + } + + @VisibleForTesting + List<HoodieSourceSplit> createBatchHoodieSplits() { + final Configuration flinkConf = this.scanContext.getConf(); + final String queryType = flinkConf.get(FlinkOptions.QUERY_TYPE); + switch (queryType) { + case FlinkOptions.QUERY_TYPE_SNAPSHOT: + final HoodieTableType tableType = HoodieTableType.valueOf(flinkConf.get(FlinkOptions.TABLE_TYPE)); + switch (tableType) { + case MERGE_ON_READ: + List<HoodieSourceSplit> splits = SplitUtils.buildHoodieSplits(metaClient, flinkConf, createFileIndex()); + if (splits.isEmpty()) { + // When there is no input splits, just return an empty source. + LOG.info("No input splits generate for MERGE_ON_READ input format. Returning empty collection"); + } + return splits; + case COPY_ON_WRITE: + return SplitUtils.baseFileOnlyHoodieSourceSplit(metaClient, storagePath, createFileIndex(), flinkConf.get(FlinkOptions.MERGE_TYPE)); + default: + throw new HoodieException("Unexpected table type: " + flinkConf.get(FlinkOptions.TABLE_TYPE)); + } + case FlinkOptions.QUERY_TYPE_READ_OPTIMIZED: + return SplitUtils.baseFileOnlyHoodieSourceSplit(metaClient, storagePath, createFileIndex(), flinkConf.get(FlinkOptions.MERGE_TYPE)); + case FlinkOptions.QUERY_TYPE_INCREMENTAL: { // Only do scan planning if nothing is restored from checkpoint state IncrementalInputSplits incrementalInputSplits = IncrementalInputSplits.builder() - .conf(scanContext.getConf()) - .path(scanContext.getPath()) - .rowType(scanContext.getRowType()) - .maxCompactionMemoryInBytes(scanContext.getMaxCompactionMemoryInBytes()) - .skipCompaction(scanContext.skipCompaction()) - .skipClustering(scanContext.skipClustering()) - .skipInsertOverwrite(scanContext.skipInsertOverwrite()).build(); - - HoodieContinuousSplitBatch batch = incrementalInputSplits.inputHoodieSourceSplits(metaClient, null, scanContext.cdcEnabled()); - splitProvider.onDiscoveredSplits(batch.getSplits()); + .conf(scanContext.getConf()) + .path(scanContext.getPath()) + .rowType(scanContext.getRowType()) + .maxCompactionMemoryInBytes(scanContext.getMaxCompactionMemoryInBytes()) + .skipCompaction(scanContext.skipCompaction()) + .skipClustering(scanContext.skipClustering()) + .partitionPruner(scanContext.partitionPruner()) + .skipInsertOverwrite(scanContext.skipInsertOverwrite()).build(); + return new ArrayList<>(incrementalInputSplits.inputHoodieSourceSplits(metaClient, null, scanContext.cdcEnabled()).getSplits()); } - - return new HoodieStaticSplitEnumerator(enumContext, splitProvider); + default: + throw new HoodieException("Unsupported query type: " + queryType); } } + + private FileIndex createFileIndex() { + return FileIndex.builder() + .path(this.storagePath) + .conf(this.scanContext.getConf()) + .rowType(scanContext.getRowType()) + .metaClient(metaClient) + .partitionPruner(scanContext.partitionPruner()) + .build(); Review Comment: the returned `FileIndex` is not closed, leading to resource leak -- 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]
