Refactoring HoodieTableFileSystemView using FileGroups/FileSlices
- Merged all filter* and get* methods - new constructor takes filestatus[] - All existing tests pass - FileGroup is all files that belong to a fileID within a partition - FileSlice is a generation of data and log files, starting at a base commit
This commit is contained in:
@@ -40,7 +40,6 @@ import java.util.Optional;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
@@ -306,7 +305,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
* @return
|
||||
*/
|
||||
private List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = getCompletedCommitTimeline();
|
||||
@@ -314,7 +312,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = getFileSystemView()
|
||||
.getLatestVersionInPartition(partitionPath, latestCommitTime.getTimestamp())
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (HoodieDataFile file : allFiles) {
|
||||
|
||||
Reference in New Issue
Block a user