-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-3206] Unify Hive's MOR implementations to avoid duplication #4559
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
329264c
9562f90
b4b95e5
c7b2c99
6c01a5e
9fdb92e
9bf09e0
5a3c7b2
16d9fc2
761fa9c
b2598cb
0a7342e
8b7dc5f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,10 +21,12 @@ | |
| import org.apache.hadoop.conf.Configurable; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.FileStatus; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.io.ArrayWritable; | ||
| import org.apache.hadoop.io.NullWritable; | ||
| import org.apache.hadoop.mapred.FileInputFormat; | ||
| import org.apache.hadoop.mapred.FileSplit; | ||
| import org.apache.hadoop.mapred.JobConf; | ||
| import org.apache.hadoop.mapreduce.Job; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
|
|
@@ -65,12 +67,32 @@ | |
| * <li>Incremental mode: reading table's state as of particular timestamp (or instant, in Hudi's terms)</li> | ||
| * <li>External mode: reading non-Hudi partitions</li> | ||
| * </ul> | ||
| * | ||
| * NOTE: This class is invariant of the underlying file-format of the files being read | ||
| */ | ||
| public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWritable, ArrayWritable> | ||
| implements Configurable { | ||
|
|
||
| protected Configuration conf; | ||
|
|
||
| @Nonnull | ||
| private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, Stream<HoodieLogFile> logFiles) { | ||
| List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); | ||
| FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); | ||
| try { | ||
| RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus); | ||
| rtFileStatus.setDeltaLogFiles(sortedLogFiles); | ||
| rtFileStatus.setBaseFilePath(baseFile.getPath()); | ||
| if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { | ||
| rtFileStatus.setBootStrapFileStatus(baseFileStatus); | ||
nsivabalan marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| return rtFileStatus; | ||
| } catch (IOException e) { | ||
| throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public final Configuration getConf() { | ||
| return conf; | ||
|
|
@@ -81,6 +103,24 @@ public final void setConf(Configuration conf) { | |
| this.conf = conf; | ||
| } | ||
|
|
||
| protected abstract boolean includeLogFilesForSnapshotView(); | ||
|
|
||
| @Override | ||
| protected boolean isSplitable(FileSystem fs, Path filename) { | ||
| return !(filename instanceof PathWithBootstrapFileStatus); | ||
| } | ||
|
|
||
| @Override | ||
| protected FileSplit makeSplit(Path file, long start, long length, | ||
| String[] hosts) { | ||
| FileSplit split = new FileSplit(file, start, length, hosts); | ||
|
|
||
| if (file instanceof PathWithBootstrapFileStatus) { | ||
| return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split); | ||
| } | ||
| return split; | ||
| } | ||
|
|
||
| @Override | ||
| public FileStatus[] listStatus(JobConf job) throws IOException { | ||
| // Segregate inputPaths[] to incremental, snapshot and non hoodie paths | ||
|
|
@@ -121,20 +161,6 @@ public FileStatus[] listStatus(JobConf job) throws IOException { | |
| return returns.toArray(new FileStatus[0]); | ||
| } | ||
|
|
||
| private void validate(List<FileStatus> targetFiles, List<FileStatus> legacyFileStatuses) { | ||
| List<FileStatus> diff = CollectionUtils.diff(targetFiles, legacyFileStatuses); | ||
| checkState(diff.isEmpty(), "Should be empty"); | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { | ||
| try { | ||
| return HoodieInputFormatUtils.getFileStatus(baseFile); | ||
| } catch (IOException ioe) { | ||
| throw new HoodieIOException("Failed to get file-status", ioe); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Abstracts and exposes {@link FileInputFormat#listStatus(JobConf)} operation to subclasses that | ||
| * lists files (returning an array of {@link FileStatus}) corresponding to the input paths specified | ||
|
|
@@ -172,35 +198,25 @@ protected List<FileStatus> listStatusForIncrementalMode(JobConf job, | |
| return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get()); | ||
| } | ||
|
|
||
| protected abstract boolean includeLogFilesForSnapshotView(); | ||
| @Override | ||
| protected FileSplit makeSplit(Path file, long start, long length, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: put two
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will clean this up in the final PR #4743 |
||
| String[] hosts, String[] inMemoryHosts) { | ||
| FileSplit split = new FileSplit(file, start, length, hosts, inMemoryHosts); | ||
| if (file instanceof PathWithBootstrapFileStatus) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you help me understand why don't I see diff FileStatuses here like RealtimeFileStatus for eg, but just PathWithBootstrapFileStatus?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My understanding is that these methods are moved from HoodieParquetInputFormat. @alexeykudinkin is the logic parquet agnostic and going to be reused by other file formats like HFile? I think if that's not the case, better to keep them in HoodieParquetInputFormat, since this base class is going to be general for different formats.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is used only for COW
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @alexeykudinkin what about my above question regarding whether the logic should reside in HoodieParquetInputFormat or this class?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @yihua GH comments are weird, when i was responding to @nsivabalan there were no comments of yours, and then it sandwiched your comment in b/w of those.
Your understanding is correct, all of this logic is file-format agnostic.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. sounds good!
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @alexeykudinkin no worries. That sg. |
||
| return makeExternalFileSplit((PathWithBootstrapFileStatus)file, split); | ||
| } | ||
| return split; | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, | ||
| Stream<HoodieLogFile> logFiles, | ||
| Option<HoodieInstant> latestCompletedInstantOpt, | ||
| HoodieTableMetaClient tableMetaClient) { | ||
| List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); | ||
| FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); | ||
| private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus file, FileSplit split) { | ||
| try { | ||
| RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus); | ||
| rtFileStatus.setDeltaLogFiles(sortedLogFiles); | ||
| rtFileStatus.setBaseFilePath(baseFile.getPath()); | ||
| rtFileStatus.setBasePath(tableMetaClient.getBasePath()); | ||
|
|
||
| if (latestCompletedInstantOpt.isPresent()) { | ||
| HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); | ||
| checkState(latestCompletedInstant.isCompleted()); | ||
|
|
||
| rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); | ||
| } | ||
|
|
||
| if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { | ||
| rtFileStatus.setBootStrapFileStatus(baseFileStatus); | ||
| } | ||
|
|
||
| return rtFileStatus; | ||
| LOG.info("Making external data split for " + file); | ||
| FileStatus externalFileStatus = file.getBootstrapFileStatus(); | ||
| FileSplit externalFileSplit = makeSplit(externalFileStatus.getPath(), 0, externalFileStatus.getLen(), | ||
| new String[0], new String[0]); | ||
| return new BootstrapBaseFileSplit(split, externalFileSplit); | ||
| } catch (IOException e) { | ||
| throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); | ||
| throw new HoodieIOException(e.getMessage(), e); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -209,38 +225,6 @@ private List<FileStatus> listStatusForSnapshotModeLegacy(JobConf job, Map<String | |
| return HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapshotView()); | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, | ||
| Stream<HoodieLogFile> logFiles, | ||
| Option<HoodieInstant> latestCompletedInstantOpt, | ||
| HoodieTableMetaClient tableMetaClient) { | ||
| List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); | ||
| try { | ||
| RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus()); | ||
| rtFileStatus.setDeltaLogFiles(sortedLogFiles); | ||
| rtFileStatus.setBasePath(tableMetaClient.getBasePath()); | ||
|
|
||
| if (latestCompletedInstantOpt.isPresent()) { | ||
| HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); | ||
| checkState(latestCompletedInstant.isCompleted()); | ||
|
|
||
| rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); | ||
| } | ||
|
|
||
| return rtFileStatus; | ||
| } catch (IOException e) { | ||
| throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); | ||
| } | ||
| } | ||
|
|
||
| private static Option<HoodieInstant> fromScala(scala.Option<HoodieInstant> opt) { | ||
| if (opt.isDefined()) { | ||
| return Option.of(opt.get()); | ||
| } | ||
|
|
||
| return Option.empty(); | ||
| } | ||
|
|
||
| @Nonnull | ||
| private List<FileStatus> listStatusForSnapshotMode(JobConf job, | ||
| Map<String, HoodieTableMetaClient> tableMetaClientMap, | ||
|
|
@@ -317,4 +301,80 @@ private List<FileStatus> listStatusForSnapshotMode(JobConf job, | |
|
|
||
| return targetFiles; | ||
| } | ||
|
|
||
| private void validate(List<FileStatus> targetFiles, List<FileStatus> legacyFileStatuses) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are the methods below merely moved? Looks like the order changes.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, GJF rule is applied (non-static methods) (static methods) |
||
| List<FileStatus> diff = CollectionUtils.diff(targetFiles, legacyFileStatuses); | ||
| checkState(diff.isEmpty(), "Should be empty"); | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { | ||
| try { | ||
| return HoodieInputFormatUtils.getFileStatus(baseFile); | ||
| } catch (IOException ioe) { | ||
| throw new HoodieIOException("Failed to get file-status", ioe); | ||
| } | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, | ||
| Stream<HoodieLogFile> logFiles, | ||
| Option<HoodieInstant> latestCompletedInstantOpt, | ||
| HoodieTableMetaClient tableMetaClient) { | ||
| List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); | ||
| FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); | ||
| try { | ||
| RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus); | ||
| rtFileStatus.setDeltaLogFiles(sortedLogFiles); | ||
| rtFileStatus.setBaseFilePath(baseFile.getPath()); | ||
| rtFileStatus.setBasePath(tableMetaClient.getBasePath()); | ||
|
|
||
| if (latestCompletedInstantOpt.isPresent()) { | ||
| HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); | ||
| checkState(latestCompletedInstant.isCompleted()); | ||
|
|
||
| rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); | ||
| } | ||
|
|
||
| if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { | ||
| rtFileStatus.setBootStrapFileStatus(baseFileStatus); | ||
| } | ||
|
|
||
| return rtFileStatus; | ||
| } catch (IOException e) { | ||
| throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); | ||
| } | ||
| } | ||
|
|
||
| @Nonnull | ||
| private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, | ||
| Stream<HoodieLogFile> logFiles, | ||
| Option<HoodieInstant> latestCompletedInstantOpt, | ||
| HoodieTableMetaClient tableMetaClient) { | ||
| List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); | ||
| try { | ||
| RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus()); | ||
| rtFileStatus.setDeltaLogFiles(sortedLogFiles); | ||
| rtFileStatus.setBasePath(tableMetaClient.getBasePath()); | ||
|
|
||
| if (latestCompletedInstantOpt.isPresent()) { | ||
| HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); | ||
| checkState(latestCompletedInstant.isCompleted()); | ||
|
|
||
| rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); | ||
| } | ||
|
|
||
| return rtFileStatus; | ||
| } catch (IOException e) { | ||
| throw new HoodieIOException(String.format("Failed to init %s", RealtimeFileStatus.class.getSimpleName()), e); | ||
| } | ||
| } | ||
|
|
||
| private static Option<HoodieInstant> fromScala(scala.Option<HoodieInstant> opt) { | ||
| if (opt.isDefined()) { | ||
| return Option.of(opt.get()); | ||
| } | ||
|
|
||
| return Option.empty(); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.