From 20140487964c1569ead043bbabf2f91185587134 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Wed, 2 Feb 2022 20:04:53 -0800 Subject: [PATCH 01/30] Cleaned up conditional that should not be hit; --- .../utils/HoodieRealtimeInputFormatUtils.java | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 7dc58d1e2f57e..04923385f7ec4 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -85,13 +85,9 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, List Option hoodieVirtualKeyInfoOpt = getHoodieVirtualKeyInfo(metaClient); - // NOTE: This timeline is kept in sync w/ {@code HoodieTableFileIndexBase} - HoodieInstant latestCommitInstant = - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant().get(); - InputSplit[] finalSplits = fileSplits.stream() .map(split -> { - // There are 4 types of splits could we have to handle here + // There are 4 types of splits we have to handle here // - {@code BootstrapBaseFileSplit}: in case base file does have associated bootstrap file, // but does NOT have any log files appended (convert it to {@code RealtimeBootstrapBaseFileSplit}) // - {@code RealtimeBootstrapBaseFileSplit}: in case base file does have associated bootstrap file @@ -101,18 +97,13 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, List // - {@code FileSplit}: in case Hive passed down non-Hudi path if (split instanceof RealtimeBootstrapBaseFileSplit) { return split; - } else if (split instanceof BootstrapBaseFileSplit) { - BootstrapBaseFileSplit bootstrapBaseFileSplit = unsafeCast(split); - return createRealtimeBoostrapBaseFileSplit( - bootstrapBaseFileSplit, - metaClient.getBasePath(), - Collections.emptyList(), - latestCommitInstant.getTimestamp(), - false); } else if (split instanceof BaseFileWithLogsSplit) { BaseFileWithLogsSplit baseFileWithLogsSplit = unsafeCast(split); return createHoodieRealtimeSplitUnchecked(baseFileWithLogsSplit, hoodieVirtualKeyInfoOpt); } else { + // TODO cleanup + checkState(!(split instanceof BootstrapBaseFileSplit)); + // Non-Hudi paths might result in just generic {@code FileSplit} being // propagated up to this point return split; From 17a72c874423d26f81c165ba72e2eb33f0003099 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 11:22:31 -0800 Subject: [PATCH 02/30] Made `BaseFileWithLogsSplit` implement `RealtimeSplit`; Moved `BaseFileWithLogsSplit` under `realtime` package --- .../hudi/hadoop/PathWithLogFilePath.java | 4 +-- .../{ => realtime}/BaseFileWithLogsSplit.java | 32 ++++++++++++++--- .../hudi/hadoop/realtime/RealtimeSplit.java | 36 ++++++++++--------- .../utils/HoodieRealtimeInputFormatUtils.java | 3 +- 4 files changed, 50 insertions(+), 25 deletions(-) rename hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/{ => realtime}/BaseFileWithLogsSplit.java (82%) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java index 7983e09290180..502072f32a935 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java @@ -18,9 +18,9 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.model.HoodieLogFile; - import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.hadoop.realtime.BaseFileWithLogsSplit; import java.util.ArrayList; import java.util.List; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java similarity index 82% rename from hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java rename to hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java index d0b168f29f75e..449fc861d3bab 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BaseFileWithLogsSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.hudi.hadoop; - -import org.apache.hudi.common.model.HoodieLogFile; +package org.apache.hudi.hadoop.realtime; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.FileSplit; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; import java.io.DataInput; import java.io.DataOutput; @@ -34,7 +34,7 @@ * Encode additional information in split to track matching log file and base files. * Hence, this class tracks a log/base file split. */ -public class BaseFileWithLogsSplit extends FileSplit { +public class BaseFileWithLogsSplit extends FileSplit implements RealtimeSplit { // a flag to mark this split is produced by incremental query or not. private boolean belongsToIncrementalQuery = false; // the log file paths of this split. @@ -46,6 +46,8 @@ public class BaseFileWithLogsSplit extends FileSplit { // the base file belong to this split. private String baseFilePath = ""; + private Option virtualKeyInfo = Option.empty(); + public BaseFileWithLogsSplit(Path file, long start, long length, String[] hosts) { super(file, start, length, hosts); } @@ -89,30 +91,46 @@ public void setBelongsToIncrementalQuery(boolean belongsToIncrementalQuery) { this.belongsToIncrementalQuery = belongsToIncrementalQuery; } + @Override public List getDeltaLogFiles() { return deltaLogFiles; } + @Override public void setDeltaLogFiles(List deltaLogFiles) { this.deltaLogFiles = deltaLogFiles; } + @Override public String getMaxCommitTime() { return maxCommitTime; } + @Override public void setMaxCommitTime(String maxCommitTime) { this.maxCommitTime = maxCommitTime; } + @Override public String getBasePath() { return basePath; } + @Override public void setBasePath(String basePath) { this.basePath = basePath; } + @Override + public Option getHoodieVirtualKeyInfo() { + return virtualKeyInfo; + } + + @Override + public void setHoodieVirtualKeyInfo(Option hoodieVirtualKeyInfo) { + this.virtualKeyInfo = hoodieVirtualKeyInfo; + } + public String getBaseFilePath() { return baseFilePath; } @@ -120,4 +138,10 @@ public String getBaseFilePath() { public void setBaseFilePath(String baseFilePath) { this.baseFilePath = baseFilePath; } + + @Override + public String toString() { + return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + getDeltaLogPaths() + + ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}'; + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java index 6c1e02cf64bbf..6be7172db8369 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java @@ -18,18 +18,18 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.InputSplitWithLocationInfo; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.InputSplitUtils; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.InputSplitWithLocationInfo; - import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; /** * Realtime Input Split Interface. @@ -40,10 +40,14 @@ public interface RealtimeSplit extends InputSplitWithLocationInfo { * Return Log File Paths. * @return */ - List getDeltaLogPaths(); + default List getDeltaLogPaths() { + return getDeltaLogFiles().stream().map(entry -> entry.getPath().toString()).collect(Collectors.toList()); + } List getDeltaLogFiles(); + void setDeltaLogFiles(List deltaLogFiles); + /** * Return Max Instant Time. * @return @@ -62,13 +66,6 @@ public interface RealtimeSplit extends InputSplitWithLocationInfo { */ Option getHoodieVirtualKeyInfo(); - /** - * Update Log File Paths. - * - * @param deltaLogPaths - */ - void setDeltaLogPaths(List deltaLogPaths); - /** * Update Maximum valid instant time. * @param maxCommitTime @@ -86,9 +83,10 @@ public interface RealtimeSplit extends InputSplitWithLocationInfo { default void writeToOutput(DataOutput out) throws IOException { InputSplitUtils.writeString(getBasePath(), out); InputSplitUtils.writeString(getMaxCommitTime(), out); - out.writeInt(getDeltaLogPaths().size()); - for (String logFilePath : getDeltaLogPaths()) { - InputSplitUtils.writeString(logFilePath, out); + out.writeInt(getDeltaLogFiles().size()); + for (HoodieLogFile logFile : getDeltaLogFiles()) { + InputSplitUtils.writeString(logFile.getPath().toString(), out); + out.writeLong(logFile.getFileSize()); } Option virtualKeyInfoOpt = getHoodieVirtualKeyInfo(); @@ -106,12 +104,16 @@ default void writeToOutput(DataOutput out) throws IOException { default void readFromInput(DataInput in) throws IOException { setBasePath(InputSplitUtils.readString(in)); setMaxCommitTime(InputSplitUtils.readString(in)); + int totalLogFiles = in.readInt(); - List deltaLogPaths = new ArrayList<>(totalLogFiles); + List deltaLogPaths = new ArrayList<>(totalLogFiles); for (int i = 0; i < totalLogFiles; i++) { - deltaLogPaths.add(InputSplitUtils.readString(in)); + String logFilePath = InputSplitUtils.readString(in); + long logFileSize = in.readLong(); + deltaLogPaths.add(new HoodieLogFile(new Path(logFilePath), logFileSize)); } - setDeltaLogPaths(deltaLogPaths); + setDeltaLogFiles(deltaLogPaths); + boolean hoodieVirtualKeyPresent = InputSplitUtils.readBoolean(in); if (hoodieVirtualKeyPresent) { String recordKeyField = InputSplitUtils.readString(in); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 04923385f7ec4..642baaa4a4306 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -43,8 +43,8 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.BaseFileWithLogsSplit; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; +import org.apache.hudi.hadoop.realtime.BaseFileWithLogsSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit; @@ -56,7 +56,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; From 258bd276e74db2f6140ac96d9915a16adc21d05e Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 11:39:49 -0800 Subject: [PATCH 03/30] Tidying up `HoodieRealtimeFileSplit` --- .../realtime/BaseFileWithLogsSplit.java | 6 +- .../HoodieParquetRealtimeInputFormat.java | 2 +- .../realtime/HoodieRealtimeFileSplit.java | 60 +++++++++++-------- .../RealtimeCompactedRecordReader.java | 17 +++--- .../hudi/hadoop/realtime/RealtimeSplit.java | 14 ++--- .../utils/HoodieRealtimeInputFormatUtils.java | 2 +- 6 files changed, 54 insertions(+), 47 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java index 449fc861d3bab..9f63d8d4bbaa7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java @@ -122,13 +122,13 @@ public void setBasePath(String basePath) { } @Override - public Option getHoodieVirtualKeyInfo() { + public Option getVirtualKeyInfo() { return virtualKeyInfo; } @Override - public void setHoodieVirtualKeyInfo(Option hoodieVirtualKeyInfo) { - this.virtualKeyInfo = hoodieVirtualKeyInfo; + public void setVirtualKeyInfo(Option virtualKeyInfo) { + this.virtualKeyInfo = virtualKeyInfo; } public String getBaseFilePath() { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java index e204c3b075e6a..e8c806ed2cf67 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieParquetRealtimeInputFormat.java @@ -94,7 +94,7 @@ void addProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf job // TO fix this, hoodie columns are appended late at the time record-reader gets built instead of construction // time. if (!realtimeSplit.getDeltaLogPaths().isEmpty()) { - HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf, realtimeSplit.getHoodieVirtualKeyInfo()); + HoodieRealtimeInputFormatUtils.addRequiredProjectionFields(jobConf, realtimeSplit.getVirtualKeyInfo()); } jobConf.set(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP, "true"); setConf(jobConf); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 2b45fe3f3c324..b9da3b56a3bfd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -18,56 +18,68 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; -import org.apache.hadoop.mapred.FileSplit; - import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; /** - * Filesplit that wraps the base split and a list of log files to merge deltas from. + * {@link FileSplit} implementation that holds + *
    + *
  1. Split corresponding to the base file
  2. + *
  3. List of {@link HoodieLogFile} that holds the delta to be merged (upon reading)
  4. + *
+ * + * This split is correspondent to a single file-slice in the Hudi terminology. * * NOTE: If you're adding fields here you need to make sure that you appropriately de-/serialize them * in {@link #readFromInput(DataInput)} and {@link #writeToOutput(DataOutput)} */ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { - - private List deltaLogPaths; + /** + * List of delta log-files holding updated records + */ private List deltaLogFiles = new ArrayList<>(); - + /** + * Latest commit instant available at the time of the query in which all of the files + * pertaining to this split are represented + */ private String maxCommitTime; - + /** + * Base file's path + */ private String basePath; - - private Option hoodieVirtualKeyInfo = Option.empty(); + /** + * Virtual key configuration of the table this split belongs to + */ + private Option virtualKeyInfo = Option.empty(); public HoodieRealtimeFileSplit() {} public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxCommitTime, - Option hoodieVirtualKeyInfo) + Option virtualKeyInfo) throws IOException { super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); this.deltaLogFiles = deltaLogFiles; - this.deltaLogPaths = deltaLogFiles.stream().map(entry -> entry.getPath().toString()).collect(Collectors.toList()); this.maxCommitTime = maxCommitTime; this.basePath = basePath; - this.hoodieVirtualKeyInfo = hoodieVirtualKeyInfo; - } - - public List getDeltaLogPaths() { - return deltaLogPaths; + this.virtualKeyInfo = virtualKeyInfo; } public List getDeltaLogFiles() { return deltaLogFiles; } + @Override + public void setDeltaLogFiles(List deltaLogFiles) { + this.deltaLogFiles = deltaLogFiles; + } + public String getMaxCommitTime() { return maxCommitTime; } @@ -77,17 +89,13 @@ public String getBasePath() { } @Override - public void setHoodieVirtualKeyInfo(Option hoodieVirtualKeyInfo) { - this.hoodieVirtualKeyInfo = hoodieVirtualKeyInfo; + public void setVirtualKeyInfo(Option virtualKeyInfo) { + this.virtualKeyInfo = virtualKeyInfo; } @Override - public Option getHoodieVirtualKeyInfo() { - return hoodieVirtualKeyInfo; - } - - public void setDeltaLogPaths(List deltaLogPaths) { - this.deltaLogPaths = deltaLogPaths; + public Option getVirtualKeyInfo() { + return virtualKeyInfo; } public void setMaxCommitTime(String maxCommitTime) { @@ -112,7 +120,7 @@ public void readFields(DataInput in) throws IOException { @Override public String toString() { - return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + deltaLogPaths + return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + getDeltaLogPaths() + ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}'; } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index d35df9f33776e..7d92b340e5693 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -18,6 +18,12 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.fs.FSUtils; @@ -29,13 +35,6 @@ import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; - -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -65,8 +64,8 @@ public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, this.mergedLogRecordScanner = getMergedLogRecordScanner(); this.deltaRecordMap = mergedLogRecordScanner.getRecords(); this.deltaRecordKeys = new HashSet<>(this.deltaRecordMap.keySet()); - if (split.getHoodieVirtualKeyInfo().isPresent()) { - this.recordKeyIndex = split.getHoodieVirtualKeyInfo().get().getRecordKeyFieldIndex(); + if (split.getVirtualKeyInfo().isPresent()) { + this.recordKeyIndex = split.getVirtualKeyInfo().get().getRecordKeyFieldIndex(); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java index 6be7172db8369..3a04fb11fa1d7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java @@ -64,7 +64,7 @@ default List getDeltaLogPaths() { * Returns Virtual key info if meta fields are disabled. * @return */ - Option getHoodieVirtualKeyInfo(); + Option getVirtualKeyInfo(); /** * Update Maximum valid instant time. @@ -78,7 +78,7 @@ default List getDeltaLogPaths() { */ void setBasePath(String basePath); - void setHoodieVirtualKeyInfo(Option hoodieVirtualKeyInfo); + void setVirtualKeyInfo(Option virtualKeyInfo); default void writeToOutput(DataOutput out) throws IOException { InputSplitUtils.writeString(getBasePath(), out); @@ -89,7 +89,7 @@ default void writeToOutput(DataOutput out) throws IOException { out.writeLong(logFile.getFileSize()); } - Option virtualKeyInfoOpt = getHoodieVirtualKeyInfo(); + Option virtualKeyInfoOpt = getVirtualKeyInfo(); if (!virtualKeyInfoOpt.isPresent()) { InputSplitUtils.writeBoolean(false, out); } else { @@ -120,22 +120,22 @@ default void readFromInput(DataInput in) throws IOException { String partitionPathField = InputSplitUtils.readString(in); int recordFieldIndex = Integer.parseInt(InputSplitUtils.readString(in)); int partitionPathIndex = Integer.parseInt(InputSplitUtils.readString(in)); - setHoodieVirtualKeyInfo(Option.of(new HoodieVirtualKeyInfo(recordKeyField, partitionPathField, recordFieldIndex, partitionPathIndex))); + setVirtualKeyInfo(Option.of(new HoodieVirtualKeyInfo(recordKeyField, partitionPathField, recordFieldIndex, partitionPathIndex))); } } /** * The file containing this split's data. */ - public Path getPath(); + Path getPath(); /** * The position of the first byte in the file to process. */ - public long getStart(); + long getStart(); /** * The number of bytes in the file to process. */ - public long getLength(); + long getLength(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 642baaa4a4306..98d3c0faab251 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -372,7 +372,7 @@ public static boolean requiredProjectionFieldsExistInConf(Configuration configur public static boolean canAddProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf jobConf) { return jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null - || (!realtimeSplit.getDeltaLogPaths().isEmpty() && !HoodieRealtimeInputFormatUtils.requiredProjectionFieldsExistInConf(jobConf, realtimeSplit.getHoodieVirtualKeyInfo())); + || (!realtimeSplit.getDeltaLogPaths().isEmpty() && !HoodieRealtimeInputFormatUtils.requiredProjectionFieldsExistInConf(jobConf, realtimeSplit.getVirtualKeyInfo())); } /** From a13d3ac26053935797755557b57af86ccfc6b245 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 11:43:53 -0800 Subject: [PATCH 04/30] `PathWithLogFilePath` > `RealtimePath`; --- .../hudi/hadoop/RealtimeFileStatus.java | 20 +++---- .../HoodieMergeOnReadTableInputFormat.java | 15 +++--- .../RealtimePath.java} | 8 +-- .../TestHoodieRealtimeRecordReader.java | 53 +++++++++---------- 4 files changed, 46 insertions(+), 50 deletions(-) rename hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/{PathWithLogFilePath.java => realtime/RealtimePath.java} (94%) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index 1d732f5a612a3..9640a7008c3a1 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -18,10 +18,10 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.model.HoodieLogFile; - import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.hadoop.realtime.RealtimePath; import java.io.IOException; import java.util.ArrayList; @@ -55,16 +55,16 @@ public RealtimeFileStatus(FileStatus fileStatus) throws IOException { @Override public Path getPath() { Path path = super.getPath(); - PathWithLogFilePath pathWithLogFilePath = new PathWithLogFilePath(path.getParent(), path.getName()); - pathWithLogFilePath.setBelongsToIncrementalPath(belongToIncrementalFileStatus); - pathWithLogFilePath.setDeltaLogFiles(deltaLogFiles); - pathWithLogFilePath.setMaxCommitTime(maxCommitTime); - pathWithLogFilePath.setBasePath(basePath); - pathWithLogFilePath.setBaseFilePath(baseFilePath); + RealtimePath realtimePath = new RealtimePath(path.getParent(), path.getName()); + realtimePath.setBelongsToIncrementalPath(belongToIncrementalFileStatus); + realtimePath.setDeltaLogFiles(deltaLogFiles); + realtimePath.setMaxCommitTime(maxCommitTime); + realtimePath.setBasePath(basePath); + realtimePath.setBaseFilePath(baseFilePath); if (bootStrapFileStatus != null) { - pathWithLogFilePath.setPathWithBootstrapFileStatus((PathWithBootstrapFileStatus)bootStrapFileStatus.getPath()); + realtimePath.setPathWithBootstrapFileStatus((PathWithBootstrapFileStatus)bootStrapFileStatus.getPath()); } - return pathWithLogFilePath; + return realtimePath; } public void setBelongToIncrementalFileStatus(boolean belongToIncrementalFileStatus) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 7b482f4155de3..58dc797903503 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -41,7 +41,6 @@ import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; -import org.apache.hudi.hadoop.PathWithLogFilePath; import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; @@ -165,8 +164,8 @@ protected boolean includeLogFilesForSnapshotView() { @Override protected boolean isSplitable(FileSystem fs, Path filename) { - if (filename instanceof PathWithLogFilePath) { - return ((PathWithLogFilePath)filename).splitable(); + if (filename instanceof RealtimePath) { + return ((RealtimePath)filename).splitable(); } return super.isSplitable(fs, filename); @@ -177,16 +176,16 @@ protected boolean isSplitable(FileSystem fs, Path filename) { // PathWithLogFilePath, so those bootstrap files should be processed int this function. @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts) { - if (file instanceof PathWithLogFilePath) { - return doMakeSplitForPathWithLogFilePath((PathWithLogFilePath) file, start, length, hosts, null); + if (file instanceof RealtimePath) { + return doMakeSplitForPathWithLogFilePath((RealtimePath) file, start, length, hosts, null); } return super.makeSplit(file, start, length, hosts); } @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) { - if (file instanceof PathWithLogFilePath) { - return doMakeSplitForPathWithLogFilePath((PathWithLogFilePath) file, start, length, hosts, inMemoryHosts); + if (file instanceof RealtimePath) { + return doMakeSplitForPathWithLogFilePath((RealtimePath) file, start, length, hosts, inMemoryHosts); } return super.makeSplit(file, start, length, hosts, inMemoryHosts); } @@ -235,7 +234,7 @@ private List collectAllIncrementalFiles(List fileGr return result; } - private FileSplit doMakeSplitForPathWithLogFilePath(PathWithLogFilePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { + private FileSplit doMakeSplitForPathWithLogFilePath(RealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { if (!path.includeBootstrapFilePath()) { return path.buildSplit(path, start, length, hosts); } else { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java similarity index 94% rename from hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java rename to hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java index 502072f32a935..3b96cd730cac5 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/PathWithLogFilePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.hudi.hadoop; +package org.apache.hudi.hadoop.realtime; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.hadoop.realtime.BaseFileWithLogsSplit; +import org.apache.hudi.hadoop.PathWithBootstrapFileStatus; import java.util.ArrayList; import java.util.List; @@ -29,7 +29,7 @@ * Encode additional information in Path to track matching log file and base files. * Hence, this class tracks a log/base file status. */ -public class PathWithLogFilePath extends Path { +public class RealtimePath extends Path { // a flag to mark this split is produced by incremental query or not. private boolean belongsToIncrementalPath = false; // the log files belong this path. @@ -44,7 +44,7 @@ public class PathWithLogFilePath extends Path { // only if current query table is bootstrap table, this field is used. private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; - public PathWithLogFilePath(Path parent, String child) { + public RealtimePath(Path parent, String child) { super(parent, child); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index f8daf70542053..6c25684b32917 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -18,8 +18,28 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +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.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; import org.apache.hadoop.mapred.Reporter; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieCompactionPlan; @@ -30,8 +50,8 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; import org.apache.hudi.common.table.log.block.HoodieLogBlock; @@ -44,32 +64,9 @@ import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hadoop.BaseFileWithLogsSplit; -import org.apache.hudi.hadoop.PathWithLogFilePath; import org.apache.hudi.hadoop.RealtimeFileStatus; -import org.apache.hudi.hadoop.testutils.InputFormatTestUtil; import org.apache.hudi.hadoop.config.HoodieRealtimeConfig; - -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.BooleanWritable; -import org.apache.hadoop.io.DoubleWritable; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; +import org.apache.hudi.hadoop.testutils.InputFormatTestUtil; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -84,12 +81,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; -import java.util.Map; -import java.util.HashMap; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -773,7 +770,7 @@ public void testLogOnlyReader() throws Exception { realtimeFileStatus.setMaxCommitTime(instantTime); realtimeFileStatus.setBasePath(basePath.toString()); realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); - PathWithLogFilePath pathWithLogFileStatus = (PathWithLogFilePath) realtimeFileStatus.getPath(); + RealtimePath pathWithLogFileStatus = (RealtimePath) realtimeFileStatus.getPath(); BaseFileWithLogsSplit bs = pathWithLogFileStatus.buildSplit(pathWithLogFileStatus, 0, 0, new String[] {""}); HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), Option.empty()); From cf87b0eea46e8b76137a6084c95905091d018320 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 11:58:44 -0800 Subject: [PATCH 05/30] Refactored `RealtimePath` to produce `HoodieRealtimeFileSplit` --- .../HoodieMergeOnReadTableInputFormat.java | 2 +- .../realtime/HoodieRealtimeFileSplit.java | 12 +++++- .../hudi/hadoop/realtime/RealtimePath.java | 42 ++++++++++--------- .../TestHoodieRealtimeRecordReader.java | 5 +-- 4 files changed, 36 insertions(+), 25 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 58dc797903503..4c23dc24b3eab 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -165,7 +165,7 @@ protected boolean includeLogFilesForSnapshotView() { @Override protected boolean isSplitable(FileSystem fs, Path filename) { if (filename instanceof RealtimePath) { - return ((RealtimePath)filename).splitable(); + return ((RealtimePath)filename).isSplitable(); } return super.isSplitable(fs, filename); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index b9da3b56a3bfd..924bfd649b0b3 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -18,6 +18,7 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; @@ -42,7 +43,7 @@ */ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { /** - * List of delta log-files holding updated records + * List of delta log-files holding updated records for this base-file */ private List deltaLogFiles = new ArrayList<>(); /** @@ -51,7 +52,7 @@ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit */ private String maxCommitTime; /** - * Base file's path + * Base path of the table this path belongs to */ private String basePath; /** @@ -61,6 +62,13 @@ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit public HoodieRealtimeFileSplit() {} + HoodieRealtimeFileSplit(Path file, long start, long length, String[] hosts) { + super(file, start, length, hosts); + } + + /** + * @deprecated + */ public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxCommitTime, Option virtualKeyInfo) throws IOException { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java index 3b96cd730cac5..7c1384d7ad336 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java @@ -26,22 +26,31 @@ import java.util.List; /** - * Encode additional information in Path to track matching log file and base files. - * Hence, this class tracks a log/base file status. + * {@link Path} implementation encoding additional information necessary to appropriately read + * base files of the MOR tables, such as list of delta log files (holding updated records) associated + * w/ the base file, etc. */ public class RealtimePath extends Path { - // a flag to mark this split is produced by incremental query or not. + /** + * Marks whether this path produced as part of Incremental Query + */ private boolean belongsToIncrementalPath = false; - // the log files belong this path. + /** + * List of delta log-files holding updated records for this base-file + */ private List deltaLogFiles = new ArrayList<>(); - // max commit time of current path. + /** + * Latest commit instant available at the time of the query in which all of the files + * pertaining to this split are represented + */ private String maxCommitTime = ""; - // the basePath of current hoodie table. + /** + * Base path of the table this path belongs to + */ private String basePath = ""; - // the base file belong to this path; - private String baseFilePath = ""; - // the bootstrap file belong to this path. - // only if current query table is bootstrap table, this field is used. + /** + * File status for the Bootstrap file (only relevant if this table is a bootstrapped table + */ private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; public RealtimePath(Path parent, String child) { @@ -80,12 +89,8 @@ public void setBasePath(String basePath) { this.basePath = basePath; } - public void setBaseFilePath(String baseFilePath) { - this.baseFilePath = baseFilePath; - } - - public boolean splitable() { - return !baseFilePath.isEmpty(); + public boolean isSplitable() { + return !toString().isEmpty(); } public PathWithBootstrapFileStatus getPathWithBootstrapFileStatus() { @@ -100,13 +105,12 @@ public boolean includeBootstrapFilePath() { return pathWithBootstrapFileStatus != null; } - public BaseFileWithLogsSplit buildSplit(Path file, long start, long length, String[] hosts) { - BaseFileWithLogsSplit bs = new BaseFileWithLogsSplit(file, start, length, hosts); + public HoodieRealtimeFileSplit buildSplit(Path file, long start, long length, String[] hosts) { + HoodieRealtimeFileSplit bs = new HoodieRealtimeFileSplit(file, start, length, hosts); bs.setBelongsToIncrementalQuery(belongsToIncrementalPath); bs.setDeltaLogFiles(deltaLogFiles); bs.setMaxCommitTime(maxCommitTime); bs.setBasePath(basePath); - bs.setBaseFilePath(baseFilePath); return bs; } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 6c25684b32917..1a7a64c65c78b 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -770,9 +770,8 @@ public void testLogOnlyReader() throws Exception { realtimeFileStatus.setMaxCommitTime(instantTime); realtimeFileStatus.setBasePath(basePath.toString()); realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); - RealtimePath pathWithLogFileStatus = (RealtimePath) realtimeFileStatus.getPath(); - BaseFileWithLogsSplit bs = pathWithLogFileStatus.buildSplit(pathWithLogFileStatus, 0, 0, new String[] {""}); - HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), Option.empty()); + RealtimePath realtimePath = (RealtimePath) realtimeFileStatus.getPath(); + HoodieRealtimeFileSplit split = realtimePath.buildSplit(realtimePath, 0, 0, new String[] {""}); JobConf newJobConf = new JobConf(baseJobConf); List fields = schema.getFields(); From 4770a891a2a097a6a46f8bb305bdc8d0496297d6 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 11:59:33 -0800 Subject: [PATCH 06/30] Prefix w/ "Hoodie" --- .../org/apache/hudi/hadoop/RealtimeFileStatus.java | 4 ++-- .../HoodieMergeOnReadTableInputFormat.java | 14 +++++++------- .../{RealtimePath.java => HoodieRealtimePath.java} | 4 ++-- .../realtime/TestHoodieRealtimeRecordReader.java | 2 +- 4 files changed, 12 insertions(+), 12 deletions(-) rename hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/{RealtimePath.java => HoodieRealtimePath.java} (97%) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index 9640a7008c3a1..bd9eb004e2561 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.hadoop.realtime.RealtimePath; +import org.apache.hudi.hadoop.realtime.HoodieRealtimePath; import java.io.IOException; import java.util.ArrayList; @@ -55,7 +55,7 @@ public RealtimeFileStatus(FileStatus fileStatus) throws IOException { @Override public Path getPath() { Path path = super.getPath(); - RealtimePath realtimePath = new RealtimePath(path.getParent(), path.getName()); + HoodieRealtimePath realtimePath = new HoodieRealtimePath(path.getParent(), path.getName()); realtimePath.setBelongsToIncrementalPath(belongToIncrementalFileStatus); realtimePath.setDeltaLogFiles(deltaLogFiles); realtimePath.setMaxCommitTime(maxCommitTime); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 4c23dc24b3eab..4e96c4f3886a0 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -164,8 +164,8 @@ protected boolean includeLogFilesForSnapshotView() { @Override protected boolean isSplitable(FileSystem fs, Path filename) { - if (filename instanceof RealtimePath) { - return ((RealtimePath)filename).isSplitable(); + if (filename instanceof HoodieRealtimePath) { + return ((HoodieRealtimePath)filename).isSplitable(); } return super.isSplitable(fs, filename); @@ -176,16 +176,16 @@ protected boolean isSplitable(FileSystem fs, Path filename) { // PathWithLogFilePath, so those bootstrap files should be processed int this function. @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts) { - if (file instanceof RealtimePath) { - return doMakeSplitForPathWithLogFilePath((RealtimePath) file, start, length, hosts, null); + if (file instanceof HoodieRealtimePath) { + return doMakeSplitForPathWithLogFilePath((HoodieRealtimePath) file, start, length, hosts, null); } return super.makeSplit(file, start, length, hosts); } @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) { - if (file instanceof RealtimePath) { - return doMakeSplitForPathWithLogFilePath((RealtimePath) file, start, length, hosts, inMemoryHosts); + if (file instanceof HoodieRealtimePath) { + return doMakeSplitForPathWithLogFilePath((HoodieRealtimePath) file, start, length, hosts, inMemoryHosts); } return super.makeSplit(file, start, length, hosts, inMemoryHosts); } @@ -234,7 +234,7 @@ private List collectAllIncrementalFiles(List fileGr return result; } - private FileSplit doMakeSplitForPathWithLogFilePath(RealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { + private FileSplit doMakeSplitForPathWithLogFilePath(HoodieRealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { if (!path.includeBootstrapFilePath()) { return path.buildSplit(path, start, length, hosts); } else { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java similarity index 97% rename from hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java rename to hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index 7c1384d7ad336..18c230d0cff68 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -30,7 +30,7 @@ * base files of the MOR tables, such as list of delta log files (holding updated records) associated * w/ the base file, etc. */ -public class RealtimePath extends Path { +public class HoodieRealtimePath extends Path { /** * Marks whether this path produced as part of Incremental Query */ @@ -53,7 +53,7 @@ public class RealtimePath extends Path { */ private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; - public RealtimePath(Path parent, String child) { + public HoodieRealtimePath(Path parent, String child) { super(parent, child); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 1a7a64c65c78b..00a579fcd37d7 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -770,7 +770,7 @@ public void testLogOnlyReader() throws Exception { realtimeFileStatus.setMaxCommitTime(instantTime); realtimeFileStatus.setBasePath(basePath.toString()); realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); - RealtimePath realtimePath = (RealtimePath) realtimeFileStatus.getPath(); + HoodieRealtimePath realtimePath = (HoodieRealtimePath) realtimeFileStatus.getPath(); HoodieRealtimeFileSplit split = realtimePath.buildSplit(realtimePath, 0, 0, new String[] {""}); JobConf newJobConf = new JobConf(baseJobConf); From f0a86b5394f54071d205ba66ead4da9343640af7 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 12:02:18 -0800 Subject: [PATCH 07/30] Fixed `HoodieRealtimeFileSplit` to bear `belongsToIncrementalQuery` flag --- .../hudi/hadoop/RealtimeFileStatus.java | 3 +-- .../realtime/HoodieRealtimeFileSplit.java | 20 +++++++++++++++---- .../hadoop/realtime/HoodieRealtimePath.java | 18 ++++++++--------- 3 files changed, 26 insertions(+), 15 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index bd9eb004e2561..100d28bc49152 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -56,11 +56,10 @@ public RealtimeFileStatus(FileStatus fileStatus) throws IOException { public Path getPath() { Path path = super.getPath(); HoodieRealtimePath realtimePath = new HoodieRealtimePath(path.getParent(), path.getName()); - realtimePath.setBelongsToIncrementalPath(belongToIncrementalFileStatus); + realtimePath.setBelongsToIncrementalQuery(belongToIncrementalFileStatus); realtimePath.setDeltaLogFiles(deltaLogFiles); realtimePath.setMaxCommitTime(maxCommitTime); realtimePath.setBasePath(basePath); - realtimePath.setBaseFilePath(baseFilePath); if (bootStrapFileStatus != null) { realtimePath.setPathWithBootstrapFileStatus((PathWithBootstrapFileStatus)bootStrapFileStatus.getPath()); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 924bfd649b0b3..76a8a9fc35a97 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -42,6 +42,10 @@ * in {@link #readFromInput(DataInput)} and {@link #writeToOutput(DataOutput)} */ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { + /** + * Marks whether this path produced as part of Incremental Query + */ + private boolean belongsToIncrementalQuery = false; /** * List of delta log-files holding updated records for this base-file */ @@ -92,10 +96,18 @@ public String getMaxCommitTime() { return maxCommitTime; } + public void setMaxCommitTime(String maxCommitTime) { + this.maxCommitTime = maxCommitTime; + } + public String getBasePath() { return basePath; } + public void setBasePath(String basePath) { + this.basePath = basePath; + } + @Override public void setVirtualKeyInfo(Option virtualKeyInfo) { this.virtualKeyInfo = virtualKeyInfo; @@ -106,12 +118,12 @@ public Option getVirtualKeyInfo() { return virtualKeyInfo; } - public void setMaxCommitTime(String maxCommitTime) { - this.maxCommitTime = maxCommitTime; + public boolean getBelongsToIncrementalQuery() { + return belongsToIncrementalQuery; } - public void setBasePath(String basePath) { - this.basePath = basePath; + public void setBelongsToIncrementalQuery(boolean belongsToIncrementalPath) { + this.belongsToIncrementalQuery = belongsToIncrementalPath; } @Override diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index 18c230d0cff68..ebd952a288a15 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -34,7 +34,7 @@ public class HoodieRealtimePath extends Path { /** * Marks whether this path produced as part of Incremental Query */ - private boolean belongsToIncrementalPath = false; + private boolean belongsToIncrementalQuery = false; /** * List of delta log-files holding updated records for this base-file */ @@ -57,10 +57,6 @@ public HoodieRealtimePath(Path parent, String child) { super(parent, child); } - public void setBelongsToIncrementalPath(boolean belongsToIncrementalPath) { - this.belongsToIncrementalPath = belongsToIncrementalPath; - } - public List getDeltaLogFiles() { return deltaLogFiles; } @@ -81,12 +77,16 @@ public String getBasePath() { return basePath; } + public void setBasePath(String basePath) { + this.basePath = basePath; + } + public boolean getBelongsToIncrementalQuery() { - return belongsToIncrementalPath; + return belongsToIncrementalQuery; } - public void setBasePath(String basePath) { - this.basePath = basePath; + public void setBelongsToIncrementalQuery(boolean belongsToIncrementalQuery) { + this.belongsToIncrementalQuery = belongsToIncrementalQuery; } public boolean isSplitable() { @@ -107,7 +107,7 @@ public boolean includeBootstrapFilePath() { public HoodieRealtimeFileSplit buildSplit(Path file, long start, long length, String[] hosts) { HoodieRealtimeFileSplit bs = new HoodieRealtimeFileSplit(file, start, length, hosts); - bs.setBelongsToIncrementalQuery(belongsToIncrementalPath); + bs.setBelongsToIncrementalQuery(belongsToIncrementalQuery); bs.setDeltaLogFiles(deltaLogFiles); bs.setMaxCommitTime(maxCommitTime); bs.setBasePath(basePath); From 6ae1a7108137d629f50169b631fda4dcf6463bb4 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 13:45:58 -0800 Subject: [PATCH 08/30] Tidying up --- .../hudi/hadoop/BootstrapBaseFileSplit.java | 4 +- .../HoodieCopyOnWriteTableInputFormat.java | 1 - .../HoodieMergeOnReadTableInputFormat.java | 28 ++++- .../RealtimeBootstrapBaseFileSplit.java | 25 ++--- .../utils/HoodieRealtimeInputFormatUtils.java | 103 ------------------ 5 files changed, 32 insertions(+), 129 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java index 1a609e042c854..6db1751771904 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/BootstrapBaseFileSplit.java @@ -36,9 +36,7 @@ public class BootstrapBaseFileSplit extends FileSplit { * NOTE: This ctor is necessary for Hive to be able to serialize and * then instantiate it when deserializing back */ - public BootstrapBaseFileSplit() { - super(); - } + public BootstrapBaseFileSplit() {} public BootstrapBaseFileSplit(FileSplit baseSplit, FileSplit bootstrapFileSplit) throws IOException { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index fd5ef8da781dd..58a51fa67af4e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -280,7 +280,6 @@ private List listStatusForSnapshotMode(JobConf job, throw new IllegalStateException("Invalid state: base-file has to be present"); } } - }) .collect(Collectors.toList()) ); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 4e96c4f3886a0..3aeabb5b8cbe9 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.SplitLocationInfo; import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -37,6 +38,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat; @@ -235,20 +237,38 @@ private List collectAllIncrementalFiles(List fileGr } private FileSplit doMakeSplitForPathWithLogFilePath(HoodieRealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { - if (!path.includeBootstrapFilePath()) { - return path.buildSplit(path, start, length, hosts); - } else { + if (path.includeBootstrapFilePath()) { FileSplit bf = inMemoryHosts == null ? super.makeSplit(path.getPathWithBootstrapFileStatus(), start, length, hosts) : super.makeSplit(path.getPathWithBootstrapFileStatus(), start, length, hosts, inMemoryHosts); - return HoodieRealtimeInputFormatUtils.createRealtimeBoostrapBaseFileSplit( + return createRealtimeBoostrapBaseFileSplit( (BootstrapBaseFileSplit) bf, path.getBasePath(), path.getDeltaLogFiles(), path.getMaxCommitTime(), path.getBelongsToIncrementalQuery()); } + + return path.buildSplit(path, start, length, hosts); + } + + public static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, + String basePath, + List logFiles, + String maxInstantTime, + boolean belongsToIncrementalQuery) { + try { + String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) + .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; + String[] inMemoryHosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) + .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; + FileSplit baseSplit = new FileSplit(split.getPath(), split.getStart(), split.getLength(), + hosts, inMemoryHosts); + return new RealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), belongsToIncrementalQuery); + } catch (IOException e) { + throw new HoodieIOException("Error creating hoodie real time split ", e); + } } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java index 2ac7204467618..ba4070ff27982 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; /** * Realtime File Split with external base file. @@ -39,7 +38,6 @@ */ public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { - private List deltaLogPaths; private List deltaLogFiles = new ArrayList<>(); private String maxInstantTime; @@ -52,9 +50,7 @@ public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit imple * NOTE: This ctor is necessary for Hive to be able to serialize and * then instantiate it when deserializing back */ - public RealtimeBootstrapBaseFileSplit() { - super(); - } + public RealtimeBootstrapBaseFileSplit() {} public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, String basePath, @@ -65,7 +61,6 @@ public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, super(baseSplit, externalFileSplit); this.maxInstantTime = maxInstantTime; this.deltaLogFiles = deltaLogFiles; - this.deltaLogPaths = deltaLogFiles.stream().map(entry -> entry.getPath().toString()).collect(Collectors.toList()); this.basePath = basePath; this.belongsToIncrementalSplit = belongsToIncrementalQuery; } @@ -85,13 +80,13 @@ public void readFields(DataInput in) throws IOException { } @Override - public List getDeltaLogPaths() { - return deltaLogPaths; + public List getDeltaLogFiles() { + return deltaLogFiles; } @Override - public List getDeltaLogFiles() { - return deltaLogFiles; + public void setDeltaLogFiles(List deltaLogFiles) { + this.deltaLogFiles = deltaLogFiles; } @Override @@ -105,7 +100,7 @@ public String getBasePath() { } @Override - public Option getHoodieVirtualKeyInfo() { + public Option getVirtualKeyInfo() { return Option.empty(); } @@ -113,11 +108,6 @@ public boolean getBelongsToIncrementalQuery() { return belongsToIncrementalSplit; } - @Override - public void setDeltaLogPaths(List deltaLogPaths) { - this.deltaLogPaths = deltaLogPaths; - } - @Override public void setMaxCommitTime(String maxInstantTime) { this.maxInstantTime = maxInstantTime; @@ -129,6 +119,5 @@ public void setBasePath(String basePath) { } @Override - public void setHoodieVirtualKeyInfo(Option hoodieVirtualKeyInfo) {} - + public void setVirtualKeyInfo(Option virtualKeyInfo) {} } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 98d3c0faab251..23176737aa539 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -24,8 +24,6 @@ import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.SplitLocationInfo; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -35,10 +33,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -55,8 +50,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -115,84 +108,6 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, List return finalSplits; } - /** - * @deprecated - */ - public static InputSplit[] getRealtimeSplitsLegacy(Configuration conf, Stream fileSplits) { - Map> partitionsToParquetSplits = - fileSplits.collect(Collectors.groupingBy(split -> split.getPath().getParent())); - // TODO(vc): Should we handle also non-hoodie splits here? - Map partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionsToParquetSplits.keySet()); - - // Create file system cache so metadata table is only instantiated once. Also can benefit normal file listing if - // partition path is listed twice so file groups will already be loaded in file system - Map fsCache = new HashMap<>(); - // for all unique split parents, obtain all delta files based on delta commit timeline, - // grouped on file id - List rtSplits = new ArrayList<>(); - try { - // Pre process tableConfig from first partition to fetch virtual key info - Option hoodieVirtualKeyInfo = Option.empty(); - if (partitionsToParquetSplits.size() > 0) { - HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionsToParquetSplits.keySet().iterator().next()); - hoodieVirtualKeyInfo = getHoodieVirtualKeyInfo(metaClient); - } - Option finalHoodieVirtualKeyInfo = hoodieVirtualKeyInfo; - partitionsToParquetSplits.keySet().forEach(partitionPath -> { - // for each partition path obtain the data & log file groupings, then map back to inputsplits - HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath); - if (!fsCache.containsKey(metaClient)) { - HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf); - HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, - metaClient, HoodieInputFormatUtils.buildMetadataConfig(conf), metaClient.getActiveTimeline()); - fsCache.put(metaClient, fsView); - } - HoodieTableFileSystemView fsView = fsCache.get(metaClient); - - String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath); - // Both commit and delta-commits are included - pick the latest completed one - Option latestCompletedInstant = - metaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(); - - Stream latestFileSlices = latestCompletedInstant - .map(instant -> fsView.getLatestMergedFileSlicesBeforeOrOn(relPartitionPath, instant.getTimestamp())) - .orElse(Stream.empty()); - - // subgroup splits again by file id & match with log files. - Map> groupedInputSplits = partitionsToParquetSplits.get(partitionPath).stream() - .collect(Collectors.groupingBy(split -> FSUtils.getFileIdFromFilePath(split.getPath()))); - // Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table - String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, - HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); - latestFileSlices.forEach(fileSlice -> { - List dataFileSplits = groupedInputSplits.getOrDefault(fileSlice.getFileId(), new ArrayList<>()); - dataFileSplits.forEach(split -> { - try { - List logFiles = fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) - .collect(Collectors.toList()); - if (split instanceof BootstrapBaseFileSplit) { - BootstrapBaseFileSplit eSplit = (BootstrapBaseFileSplit) split; - rtSplits.add(createRealtimeBoostrapBaseFileSplit(eSplit, metaClient.getBasePath(), logFiles, maxCommitTime, false)); - } else { - rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFiles, maxCommitTime, finalHoodieVirtualKeyInfo)); - } - } catch (IOException e) { - throw new HoodieIOException("Error creating hoodie real time split ", e); - } - }); - }); - }); - } catch (Exception e) { - throw new HoodieException("Error obtaining data file/log file grouping ", e); - } finally { - // close all the open fs views. - fsCache.forEach((k, view) -> view.close()); - } - LOG.info("Returning a total splits of " + rtSplits.size()); - return rtSplits.toArray(new InputSplit[0]); - } - /** * @deprecated will be replaced w/ {@link #getRealtimeSplits(Configuration, List)} */ @@ -262,24 +177,6 @@ public static boolean isIncrementalQuerySplits(List fileSplits) { return fileSplits.stream().anyMatch(HoodieRealtimeInputFormatUtils::doesBelongToIncrementalQuery); } - public static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, - String basePath, - List logFiles, - String maxInstantTime, - boolean belongsToIncrementalQuery) { - try { - String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) - .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; - String[] inMemoryHosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) - .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; - FileSplit baseSplit = new FileSplit(split.getPath(), split.getStart(), split.getLength(), - hosts, inMemoryHosts); - return new RealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), belongsToIncrementalQuery); - } catch (IOException e) { - throw new HoodieIOException("Error creating hoodie real time split ", e); - } - } - // Return parquet file with a list of log files in the same file group. public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { Set partitionSet = new HashSet<>(partitionPaths); From 04c024a3a884b46fccae4283f34499d88e5480d0 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 14:04:37 -0800 Subject: [PATCH 09/30] Removed `BaseFileWithLogsSplit` --- .../realtime/BaseFileWithLogsSplit.java | 147 ------------------ .../utils/HoodieRealtimeInputFormatUtils.java | 42 ++--- 2 files changed, 11 insertions(+), 178 deletions(-) delete mode 100644 hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java deleted file mode 100644 index 9f63d8d4bbaa7..0000000000000 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/BaseFileWithLogsSplit.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.hadoop.realtime; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Encode additional information in split to track matching log file and base files. - * Hence, this class tracks a log/base file split. - */ -public class BaseFileWithLogsSplit extends FileSplit implements RealtimeSplit { - // a flag to mark this split is produced by incremental query or not. - private boolean belongsToIncrementalQuery = false; - // the log file paths of this split. - private List deltaLogFiles = new ArrayList<>(); - // max commit time of current split. - private String maxCommitTime = ""; - // the basePath of current hoodie table. - private String basePath = ""; - // the base file belong to this split. - private String baseFilePath = ""; - - private Option virtualKeyInfo = Option.empty(); - - public BaseFileWithLogsSplit(Path file, long start, long length, String[] hosts) { - super(file, start, length, hosts); - } - - @Override - public void write(DataOutput out) throws IOException { - super.write(out); - out.writeBoolean(belongsToIncrementalQuery); - Text.writeString(out, maxCommitTime); - Text.writeString(out, basePath); - Text.writeString(out, baseFilePath); - out.writeInt(deltaLogFiles.size()); - for (HoodieLogFile logFile : deltaLogFiles) { - Text.writeString(out, logFile.getPath().toString()); - out.writeLong(logFile.getFileSize()); - } - } - - @Override - public void readFields(DataInput in) throws IOException { - super.readFields(in); - belongsToIncrementalQuery = in.readBoolean(); - maxCommitTime = Text.readString(in); - basePath = Text.readString(in); - baseFilePath = Text.readString(in); - int deltaLogSize = in.readInt(); - List tempDeltaLogs = new ArrayList<>(); - for (int i = 0; i < deltaLogSize; i++) { - String logPath = Text.readString(in); - long logFileSize = in.readLong(); - tempDeltaLogs.add(new HoodieLogFile(new Path(logPath), logFileSize)); - } - deltaLogFiles = tempDeltaLogs; - } - - public boolean getBelongsToIncrementalQuery() { - return belongsToIncrementalQuery; - } - - public void setBelongsToIncrementalQuery(boolean belongsToIncrementalQuery) { - this.belongsToIncrementalQuery = belongsToIncrementalQuery; - } - - @Override - public List getDeltaLogFiles() { - return deltaLogFiles; - } - - @Override - public void setDeltaLogFiles(List deltaLogFiles) { - this.deltaLogFiles = deltaLogFiles; - } - - @Override - public String getMaxCommitTime() { - return maxCommitTime; - } - - @Override - public void setMaxCommitTime(String maxCommitTime) { - this.maxCommitTime = maxCommitTime; - } - - @Override - public String getBasePath() { - return basePath; - } - - @Override - public void setBasePath(String basePath) { - this.basePath = basePath; - } - - @Override - public Option getVirtualKeyInfo() { - return virtualKeyInfo; - } - - @Override - public void setVirtualKeyInfo(Option virtualKeyInfo) { - this.virtualKeyInfo = virtualKeyInfo; - } - - public String getBaseFilePath() { - return baseFilePath; - } - - public void setBaseFilePath(String baseFilePath) { - this.baseFilePath = baseFilePath; - } - - @Override - public String toString() { - return "HoodieRealtimeFileSplit{DataPath=" + getPath() + ", deltaLogPaths=" + getDeltaLogPaths() - + ", maxCommitTime='" + maxCommitTime + '\'' + ", basePath='" + basePath + '\'' + '}'; - } -} diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 23176737aa539..e77c81bb7bc2b 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -37,9 +37,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; -import org.apache.hudi.hadoop.realtime.BaseFileWithLogsSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit; @@ -89,9 +87,8 @@ public static InputSplit[] getRealtimeSplits(Configuration conf, List // - {@code FileSplit}: in case Hive passed down non-Hudi path if (split instanceof RealtimeBootstrapBaseFileSplit) { return split; - } else if (split instanceof BaseFileWithLogsSplit) { - BaseFileWithLogsSplit baseFileWithLogsSplit = unsafeCast(split); - return createHoodieRealtimeSplitUnchecked(baseFileWithLogsSplit, hoodieVirtualKeyInfoOpt); + } else if (split instanceof HoodieRealtimeFileSplit) { + return split; } else { // TODO cleanup checkState(!(split instanceof BootstrapBaseFileSplit)); @@ -127,15 +124,12 @@ public static InputSplit[] getIncrementalRealtimeSplits(Configuration conf, List Option finalHoodieVirtualKeyInfo = hoodieVirtualKeyInfo; fileSplits.stream().forEach(s -> { // deal with incremental query. - try { - if (s instanceof BaseFileWithLogsSplit) { - BaseFileWithLogsSplit bs = unsafeCast(s); - rtSplits.add(new HoodieRealtimeFileSplit(bs, bs.getBasePath(), bs.getDeltaLogFiles(), bs.getMaxCommitTime(), finalHoodieVirtualKeyInfo)); - } else if (s instanceof RealtimeBootstrapBaseFileSplit) { - rtSplits.add(s); - } - } catch (IOException e) { - throw new HoodieIOException("Error creating hoodie real time split ", e); + if (s instanceof HoodieRealtimeFileSplit) { + HoodieRealtimeFileSplit bs = unsafeCast(s); + rtSplits.add(bs); + } else if (s instanceof RealtimeBootstrapBaseFileSplit) { + + rtSplits.add(s); } }); LOG.info("Returning a total splits of " + rtSplits.size()); @@ -158,9 +152,9 @@ public static Option getHoodieVirtualKeyInfo(HoodieTableMe return Option.empty(); } - private static boolean doesBelongToIncrementalQuery(FileSplit s) { - if (s instanceof BaseFileWithLogsSplit) { - BaseFileWithLogsSplit bs = unsafeCast(s); + public static boolean doesBelongToIncrementalQuery(FileSplit s) { + if (s instanceof HoodieRealtimeFileSplit) { + HoodieRealtimeFileSplit bs = unsafeCast(s); return bs.getBelongsToIncrementalQuery(); } else if (s instanceof RealtimeBootstrapBaseFileSplit) { RealtimeBootstrapBaseFileSplit bs = unsafeCast(s); @@ -287,18 +281,4 @@ public static void cleanProjectionColumnIds(Configuration conf) { } } } - - private static HoodieRealtimeFileSplit createHoodieRealtimeSplitUnchecked(BaseFileWithLogsSplit baseFileWithLogsSplit, - Option hoodieVirtualKeyInfoOpt) { - try { - return new HoodieRealtimeFileSplit( - baseFileWithLogsSplit, - baseFileWithLogsSplit.getBasePath(), - baseFileWithLogsSplit.getDeltaLogFiles(), - baseFileWithLogsSplit.getMaxCommitTime(), - hoodieVirtualKeyInfoOpt); - } catch (IOException e) { - throw new HoodieIOException(String.format("Failed to init %s", HoodieRealtimeFileSplit.class.getSimpleName()), e); - } - } } From c2d8b04f7d39de48b20363d25957618c548c7ef6 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 14:16:32 -0800 Subject: [PATCH 10/30] Cleaned up dead-code --- .../HoodieMergeOnReadTableInputFormat.java | 26 +++-- .../utils/HoodieRealtimeInputFormatUtils.java | 104 ------------------ 2 files changed, 18 insertions(+), 112 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 3aeabb5b8cbe9..d871fc534ae9c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -71,11 +71,12 @@ public class HoodieMergeOnReadTableInputFormat extends HoodieCopyOnWriteTableInp @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - List fileSplits = Arrays.stream(super.getSplits(job, numSplits)).map(is -> (FileSplit) is).collect(Collectors.toList()); + List fileSplits = Arrays.stream(super.getSplits(job, numSplits)) + .map(is -> (FileSplit) is) + .collect(Collectors.toList()); - return HoodieRealtimeInputFormatUtils.isIncrementalQuerySplits(fileSplits) - ? HoodieRealtimeInputFormatUtils.getIncrementalRealtimeSplits(job, fileSplits) - : HoodieRealtimeInputFormatUtils.getRealtimeSplits(job, fileSplits); + return (containsIncrementalQuerySplits(fileSplits) ? filterIncrementalQueryFileSplits(fileSplits) : fileSplits) + .toArray(new FileSplit[0]); } /** @@ -179,7 +180,7 @@ protected boolean isSplitable(FileSystem fs, Path filename) { @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts) { if (file instanceof HoodieRealtimePath) { - return doMakeSplitForPathWithLogFilePath((HoodieRealtimePath) file, start, length, hosts, null); + return doMakeSplitForRealtimePath((HoodieRealtimePath) file, start, length, hosts, null); } return super.makeSplit(file, start, length, hosts); } @@ -187,7 +188,7 @@ protected FileSplit makeSplit(Path file, long start, long length, String[] hosts @Override protected FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) { if (file instanceof HoodieRealtimePath) { - return doMakeSplitForPathWithLogFilePath((HoodieRealtimePath) file, start, length, hosts, inMemoryHosts); + return doMakeSplitForRealtimePath((HoodieRealtimePath) file, start, length, hosts, inMemoryHosts); } return super.makeSplit(file, start, length, hosts, inMemoryHosts); } @@ -236,7 +237,7 @@ private List collectAllIncrementalFiles(List fileGr return result; } - private FileSplit doMakeSplitForPathWithLogFilePath(HoodieRealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { + private FileSplit doMakeSplitForRealtimePath(HoodieRealtimePath path, long start, long length, String[] hosts, String[] inMemoryHosts) { if (path.includeBootstrapFilePath()) { FileSplit bf = inMemoryHosts == null @@ -253,7 +254,16 @@ private FileSplit doMakeSplitForPathWithLogFilePath(HoodieRealtimePath path, lon return path.buildSplit(path, start, length, hosts); } - public static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, + private static boolean containsIncrementalQuerySplits(List fileSplits) { + return fileSplits.stream().anyMatch(HoodieRealtimeInputFormatUtils::doesBelongToIncrementalQuery); + } + + private static List filterIncrementalQueryFileSplits(List fileSplits) { + return fileSplits.stream().filter(HoodieRealtimeInputFormatUtils::doesBelongToIncrementalQuery) + .collect(Collectors.toList()); + } + + private static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, String basePath, List logFiles, String maxInstantTime, diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index e77c81bb7bc2b..05d5dbedd9047 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -22,31 +22,25 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.RealtimeSplit; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; -import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -56,102 +50,11 @@ import java.util.stream.Stream; import static org.apache.hudi.TypeUtils.unsafeCast; -import static org.apache.hudi.common.util.ValidationUtils.checkState; public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils { private static final Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormatUtils.class); - public static InputSplit[] getRealtimeSplits(Configuration conf, List fileSplits) throws IOException { - if (fileSplits.isEmpty()) { - return new InputSplit[0]; - } - - FileSplit fileSplit = fileSplits.get(0); - - // Pre-process table-config to fetch virtual key info - Path partitionPath = fileSplit.getPath().getParent(); - HoodieTableMetaClient metaClient = getTableMetaClientForBasePathUnchecked(conf, partitionPath); - - Option hoodieVirtualKeyInfoOpt = getHoodieVirtualKeyInfo(metaClient); - - InputSplit[] finalSplits = fileSplits.stream() - .map(split -> { - // There are 4 types of splits we have to handle here - // - {@code BootstrapBaseFileSplit}: in case base file does have associated bootstrap file, - // but does NOT have any log files appended (convert it to {@code RealtimeBootstrapBaseFileSplit}) - // - {@code RealtimeBootstrapBaseFileSplit}: in case base file does have associated bootstrap file - // and does have log files appended - // - {@code BaseFileWithLogsSplit}: in case base file does NOT have associated bootstrap file - // and does have log files appended; - // - {@code FileSplit}: in case Hive passed down non-Hudi path - if (split instanceof RealtimeBootstrapBaseFileSplit) { - return split; - } else if (split instanceof HoodieRealtimeFileSplit) { - return split; - } else { - // TODO cleanup - checkState(!(split instanceof BootstrapBaseFileSplit)); - - // Non-Hudi paths might result in just generic {@code FileSplit} being - // propagated up to this point - return split; - } - }) - .toArray(InputSplit[]::new); - - LOG.info("Returning a total splits of " + finalSplits.length); - - return finalSplits; - } - - /** - * @deprecated will be replaced w/ {@link #getRealtimeSplits(Configuration, List)} - */ - // get IncrementalRealtimeSplits - public static InputSplit[] getIncrementalRealtimeSplits(Configuration conf, List fileSplits) throws IOException { - checkState(fileSplits.stream().allMatch(HoodieRealtimeInputFormatUtils::doesBelongToIncrementalQuery), - "All splits have to belong to incremental query"); - - List rtSplits = new ArrayList<>(); - Set partitionSet = fileSplits.stream().map(f -> f.getPath().getParent()).collect(Collectors.toSet()); - Map partitionsToMetaClient = getTableMetaClientByPartitionPath(conf, partitionSet); - // Pre process tableConfig from first partition to fetch virtual key info - Option hoodieVirtualKeyInfo = Option.empty(); - if (partitionSet.size() > 0) { - hoodieVirtualKeyInfo = getHoodieVirtualKeyInfo(partitionsToMetaClient.get(partitionSet.iterator().next())); - } - Option finalHoodieVirtualKeyInfo = hoodieVirtualKeyInfo; - fileSplits.stream().forEach(s -> { - // deal with incremental query. - if (s instanceof HoodieRealtimeFileSplit) { - HoodieRealtimeFileSplit bs = unsafeCast(s); - rtSplits.add(bs); - } else if (s instanceof RealtimeBootstrapBaseFileSplit) { - - rtSplits.add(s); - } - }); - LOG.info("Returning a total splits of " + rtSplits.size()); - return rtSplits.toArray(new InputSplit[0]); - } - - public static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { - HoodieTableConfig tableConfig = metaClient.getTableConfig(); - if (!tableConfig.populateMetaFields()) { - TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); - try { - MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema(); - return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), - tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()), - parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp()))); - } catch (Exception exception) { - throw new HoodieException("Fetching table schema failed with exception ", exception); - } - } - return Option.empty(); - } - public static boolean doesBelongToIncrementalQuery(FileSplit s) { if (s instanceof HoodieRealtimeFileSplit) { HoodieRealtimeFileSplit bs = unsafeCast(s); @@ -164,13 +67,6 @@ public static boolean doesBelongToIncrementalQuery(FileSplit s) { return false; } - public static boolean isIncrementalQuerySplits(List fileSplits) { - if (fileSplits == null || fileSplits.size() == 0) { - return false; - } - return fileSplits.stream().anyMatch(HoodieRealtimeInputFormatUtils::doesBelongToIncrementalQuery); - } - // Return parquet file with a list of log files in the same file group. public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { Set partitionSet = new HashSet<>(partitionPaths); From 5fda93ef657036f216a4d174fd615dbb05c60735 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 14:22:15 -0800 Subject: [PATCH 11/30] Fixed `RealtimeBootstrapBaseFileSplit` --- .../realtime/HoodieRealtimeFileSplit.java | 2 + .../RealtimeBootstrapBaseFileSplit.java | 52 +++++++++++++------ .../hudi/hadoop/realtime/RealtimeSplit.java | 13 +++++ 3 files changed, 50 insertions(+), 17 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 76a8a9fc35a97..61690758af9a2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -118,10 +118,12 @@ public Option getVirtualKeyInfo() { return virtualKeyInfo; } + @Override public boolean getBelongsToIncrementalQuery() { return belongsToIncrementalQuery; } + @Override public void setBelongsToIncrementalQuery(boolean belongsToIncrementalPath) { this.belongsToIncrementalQuery = belongsToIncrementalPath; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java index ba4070ff27982..c4c59e97e85a2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; -import org.apache.hudi.hadoop.InputSplitUtils; import java.io.DataInput; import java.io.DataOutput; @@ -31,20 +30,33 @@ import java.util.List; /** - * Realtime File Split with external base file. + * Realtime {@link FileSplit} with external base file * * NOTE: If you're adding fields here you need to make sure that you appropriately de-/serialize them * in {@link #readFromInput(DataInput)} and {@link #writeToOutput(DataOutput)} */ public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { - + /** + * Marks whether this path produced as part of Incremental Query + */ + private boolean belongsToIncrementalQuery = false; + /** + * List of delta log-files holding updated records for this base-file + */ private List deltaLogFiles = new ArrayList<>(); - - private String maxInstantTime; - + /** + * Latest commit instant available at the time of the query in which all of the files + * pertaining to this split are represented + */ + private String maxCommitTime; + /** + * Base path of the table this path belongs to + */ private String basePath; - - private boolean belongsToIncrementalSplit; + /** + * Virtual key configuration of the table this split belongs to + */ + private Option virtualKeyInfo = Option.empty(); /** * NOTE: This ctor is necessary for Hive to be able to serialize and @@ -59,24 +71,22 @@ public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, FileSplit externalFileSplit, boolean belongsToIncrementalQuery) throws IOException { super(baseSplit, externalFileSplit); - this.maxInstantTime = maxInstantTime; + this.maxCommitTime = maxInstantTime; this.deltaLogFiles = deltaLogFiles; this.basePath = basePath; - this.belongsToIncrementalSplit = belongsToIncrementalQuery; + this.belongsToIncrementalQuery = belongsToIncrementalQuery; } @Override public void write(DataOutput out) throws IOException { super.write(out); writeToOutput(out); - InputSplitUtils.writeBoolean(belongsToIncrementalSplit, out); } @Override public void readFields(DataInput in) throws IOException { super.readFields(in); readFromInput(in); - belongsToIncrementalSplit = InputSplitUtils.readBoolean(in); } @Override @@ -91,7 +101,7 @@ public void setDeltaLogFiles(List deltaLogFiles) { @Override public String getMaxCommitTime() { - return maxInstantTime; + return maxCommitTime; } @Override @@ -101,16 +111,22 @@ public String getBasePath() { @Override public Option getVirtualKeyInfo() { - return Option.empty(); + return virtualKeyInfo; } + @Override public boolean getBelongsToIncrementalQuery() { - return belongsToIncrementalSplit; + return belongsToIncrementalQuery; + } + + @Override + public void setBelongsToIncrementalQuery(boolean belongsToIncrementalPath) { + this.belongsToIncrementalQuery = belongsToIncrementalPath; } @Override public void setMaxCommitTime(String maxInstantTime) { - this.maxInstantTime = maxInstantTime; + this.maxCommitTime = maxInstantTime; } @Override @@ -119,5 +135,7 @@ public void setBasePath(String basePath) { } @Override - public void setVirtualKeyInfo(Option virtualKeyInfo) {} + public void setVirtualKeyInfo(Option virtualKeyInfo) { + this.virtualKeyInfo = virtualKeyInfo; + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java index 3a04fb11fa1d7..d9b1923c60f80 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeSplit.java @@ -66,6 +66,11 @@ default List getDeltaLogPaths() { */ Option getVirtualKeyInfo(); + /** + * Returns the flag whether this split belongs to an Incremental Query + */ + boolean getBelongsToIncrementalQuery(); + /** * Update Maximum valid instant time. * @param maxCommitTime @@ -78,11 +83,18 @@ default List getDeltaLogPaths() { */ void setBasePath(String basePath); + /** + * Sets the flag whether this split belongs to an Incremental Query + */ + void setBelongsToIncrementalQuery(boolean belongsToIncrementalQuery); + void setVirtualKeyInfo(Option virtualKeyInfo); default void writeToOutput(DataOutput out) throws IOException { InputSplitUtils.writeString(getBasePath(), out); InputSplitUtils.writeString(getMaxCommitTime(), out); + InputSplitUtils.writeBoolean(getBelongsToIncrementalQuery(), out); + out.writeInt(getDeltaLogFiles().size()); for (HoodieLogFile logFile : getDeltaLogFiles()) { InputSplitUtils.writeString(logFile.getPath().toString(), out); @@ -104,6 +116,7 @@ default void writeToOutput(DataOutput out) throws IOException { default void readFromInput(DataInput in) throws IOException { setBasePath(InputSplitUtils.readString(in)); setMaxCommitTime(InputSplitUtils.readString(in)); + setBelongsToIncrementalQuery(InputSplitUtils.readBoolean(in)); int totalLogFiles = in.readInt(); List deltaLogPaths = new ArrayList<>(totalLogFiles); From ba49892ddad4cf1e9557a163a3d541bda6b2dfb9 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 14:23:07 -0800 Subject: [PATCH 12/30] `RealtimeBootstrapBaseFileSplit` > `HoodieRealtimeBootstrapBaseFileSplit` --- .../HoodieMergeOnReadTableInputFormat.java | 12 ++++++------ ... HoodieRealtimeBootstrapBaseFileSplit.java} | 18 +++++++++--------- .../utils/HoodieRealtimeInputFormatUtils.java | 6 +++--- 3 files changed, 18 insertions(+), 18 deletions(-) rename hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/{RealtimeBootstrapBaseFileSplit.java => HoodieRealtimeBootstrapBaseFileSplit.java} (86%) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index d871fc534ae9c..fb91a8617253d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -263,11 +263,11 @@ private static List filterIncrementalQueryFileSplits(List .collect(Collectors.toList()); } - private static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, - String basePath, - List logFiles, - String maxInstantTime, - boolean belongsToIncrementalQuery) { + private static HoodieRealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, + String basePath, + List logFiles, + String maxInstantTime, + boolean belongsToIncrementalQuery) { try { String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; @@ -275,7 +275,7 @@ private static RealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSpli .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; FileSplit baseSplit = new FileSplit(split.getPath(), split.getStart(), split.getLength(), hosts, inMemoryHosts); - return new RealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), belongsToIncrementalQuery); + return new HoodieRealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), belongsToIncrementalQuery); } catch (IOException e) { throw new HoodieIOException("Error creating hoodie real time split ", e); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java similarity index 86% rename from hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java rename to hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java index c4c59e97e85a2..bf15c2cecb7aa 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeBootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java @@ -35,7 +35,7 @@ * NOTE: If you're adding fields here you need to make sure that you appropriately de-/serialize them * in {@link #readFromInput(DataInput)} and {@link #writeToOutput(DataOutput)} */ -public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { +public class HoodieRealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit implements RealtimeSplit { /** * Marks whether this path produced as part of Incremental Query */ @@ -62,14 +62,14 @@ public class RealtimeBootstrapBaseFileSplit extends BootstrapBaseFileSplit imple * NOTE: This ctor is necessary for Hive to be able to serialize and * then instantiate it when deserializing back */ - public RealtimeBootstrapBaseFileSplit() {} - - public RealtimeBootstrapBaseFileSplit(FileSplit baseSplit, - String basePath, - List deltaLogFiles, - String maxInstantTime, - FileSplit externalFileSplit, - boolean belongsToIncrementalQuery) throws IOException { + public HoodieRealtimeBootstrapBaseFileSplit() {} + + public HoodieRealtimeBootstrapBaseFileSplit(FileSplit baseSplit, + String basePath, + List deltaLogFiles, + String maxInstantTime, + FileSplit externalFileSplit, + boolean belongsToIncrementalQuery) throws IOException { super(baseSplit, externalFileSplit); this.maxCommitTime = maxInstantTime; this.deltaLogFiles = deltaLogFiles; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 05d5dbedd9047..396782d96eeec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -34,9 +34,9 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hadoop.realtime.HoodieRealtimeBootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit; import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; -import org.apache.hudi.hadoop.realtime.RealtimeBootstrapBaseFileSplit; import org.apache.hudi.hadoop.realtime.RealtimeSplit; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -59,8 +59,8 @@ public static boolean doesBelongToIncrementalQuery(FileSplit s) { if (s instanceof HoodieRealtimeFileSplit) { HoodieRealtimeFileSplit bs = unsafeCast(s); return bs.getBelongsToIncrementalQuery(); - } else if (s instanceof RealtimeBootstrapBaseFileSplit) { - RealtimeBootstrapBaseFileSplit bs = unsafeCast(s); + } else if (s instanceof HoodieRealtimeBootstrapBaseFileSplit) { + HoodieRealtimeBootstrapBaseFileSplit bs = unsafeCast(s); return bs.getBelongsToIncrementalQuery(); } From ce26a380c87ae849ee96059019842063cad2cc79 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 3 Feb 2022 14:39:01 -0800 Subject: [PATCH 13/30] Added comments --- .../hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 58a51fa67af4e..20b5500251238 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -305,6 +305,9 @@ private static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { } } + /** + * Creates {@link RealtimeFileStatus} for the file-slice where base file is present + */ @Nonnull private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, Stream logFiles, @@ -335,6 +338,9 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFi } } + /** + * Creates {@link RealtimeFileStatus} for the file-slice where base file is NOT present + */ @Nonnull private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, Stream logFiles, From ffe14516b42515cc8cff0321854ef91a11432b37 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 19:20:17 -0800 Subject: [PATCH 14/30] Propagate missing Virtual Key info into `RealtimeSplit`s --- .../hudi/hadoop/RealtimeFileStatus.java | 48 +++++++++++++------ .../HoodieMergeOnReadTableInputFormat.java | 22 +++++++-- .../HoodieRealtimeBootstrapBaseFileSplit.java | 4 +- .../realtime/HoodieRealtimeFileSplit.java | 21 ++++++-- .../hadoop/realtime/HoodieRealtimePath.java | 18 ++++--- .../realtime/TestHoodieRealtimeFileSplit.java | 7 ++- .../TestHoodieRealtimeRecordReader.java | 13 +++-- 7 files changed, 91 insertions(+), 42 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index 100d28bc49152..c8e79fdc4cf1a 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -21,7 +21,9 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.realtime.HoodieRealtimePath; +import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import java.io.IOException; import java.util.ArrayList; @@ -34,19 +36,31 @@ * in Path. */ public class RealtimeFileStatus extends FileStatus { - // a flag to mark this split is produced by incremental query or not. - private boolean belongToIncrementalFileStatus = false; - // the log files belong this fileStatus. + /** + * Marks whether this path produced as part of Incremental Query + */ + private boolean belongsToIncrementalQuery = false; + /** + * List of delta log-files holding updated records for this base-file + */ private List deltaLogFiles = new ArrayList<>(); - // max commit time of current fileStatus. + /** + * Latest commit instant available at the time of the query in which all of the files + * pertaining to this split are represented + */ private String maxCommitTime = ""; - // the basePath of current hoodie table. + /** + * Base path of the table this path belongs to + */ private String basePath = ""; - // the base file belong to this status; - private String baseFilePath = ""; - // the bootstrap file belong to this status. - // only if current query table is bootstrap table, this field is used. + /** + * File status for the Bootstrap file (only relevant if this table is a bootstrapped table + */ private FileStatus bootStrapFileStatus; + /** + * Virtual key configuration of the table this split belongs to + */ + private Option virtualKeyInfo = Option.empty(); public RealtimeFileStatus(FileStatus fileStatus) throws IOException { super(fileStatus); @@ -55,19 +69,23 @@ public RealtimeFileStatus(FileStatus fileStatus) throws IOException { @Override public Path getPath() { Path path = super.getPath(); + HoodieRealtimePath realtimePath = new HoodieRealtimePath(path.getParent(), path.getName()); - realtimePath.setBelongsToIncrementalQuery(belongToIncrementalFileStatus); + realtimePath.setBelongsToIncrementalQuery(belongsToIncrementalQuery); realtimePath.setDeltaLogFiles(deltaLogFiles); realtimePath.setMaxCommitTime(maxCommitTime); realtimePath.setBasePath(basePath); + realtimePath.setVirtualKeyInfo(virtualKeyInfo); + if (bootStrapFileStatus != null) { realtimePath.setPathWithBootstrapFileStatus((PathWithBootstrapFileStatus)bootStrapFileStatus.getPath()); } + return realtimePath; } public void setBelongToIncrementalFileStatus(boolean belongToIncrementalFileStatus) { - this.belongToIncrementalFileStatus = belongToIncrementalFileStatus; + this.belongsToIncrementalQuery = belongToIncrementalFileStatus; } public List getDeltaLogFiles() { @@ -94,11 +112,11 @@ public void setBasePath(String basePath) { this.basePath = basePath; } - public void setBaseFilePath(String baseFilePath) { - this.baseFilePath = baseFilePath; - } - public void setBootStrapFileStatus(FileStatus bootStrapFileStatus) { this.bootStrapFileStatus = bootStrapFileStatus; } + + public void setVirtualKeyInfo(Option virtualKeyInfo) { + this.virtualKeyInfo = virtualKeyInfo; + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index fb91a8617253d..354ee7033c0e7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -210,7 +210,6 @@ private List collectAllIncrementalFiles(List fileGr fileStatus.setMaxCommitTime(maxCommitTime); fileStatus.setBelongToIncrementalFileStatus(true); fileStatus.setBasePath(basePath); - fileStatus.setBaseFilePath(baseFilePath); fileStatus.setDeltaLogFiles(f.getLatestFileSlice().get().getLogFiles().collect(Collectors.toList())); // try to set bootstrapfileStatus if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { @@ -248,10 +247,12 @@ private FileSplit doMakeSplitForRealtimePath(HoodieRealtimePath path, long start path.getBasePath(), path.getDeltaLogFiles(), path.getMaxCommitTime(), - path.getBelongsToIncrementalQuery()); + path.getBelongsToIncrementalQuery(), + path.getVirtualKeyInfo() + ); } - return path.buildSplit(path, start, length, hosts); + return createRealtimeFileSplit(path, start, length, hosts); } private static boolean containsIncrementalQuerySplits(List fileSplits) { @@ -263,11 +264,21 @@ private static List filterIncrementalQueryFileSplits(List .collect(Collectors.toList()); } + + private static HoodieRealtimeFileSplit createRealtimeFileSplit(HoodieRealtimePath path, long start, long length, String[] hosts) { + try { + return new HoodieRealtimeFileSplit(new FileSplit(path, start, length, hosts), path); + } catch (IOException e) { + throw new HoodieIOException(String.format("Failed to create instance of %s", HoodieRealtimeFileSplit.class.getName()), e); + } + } + private static HoodieRealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFileSplit(BootstrapBaseFileSplit split, String basePath, List logFiles, String maxInstantTime, - boolean belongsToIncrementalQuery) { + boolean belongsToIncrementalQuery, + Option virtualKeyInfoOpt) { try { String[] hosts = split.getLocationInfo() != null ? Arrays.stream(split.getLocationInfo()) .filter(x -> !x.isInMemory()).toArray(String[]::new) : new String[0]; @@ -275,7 +286,8 @@ private static HoodieRealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFi .filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0]; FileSplit baseSplit = new FileSplit(split.getPath(), split.getStart(), split.getLength(), hosts, inMemoryHosts); - return new HoodieRealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), belongsToIncrementalQuery); + return new HoodieRealtimeBootstrapBaseFileSplit(baseSplit, basePath, logFiles, maxInstantTime, split.getBootstrapFileSplit(), + belongsToIncrementalQuery, virtualKeyInfoOpt); } catch (IOException e) { throw new HoodieIOException("Error creating hoodie real time split ", e); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java index bf15c2cecb7aa..c7022c98ad3cd 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeBootstrapBaseFileSplit.java @@ -69,12 +69,14 @@ public HoodieRealtimeBootstrapBaseFileSplit(FileSplit baseSplit, List deltaLogFiles, String maxInstantTime, FileSplit externalFileSplit, - boolean belongsToIncrementalQuery) throws IOException { + boolean belongsToIncrementalQuery, + Option virtualKeyInfoOpt) throws IOException { super(baseSplit, externalFileSplit); this.maxCommitTime = maxInstantTime; this.deltaLogFiles = deltaLogFiles; this.basePath = basePath; this.belongsToIncrementalQuery = belongsToIncrementalQuery; + this.virtualKeyInfo = virtualKeyInfoOpt; } @Override diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 61690758af9a2..895ffb209ef95 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -18,7 +18,6 @@ package org.apache.hudi.hadoop.realtime; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.FileSplit; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.util.Option; @@ -66,20 +65,32 @@ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit public HoodieRealtimeFileSplit() {} - HoodieRealtimeFileSplit(Path file, long start, long length, String[] hosts) { - super(file, start, length, hosts); + public HoodieRealtimeFileSplit(FileSplit baseSplit, + HoodieRealtimePath path) + throws IOException { + this(baseSplit, + path.getBasePath(), + path.getDeltaLogFiles(), + path.getMaxCommitTime(), + path.getBelongsToIncrementalQuery(), + path.getVirtualKeyInfo()); } /** - * @deprecated + * @VisibleInTesting */ - public HoodieRealtimeFileSplit(FileSplit baseSplit, String basePath, List deltaLogFiles, String maxCommitTime, + public HoodieRealtimeFileSplit(FileSplit baseSplit, + String basePath, + List deltaLogFiles, + String maxCommitTime, + boolean belongsToIncrementalQuery, Option virtualKeyInfo) throws IOException { super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); this.deltaLogFiles = deltaLogFiles; this.maxCommitTime = maxCommitTime; this.basePath = basePath; + this.belongsToIncrementalQuery = belongsToIncrementalQuery; this.virtualKeyInfo = virtualKeyInfo; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index ebd952a288a15..8254c717fcb98 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -20,6 +20,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.PathWithBootstrapFileStatus; import java.util.ArrayList; @@ -52,6 +53,10 @@ public class HoodieRealtimePath extends Path { * File status for the Bootstrap file (only relevant if this table is a bootstrapped table */ private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; + /** + * Virtual key configuration of the table this split belongs to + */ + private Option virtualKeyInfo = Option.empty(); public HoodieRealtimePath(Path parent, String child) { super(parent, child); @@ -105,12 +110,11 @@ public boolean includeBootstrapFilePath() { return pathWithBootstrapFileStatus != null; } - public HoodieRealtimeFileSplit buildSplit(Path file, long start, long length, String[] hosts) { - HoodieRealtimeFileSplit bs = new HoodieRealtimeFileSplit(file, start, length, hosts); - bs.setBelongsToIncrementalQuery(belongsToIncrementalQuery); - bs.setDeltaLogFiles(deltaLogFiles); - bs.setMaxCommitTime(maxCommitTime); - bs.setBasePath(basePath); - return bs; + public Option getVirtualKeyInfo() { + return virtualKeyInfo; + } + + public void setVirtualKeyInfo(Option virtualKeyInfo) { + this.virtualKeyInfo = virtualKeyInfo; } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java index 9d3855c47d663..a6ca32769cf8d 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeFileSplit.java @@ -18,12 +18,11 @@ package org.apache.hudi.hadoop.realtime; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.util.Option; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.FileSplit; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.util.Option; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -72,7 +71,7 @@ public void setUp(@TempDir java.nio.file.Path tempDir) throws Exception { baseFileSplit = new FileSplit(new Path(fileSplitName), 0, 100, new String[] {}); maxCommitTime = "10001"; - split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFiles, maxCommitTime, Option.empty()); + split = new HoodieRealtimeFileSplit(baseFileSplit, basePath, deltaLogFiles, maxCommitTime, false, Option.empty()); } @Test diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 00a579fcd37d7..7f80535bd3470 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -226,7 +226,9 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, baseJobConf), basePath.toUri().toString(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()) .collect(Collectors.toList()), - instantTime, Option.empty()); + instantTime, + false, + Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -306,7 +308,7 @@ public void testUnMergedReader() throws Exception { // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty()); + basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, false, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -385,7 +387,7 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, Option.empty()); + basePath.toUri().toString(), Collections.singletonList(writer.getLogFile()), newCommitTime, false, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -532,7 +534,7 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( new FileSplit(new Path(partitionDir + "/fileid0_1_" + instantTime + ".parquet"), 0, 1, baseJobConf), - basePath.toUri().toString(), logFiles, newCommitTime, Option.empty()); + basePath.toUri().toString(), logFiles, newCommitTime, false, Option.empty()); // create a RecordReader to be used by HoodieRealtimeRecordReader RecordReader reader = new MapredParquetInputFormat().getRecordReader( @@ -771,7 +773,8 @@ public void testLogOnlyReader() throws Exception { realtimeFileStatus.setBasePath(basePath.toString()); realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); HoodieRealtimePath realtimePath = (HoodieRealtimePath) realtimeFileStatus.getPath(); - HoodieRealtimeFileSplit split = realtimePath.buildSplit(realtimePath, 0, 0, new String[] {""}); + HoodieRealtimeFileSplit split = + new HoodieRealtimeFileSplit(new FileSplit(realtimePath, 0, 0, new String[] {""}), realtimePath); JobConf newJobConf = new JobConf(baseJobConf); List fields = schema.getFields(); From 20b8380879d0aa4e2f07ed79aa2b69b5bd3f3e0c Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 19:45:55 -0800 Subject: [PATCH 15/30] Cleaned up `RealtimeFileStatus`, `HoodieRealtimePath` to be mostly immutable --- .../HoodieCopyOnWriteTableInputFormat.java | 15 +++-- .../hudi/hadoop/RealtimeFileStatus.java | 59 ++++++++----------- .../HoodieMergeOnReadTableInputFormat.java | 24 ++++---- .../hadoop/realtime/HoodieRealtimePath.java | 49 +++++++-------- .../RealtimeCompactedRecordReader.java | 8 +-- .../hadoop/utils/HoodieInputFormatUtils.java | 38 ++++++------ .../TestHoodieRealtimeRecordReader.java | 7 ++- 7 files changed, 91 insertions(+), 109 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 20b5500251238..32543ca6f619d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -44,6 +44,7 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; @@ -313,13 +314,13 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFi Stream logFiles, Option latestCompletedInstantOpt, HoodieTableMetaClient tableMetaClient) { - List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); + List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); + Option virtualKeyInfoOpt = HoodieVirtualKeyInfo.compose(tableMetaClient); + try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus); - rtFileStatus.setDeltaLogFiles(sortedLogFiles); - rtFileStatus.setBaseFilePath(baseFile.getPath()); - rtFileStatus.setBasePath(tableMetaClient.getBasePath()); + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, tableMetaClient.getBasePath(), sortedLogFiles, + false, virtualKeyInfoOpt); if (latestCompletedInstantOpt.isPresent()) { HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); @@ -348,9 +349,7 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFil HoodieTableMetaClient tableMetaClient) { List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus()); - rtFileStatus.setDeltaLogFiles(sortedLogFiles); - rtFileStatus.setBasePath(tableMetaClient.getBasePath()); + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus(), tableMetaClient.getBasePath(), sortedLogFiles); if (latestCompletedInstantOpt.isPresent()) { HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index c8e79fdc4cf1a..abd3cb4e85a61 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -26,7 +26,6 @@ import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import java.io.IOException; -import java.util.ArrayList; import java.util.List; /** @@ -37,22 +36,22 @@ */ public class RealtimeFileStatus extends FileStatus { /** - * Marks whether this path produced as part of Incremental Query + * Base path of the table this path belongs to */ - private boolean belongsToIncrementalQuery = false; + private final String basePath; /** * List of delta log-files holding updated records for this base-file */ - private List deltaLogFiles = new ArrayList<>(); + private final List deltaLogFiles; + /** + * Marks whether this path produced as part of Incremental Query + */ + private final boolean belongsToIncrementalQuery; /** * Latest commit instant available at the time of the query in which all of the files * pertaining to this split are represented */ private String maxCommitTime = ""; - /** - * Base path of the table this path belongs to - */ - private String basePath = ""; /** * File status for the Bootstrap file (only relevant if this table is a bootstrapped table */ @@ -60,22 +59,30 @@ public class RealtimeFileStatus extends FileStatus { /** * Virtual key configuration of the table this split belongs to */ - private Option virtualKeyInfo = Option.empty(); + private final Option virtualKeyInfo; - public RealtimeFileStatus(FileStatus fileStatus) throws IOException { + public RealtimeFileStatus(FileStatus fileStatus, + String basePath, + List deltaLogFiles, + boolean belongsToIncrementalQuery, + Option virtualKeyInfo) throws IOException { super(fileStatus); + this.basePath = basePath; + this.deltaLogFiles = deltaLogFiles; + this.belongsToIncrementalQuery = belongsToIncrementalQuery; + this.virtualKeyInfo = virtualKeyInfo; + } + + public RealtimeFileStatus(FileStatus fileStatus, String basePath, List deltaLogFiles) throws IOException { + this(fileStatus, basePath, deltaLogFiles, false, Option.empty()); } @Override public Path getPath() { Path path = super.getPath(); - HoodieRealtimePath realtimePath = new HoodieRealtimePath(path.getParent(), path.getName()); - realtimePath.setBelongsToIncrementalQuery(belongsToIncrementalQuery); - realtimePath.setDeltaLogFiles(deltaLogFiles); - realtimePath.setMaxCommitTime(maxCommitTime); - realtimePath.setBasePath(basePath); - realtimePath.setVirtualKeyInfo(virtualKeyInfo); + HoodieRealtimePath realtimePath = new HoodieRealtimePath(path.getParent(), path.getName(), basePath, + deltaLogFiles, maxCommitTime, belongsToIncrementalQuery, virtualKeyInfo); if (bootStrapFileStatus != null) { realtimePath.setPathWithBootstrapFileStatus((PathWithBootstrapFileStatus)bootStrapFileStatus.getPath()); @@ -84,18 +91,14 @@ public Path getPath() { return realtimePath; } - public void setBelongToIncrementalFileStatus(boolean belongToIncrementalFileStatus) { - this.belongsToIncrementalQuery = belongToIncrementalFileStatus; + public String getBasePath() { + return basePath; } public List getDeltaLogFiles() { return deltaLogFiles; } - public void setDeltaLogFiles(List deltaLogFiles) { - this.deltaLogFiles = deltaLogFiles; - } - public String getMaxCommitTime() { return maxCommitTime; } @@ -104,19 +107,7 @@ public void setMaxCommitTime(String maxCommitTime) { this.maxCommitTime = maxCommitTime; } - public String getBasePath() { - return basePath; - } - - public void setBasePath(String basePath) { - this.basePath = basePath; - } - public void setBootStrapFileStatus(FileStatus bootStrapFileStatus) { this.bootStrapFileStatus = bootStrapFileStatus; } - - public void setVirtualKeyInfo(Option virtualKeyInfo) { - this.virtualKeyInfo = virtualKeyInfo; - } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 354ee7033c0e7..9732c5dedddb2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -156,7 +156,7 @@ protected List listStatusForIncrementalMode(JobConf job, String maxCommitTime = fsView.getLastInstant().get().getTimestamp(); // step6 - result.addAll(collectAllIncrementalFiles(fileGroups, maxCommitTime, basePath.toString(), candidateFileStatus)); + result.addAll(collectAllIncrementalFiles(fileGroups, maxCommitTime, basePath.toString(), candidateFileStatus, tableMetaClient)); return result; } @@ -193,25 +193,28 @@ protected FileSplit makeSplit(Path file, long start, long length, String[] hosts return super.makeSplit(file, start, length, hosts, inMemoryHosts); } - private List collectAllIncrementalFiles(List fileGroups, String maxCommitTime, String basePath, Map candidateFileStatus) { + private List collectAllIncrementalFiles(List fileGroups, + String maxCommitTime, + String basePath, + Map candidateFileStatus, + HoodieTableMetaClient tableMetaClient) { List result = new ArrayList<>(); fileGroups.stream().forEach(f -> { try { List baseFiles = f.getAllFileSlices().filter(slice -> slice.getBaseFile().isPresent()).collect(Collectors.toList()); + Option virtualKeyInfoOpt = HoodieVirtualKeyInfo.compose(tableMetaClient); if (!baseFiles.isEmpty()) { FileStatus baseFileStatus = HoodieInputFormatUtils.getFileStatus(baseFiles.get(0).getBaseFile().get()); String baseFilePath = baseFileStatus.getPath().toUri().toString(); if (!candidateFileStatus.containsKey(baseFilePath)) { throw new HoodieException("Error obtaining fileStatus for file: " + baseFilePath); } + List deltaLogFiles = f.getLatestFileSlice().get().getLogFiles().collect(Collectors.toList()); // We cannot use baseFileStatus.getPath() here, since baseFileStatus.getPath() missing file size information. // So we use candidateFileStatus.get(baseFileStatus.getPath()) to get a correct path. - RealtimeFileStatus fileStatus = new RealtimeFileStatus(candidateFileStatus.get(baseFilePath)); + RealtimeFileStatus fileStatus = new RealtimeFileStatus(candidateFileStatus.get(baseFilePath), + basePath, deltaLogFiles, true, virtualKeyInfoOpt); fileStatus.setMaxCommitTime(maxCommitTime); - fileStatus.setBelongToIncrementalFileStatus(true); - fileStatus.setBasePath(basePath); - fileStatus.setDeltaLogFiles(f.getLatestFileSlice().get().getLogFiles().collect(Collectors.toList())); - // try to set bootstrapfileStatus if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { fileStatus.setBootStrapFileStatus(baseFileStatus); } @@ -221,11 +224,10 @@ private List collectAllIncrementalFiles(List fileGr if (f.getLatestFileSlice().isPresent() && baseFiles.isEmpty()) { List logFileStatus = f.getLatestFileSlice().get().getLogFiles().map(logFile -> logFile.getFileStatus()).collect(Collectors.toList()); if (logFileStatus.size() > 0) { - RealtimeFileStatus fileStatus = new RealtimeFileStatus(logFileStatus.get(0)); - fileStatus.setBelongToIncrementalFileStatus(true); - fileStatus.setDeltaLogFiles(logFileStatus.stream().map(l -> new HoodieLogFile(l.getPath(), l.getLen())).collect(Collectors.toList())); + List deltaLogFiles = logFileStatus.stream().map(l -> new HoodieLogFile(l.getPath(), l.getLen())).collect(Collectors.toList()); + RealtimeFileStatus fileStatus = new RealtimeFileStatus(logFileStatus.get(0), basePath, + deltaLogFiles, true, virtualKeyInfoOpt); fileStatus.setMaxCommitTime(maxCommitTime); - fileStatus.setBasePath(basePath); result.add(fileStatus); } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index 8254c717fcb98..b508c0a6f66f2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.PathWithBootstrapFileStatus; -import java.util.ArrayList; import java.util.List; /** @@ -35,64 +34,60 @@ public class HoodieRealtimePath extends Path { /** * Marks whether this path produced as part of Incremental Query */ - private boolean belongsToIncrementalQuery = false; + private final boolean belongsToIncrementalQuery; /** * List of delta log-files holding updated records for this base-file */ - private List deltaLogFiles = new ArrayList<>(); + private final List deltaLogFiles; /** * Latest commit instant available at the time of the query in which all of the files * pertaining to this split are represented */ - private String maxCommitTime = ""; + private final String maxCommitTime; /** * Base path of the table this path belongs to */ - private String basePath = ""; + private final String basePath; /** - * File status for the Bootstrap file (only relevant if this table is a bootstrapped table + * Virtual key configuration of the table this split belongs to */ - private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; + private final Option virtualKeyInfo; /** - * Virtual key configuration of the table this split belongs to + * File status for the Bootstrap file (only relevant if this table is a bootstrapped table */ - private Option virtualKeyInfo = Option.empty(); + private PathWithBootstrapFileStatus pathWithBootstrapFileStatus; - public HoodieRealtimePath(Path parent, String child) { + public HoodieRealtimePath(Path parent, + String child, + String basePath, + List deltaLogFiles, + String maxCommitTime, + boolean belongsToIncrementalQuery, + Option virtualKeyInfo) { super(parent, child); + this.basePath = basePath; + this.deltaLogFiles = deltaLogFiles; + this.maxCommitTime = maxCommitTime; + this.belongsToIncrementalQuery = belongsToIncrementalQuery; + this.virtualKeyInfo = virtualKeyInfo; } public List getDeltaLogFiles() { return deltaLogFiles; } - public void setDeltaLogFiles(List deltaLogFiles) { - this.deltaLogFiles = deltaLogFiles; - } - public String getMaxCommitTime() { return maxCommitTime; } - public void setMaxCommitTime(String maxCommitTime) { - this.maxCommitTime = maxCommitTime; - } - public String getBasePath() { return basePath; } - public void setBasePath(String basePath) { - this.basePath = basePath; - } - public boolean getBelongsToIncrementalQuery() { return belongsToIncrementalQuery; } - public void setBelongsToIncrementalQuery(boolean belongsToIncrementalQuery) { - this.belongsToIncrementalQuery = belongsToIncrementalQuery; - } public boolean isSplitable() { return !toString().isEmpty(); @@ -113,8 +108,4 @@ public boolean includeBootstrapFilePath() { public Option getVirtualKeyInfo() { return virtualKeyInfo; } - - public void setVirtualKeyInfo(Option virtualKeyInfo) { - this.virtualKeyInfo = virtualKeyInfo; - } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java index 7d92b340e5693..b917f004bcd06 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -54,7 +54,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader private final Set deltaRecordKeys; private final HoodieMergedLogRecordScanner mergedLogRecordScanner; - private int recordKeyIndex = HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS; + private final int recordKeyIndex; private Iterator deltaItr; public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, @@ -64,9 +64,9 @@ public RealtimeCompactedRecordReader(RealtimeSplit split, JobConf job, this.mergedLogRecordScanner = getMergedLogRecordScanner(); this.deltaRecordMap = mergedLogRecordScanner.getRecords(); this.deltaRecordKeys = new HashSet<>(this.deltaRecordMap.keySet()); - if (split.getVirtualKeyInfo().isPresent()) { - this.recordKeyIndex = split.getVirtualKeyInfo().get().getRecordKeyFieldIndex(); - } + this.recordKeyIndex = split.getVirtualKeyInfo() + .map(HoodieVirtualKeyInfo::getRecordKeyFieldIndex) + .orElse(HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS); } /** diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 63abfc87462c7..07cc554502db3 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -18,16 +18,29 @@ package org.apache.hudi.hadoop.utils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; +import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; -import org.apache.hudi.common.model.HoodiePartitionMetadata; -import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -38,27 +51,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.HoodieHFileInputFormat; import org.apache.hudi.hadoop.HoodieParquetInputFormat; -import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; -import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; +import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcSerde; -import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; -import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.Job; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -490,8 +489,7 @@ public static List filterFileStatusForSnapshotMode(JobConf job, Map< } for (Map.Entry> filterLogEntry : filteredLogs.entrySet()) { - RealtimeFileStatus rs = new RealtimeFileStatus(filterLogEntry.getKey()); - rs.setDeltaLogFiles(filterLogEntry.getValue()); + RealtimeFileStatus rs = new RealtimeFileStatus(filterLogEntry.getKey(), "", filterLogEntry.getValue(), false, Option.empty()); returns.add(rs); } } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 7f80535bd3470..5aa37a6195330 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -768,10 +768,11 @@ public void testLogOnlyReader() throws Exception { FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime); // create a split with new log file(s) fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size)); - RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus(new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath())); + RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus( + new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()), + basePath.toString(), + fileSlice.getLogFiles().collect(Collectors.toList())); realtimeFileStatus.setMaxCommitTime(instantTime); - realtimeFileStatus.setBasePath(basePath.toString()); - realtimeFileStatus.setDeltaLogFiles(fileSlice.getLogFiles().collect(Collectors.toList())); HoodieRealtimePath realtimePath = (HoodieRealtimePath) realtimeFileStatus.getPath(); HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(new FileSplit(realtimePath, 0, 0, new String[] {""}), realtimePath); From f07469ca574c93f1cdea163590ea28c722e0e40e Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 19:48:49 -0800 Subject: [PATCH 16/30] XXX --- .../hadoop/realtime/HoodieVirtualKeyInfo.java | 16 ++++++++++++++++ pom.xml | 2 +- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java index 763b80d4a1f8c..df68d579686e1 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java @@ -18,6 +18,10 @@ package org.apache.hudi.hadoop.realtime; +import org.apache.hudi.common.table.HoodieTableConfig; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; + import java.io.Serializable; /** @@ -37,6 +41,18 @@ public HoodieVirtualKeyInfo(String recordKeyField, String partitionPathField, in this.partitionPathFieldIndex = partitionPathFieldIndex; } + public static Option compose(HoodieTableMetaClient metaClient) { + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + if (tableConfig.populateMetaFields()) { + return Option.empty(); + } + + return Option.of( + new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp(), + ) + ); + } + public String getRecordKeyField() { return recordKeyField; } diff --git a/pom.xml b/pom.xml index 8f2152dfa0430..58cc2d00355d1 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,7 @@ 2.11.12 2.12.10 ${scala11.version} - 2.11 + 2.12 0.12 3.3.1 3.0.1 From 3d78efd133ef93650390f6c662d79ec58dea3b7c Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 19:52:22 -0800 Subject: [PATCH 17/30] Properly set up `HoodieVirtualKeyInfo` --- .../HoodieCopyOnWriteTableInputFormat.java | 3 ++- .../HoodieMergeOnReadTableInputFormat.java | 2 +- .../hadoop/realtime/HoodieVirtualKeyInfo.java | 16 --------------- .../utils/HoodieRealtimeInputFormatUtils.java | 20 +++++++++++++++++++ 4 files changed, 23 insertions(+), 18 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 32543ca6f619d..46ae1addb7f20 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -47,6 +47,7 @@ import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; import javax.annotation.Nonnull; import java.io.IOException; @@ -316,7 +317,7 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFi HoodieTableMetaClient tableMetaClient) { FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - Option virtualKeyInfoOpt = HoodieVirtualKeyInfo.compose(tableMetaClient); + Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); try { RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, tableMetaClient.getBasePath(), sortedLogFiles, diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 9732c5dedddb2..b13f14592dbf9 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -202,7 +202,7 @@ private List collectAllIncrementalFiles(List fileGr fileGroups.stream().forEach(f -> { try { List baseFiles = f.getAllFileSlices().filter(slice -> slice.getBaseFile().isPresent()).collect(Collectors.toList()); - Option virtualKeyInfoOpt = HoodieVirtualKeyInfo.compose(tableMetaClient); + Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); if (!baseFiles.isEmpty()) { FileStatus baseFileStatus = HoodieInputFormatUtils.getFileStatus(baseFiles.get(0).getBaseFile().get()); String baseFilePath = baseFileStatus.getPath().toUri().toString(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java index df68d579686e1..763b80d4a1f8c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieVirtualKeyInfo.java @@ -18,10 +18,6 @@ package org.apache.hudi.hadoop.realtime; -import org.apache.hudi.common.table.HoodieTableConfig; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.Option; - import java.io.Serializable; /** @@ -41,18 +37,6 @@ public HoodieVirtualKeyInfo(String recordKeyField, String partitionPathField, in this.partitionPathFieldIndex = partitionPathFieldIndex; } - public static Option compose(HoodieTableMetaClient metaClient) { - HoodieTableConfig tableConfig = metaClient.getTableConfig(); - if (tableConfig.populateMetaFields()) { - return Option.empty(); - } - - return Option.of( - new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp(), - ) - ); - } - public String getRecordKeyField() { return recordKeyField; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index 396782d96eeec..dcfee743a25d2 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -28,7 +28,9 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; @@ -40,6 +42,7 @@ import org.apache.hudi.hadoop.realtime.RealtimeSplit; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import org.apache.parquet.schema.MessageType; import java.util.ArrayList; import java.util.HashSet; @@ -67,6 +70,23 @@ public static boolean doesBelongToIncrementalQuery(FileSplit s) { return false; } + public static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + if (tableConfig.populateMetaFields()) { + return Option.empty(); + } + + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); + try { + MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema(); + return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()), + parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp()))); + } catch (Exception exception) { + throw new HoodieException("Fetching table schema failed with exception ", exception); + } + } + // Return parquet file with a list of log files in the same file group. public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { Set partitionSet = new HashSet<>(partitionPaths); From 3e8382092e4aa383ae15789f573c11d154aa8a53 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 20:06:25 -0800 Subject: [PATCH 18/30] Separate COW/MOR `InputFormat`s to make sure that no MOR-specific logic is spilled into COW impl --- .../HoodieCopyOnWriteTableInputFormat.java | 99 +++---------------- .../HoodieMergeOnReadTableInputFormat.java | 79 +++++++++++++++ 2 files changed, 91 insertions(+), 87 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 46ae1addb7f20..0eb0e825f02b7 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -36,7 +36,6 @@ import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; -import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableQueryType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -44,10 +43,8 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils; import javax.annotation.Nonnull; import java.io.IOException; @@ -59,7 +56,6 @@ import java.util.Map; import java.util.Properties; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -202,6 +198,16 @@ protected List listStatusForIncrementalMode(JobConf job, return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get()); } + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient tableMetaClient) { + Option baseFileOpt = fileSlice.getBaseFile(); + + if (baseFileOpt.isPresent()) { + return getFileStatusUnchecked(baseFileOpt.get()); + } else { + throw new IllegalStateException("Invalid state: base-file has to be present"); + } + } + private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus file, FileSplit split) { try { LOG.info("Making external data split for " + file); @@ -259,30 +265,7 @@ private List listStatusForSnapshotMode(JobConf job, partitionedFileSlices.values() .stream() .flatMap(Collection::stream) - .map(fileSlice -> { - Option baseFileOpt = fileSlice.getBaseFile(); - Option latestLogFileOpt = fileSlice.getLatestLogFile(); - Stream logFiles = fileSlice.getLogFiles(); - - Option latestCompletedInstantOpt = fileIndex.getLatestCompletedInstant(); - - // Check if we're reading a MOR table - if (includeLogFilesForSnapshotView()) { - if (baseFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); - } else if (latestLogFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); - } else { - throw new IllegalStateException("Invalid state: either base-file or log-file has to be present"); - } - } else { - if (baseFileOpt.isPresent()) { - return getFileStatusUnchecked(baseFileOpt.get()); - } else { - throw new IllegalStateException("Invalid state: base-file has to be present"); - } - } - }) + .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex, tableMetaClient)) .collect(Collectors.toList()) ); } @@ -299,69 +282,11 @@ private void validate(List targetFiles, List legacyFileS } @Nonnull - private static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { + protected static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { try { return HoodieInputFormatUtils.getFileStatus(baseFile); } catch (IOException ioe) { throw new HoodieIOException("Failed to get file-status", ioe); } } - - /** - * Creates {@link RealtimeFileStatus} for the file-slice where base file is present - */ - @Nonnull - private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, - Stream logFiles, - Option latestCompletedInstantOpt, - HoodieTableMetaClient tableMetaClient) { - FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); - List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); - - try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, tableMetaClient.getBasePath(), sortedLogFiles, - false, virtualKeyInfoOpt); - - 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); - } - } - - /** - * Creates {@link RealtimeFileStatus} for the file-slice where base file is NOT present - */ - @Nonnull - private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, - Stream logFiles, - Option latestCompletedInstantOpt, - HoodieTableMetaClient tableMetaClient) { - List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus(), tableMetaClient.getBasePath(), sortedLogFiles); - - 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); - } - } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index b13f14592dbf9..5516f6bf1ab9e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -29,6 +29,7 @@ import org.apache.hadoop.mapred.SplitLocationInfo; import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieLogFile; @@ -41,6 +42,7 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.BootstrapBaseFileSplit; import org.apache.hudi.hadoop.FileStatusWithBootstrapBaseFile; +import org.apache.hudi.hadoop.HiveHoodieTableFileIndex; import org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; import org.apache.hudi.hadoop.RealtimeFileStatus; @@ -54,6 +56,9 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.common.util.ValidationUtils.checkState; /** * Base implementation of the Hive's {@link FileInputFormat} allowing for reading of Hudi's @@ -79,6 +84,24 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { .toArray(new FileSplit[0]); } + @Override + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient tableMetaClient) { + Option baseFileOpt = fileSlice.getBaseFile(); + Option latestLogFileOpt = fileSlice.getLatestLogFile(); + Stream logFiles = fileSlice.getLogFiles(); + + Option latestCompletedInstantOpt = fileIndex.getLatestCompletedInstant(); + + // Check if we're reading a MOR table + if (baseFileOpt.isPresent()) { + return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); + } else if (latestLogFileOpt.isPresent()) { + return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); + } else { + throw new IllegalStateException("Invalid state: either base-file or log-file has to be present"); + } + } + /** * Keep the logic of mor_incr_view as same as spark datasource. * Step1: Get list of commits to be fetched based on start commit and max commits(for snapshot max commits is -1). @@ -294,5 +317,61 @@ private static HoodieRealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFi throw new HoodieIOException("Error creating hoodie real time split ", e); } } + + /** + * Creates {@link RealtimeFileStatus} for the file-slice where base file is present + */ + private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, + Stream logFiles, + Option latestCompletedInstantOpt, + HoodieTableMetaClient tableMetaClient) { + FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); + List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); + Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); + + try { + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, tableMetaClient.getBasePath(), sortedLogFiles, + false, virtualKeyInfoOpt); + + 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); + } + } + + /** + * Creates {@link RealtimeFileStatus} for the file-slice where base file is NOT present + */ + private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, + Stream logFiles, + Option latestCompletedInstantOpt, + HoodieTableMetaClient tableMetaClient) { + List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); + try { + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus(), tableMetaClient.getBasePath(), sortedLogFiles); + + 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); + } + } } From ee70c2fd6cb76a0409f6af291a0cb56867cb0ad0 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 20:07:47 -0800 Subject: [PATCH 19/30] Tidying up --- .../hadoop/realtime/HoodieMergeOnReadTableInputFormat.java | 3 +-- .../org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 5516f6bf1ab9e..6a72661956a24 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -216,7 +216,7 @@ protected FileSplit makeSplit(Path file, long start, long length, String[] hosts return super.makeSplit(file, start, length, hosts, inMemoryHosts); } - private List collectAllIncrementalFiles(List fileGroups, + private static List collectAllIncrementalFiles(List fileGroups, String maxCommitTime, String basePath, Map candidateFileStatus, @@ -289,7 +289,6 @@ private static List filterIncrementalQueryFileSplits(List .collect(Collectors.toList()); } - private static HoodieRealtimeFileSplit createRealtimeFileSplit(HoodieRealtimePath path, long start, long length, String[] hosts) { try { return new HoodieRealtimeFileSplit(new FileSplit(path, start, length, hosts), path); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java index b508c0a6f66f2..bba44d5c6632c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java @@ -88,7 +88,6 @@ public boolean getBelongsToIncrementalQuery() { return belongsToIncrementalQuery; } - public boolean isSplitable() { return !toString().isEmpty(); } From f2dde38bdd04db4cd66e9fb768d85e5c3d97cd6d Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 20:42:03 -0800 Subject: [PATCH 20/30] Made sure Virtual Key info is fetched once for individual table --- .../apache/hudi/BaseHoodieTableFileIndex.java | 7 ++++ .../HoodieCopyOnWriteTableInputFormat.java | 28 +++++++++++++-- .../HoodieMergeOnReadTableInputFormat.java | 36 ++++++++++--------- .../utils/HoodieRealtimeInputFormatUtils.java | 20 ----------- .../TestHoodieRealtimeRecordReader.java | 4 ++- 5 files changed, 56 insertions(+), 39 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index d4bf2a0fd8fb7..f63f8ea420038 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -142,6 +142,13 @@ public Option getLatestCompletedInstant() { return getActiveTimeline().filterCompletedInstants().lastInstant(); } + /** + * Returns table's base-path + */ + public String getBasePath() { + return metaClient.getBasePath(); + } + /** * Fetch list of latest base files and log files per partition. * diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 0eb0e825f02b7..ade9517a5075e 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -37,14 +37,19 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieTableQueryType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.realtime.HoodieVirtualKeyInfo; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; +import org.apache.parquet.schema.MessageType; import javax.annotation.Nonnull; import java.io.IOException; @@ -198,7 +203,7 @@ protected List listStatusForIncrementalMode(JobConf job, return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get()); } - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient tableMetaClient) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, Option virtualKeyInfoOpt) { Option baseFileOpt = fileSlice.getBaseFile(); if (baseFileOpt.isPresent()) { @@ -261,11 +266,13 @@ private List listStatusForSnapshotMode(JobConf job, Map> partitionedFileSlices = fileIndex.listFileSlices(); + Option virtualKeyInfoOpt = getHoodieVirtualKeyInfo(tableMetaClient); + targetFiles.addAll( partitionedFileSlices.values() .stream() .flatMap(Collection::stream) - .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex, tableMetaClient)) + .map(fileSlice -> createFileStatusUnchecked(fileSlice, fileIndex, virtualKeyInfoOpt)) .collect(Collectors.toList()) ); } @@ -289,4 +296,21 @@ protected static FileStatus getFileStatusUnchecked(HoodieBaseFile baseFile) { throw new HoodieIOException("Failed to get file-status", ioe); } } + + protected static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + if (tableConfig.populateMetaFields()) { + return Option.empty(); + } + + TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); + try { + MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema(); + return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), + tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()), + parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp()))); + } catch (Exception exception) { + throw new HoodieException("Fetching table schema failed with exception ", exception); + } + } } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 6a72661956a24..42a94aefdeb4d 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -69,7 +69,7 @@ *
  • Incremental mode: reading table's state as of particular timestamp (or instant, in Hudi's terms)
  • *
  • External mode: reading non-Hudi partitions
  • * - * + *

    * NOTE: This class is invariant of the underlying file-format of the files being read */ public class HoodieMergeOnReadTableInputFormat extends HoodieCopyOnWriteTableInputFormat implements Configurable { @@ -85,18 +85,19 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { } @Override - protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, HoodieTableMetaClient tableMetaClient) { + protected FileStatus createFileStatusUnchecked(FileSlice fileSlice, HiveHoodieTableFileIndex fileIndex, Option virtualKeyInfoOpt) { Option baseFileOpt = fileSlice.getBaseFile(); Option latestLogFileOpt = fileSlice.getLatestLogFile(); Stream logFiles = fileSlice.getLogFiles(); Option latestCompletedInstantOpt = fileIndex.getLatestCompletedInstant(); + String tableBasePath = fileIndex.getBasePath(); // Check if we're reading a MOR table if (baseFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); + return createRealtimeFileStatusUnchecked(baseFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, virtualKeyInfoOpt); } else if (latestLogFileOpt.isPresent()) { - return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, latestCompletedInstantOpt, tableMetaClient); + return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), logFiles, tableBasePath, latestCompletedInstantOpt, virtualKeyInfoOpt); } else { throw new IllegalStateException("Invalid state: either base-file or log-file has to be present"); } @@ -177,9 +178,10 @@ protected List listStatusForIncrementalMode(JobConf job, candidateFileStatus.put(key, fileStatuses[i]); } + Option virtualKeyInfoOpt = getHoodieVirtualKeyInfo(tableMetaClient); String maxCommitTime = fsView.getLastInstant().get().getTimestamp(); // step6 - result.addAll(collectAllIncrementalFiles(fileGroups, maxCommitTime, basePath.toString(), candidateFileStatus, tableMetaClient)); + result.addAll(collectAllIncrementalFiles(fileGroups, maxCommitTime, basePath.toString(), candidateFileStatus, virtualKeyInfoOpt)); return result; } @@ -191,7 +193,7 @@ protected boolean includeLogFilesForSnapshotView() { @Override protected boolean isSplitable(FileSystem fs, Path filename) { if (filename instanceof HoodieRealtimePath) { - return ((HoodieRealtimePath)filename).isSplitable(); + return ((HoodieRealtimePath) filename).isSplitable(); } return super.isSplitable(fs, filename); @@ -217,15 +219,15 @@ protected FileSplit makeSplit(Path file, long start, long length, String[] hosts } private static List collectAllIncrementalFiles(List fileGroups, - String maxCommitTime, - String basePath, - Map candidateFileStatus, - HoodieTableMetaClient tableMetaClient) { + String maxCommitTime, + String basePath, + Map candidateFileStatus, + Option virtualKeyInfoOpt) { + List result = new ArrayList<>(); fileGroups.stream().forEach(f -> { try { List baseFiles = f.getAllFileSlices().filter(slice -> slice.getBaseFile().isPresent()).collect(Collectors.toList()); - Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); if (!baseFiles.isEmpty()) { FileStatus baseFileStatus = HoodieInputFormatUtils.getFileStatus(baseFiles.get(0).getBaseFile().get()); String baseFilePath = baseFileStatus.getPath().toUri().toString(); @@ -322,14 +324,14 @@ private static HoodieRealtimeBootstrapBaseFileSplit createRealtimeBoostrapBaseFi */ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFile baseFile, Stream logFiles, + String basePath, Option latestCompletedInstantOpt, - HoodieTableMetaClient tableMetaClient) { + Option virtualKeyInfoOpt) { FileStatus baseFileStatus = getFileStatusUnchecked(baseFile); List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - Option virtualKeyInfoOpt = HoodieRealtimeInputFormatUtils.getHoodieVirtualKeyInfo(tableMetaClient); try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, tableMetaClient.getBasePath(), sortedLogFiles, + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(baseFileStatus, basePath, sortedLogFiles, false, virtualKeyInfoOpt); if (latestCompletedInstantOpt.isPresent()) { @@ -354,11 +356,13 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFi */ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, Stream logFiles, + String basePath, Option latestCompletedInstantOpt, - HoodieTableMetaClient tableMetaClient) { + Option virtualKeyInfoOpt) { List sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); try { - RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus(), tableMetaClient.getBasePath(), sortedLogFiles); + RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus(), basePath, + sortedLogFiles, false, virtualKeyInfoOpt); if (latestCompletedInstantOpt.isPresent()) { HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java index dcfee743a25d2..396782d96eeec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeInputFormatUtils.java @@ -28,9 +28,7 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; @@ -42,7 +40,6 @@ import org.apache.hudi.hadoop.realtime.RealtimeSplit; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; import java.util.ArrayList; import java.util.HashSet; @@ -70,23 +67,6 @@ public static boolean doesBelongToIncrementalQuery(FileSplit s) { return false; } - public static Option getHoodieVirtualKeyInfo(HoodieTableMetaClient metaClient) { - HoodieTableConfig tableConfig = metaClient.getTableConfig(); - if (tableConfig.populateMetaFields()) { - return Option.empty(); - } - - TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); - try { - MessageType parquetSchema = tableSchemaResolver.getTableParquetSchema(); - return Option.of(new HoodieVirtualKeyInfo(tableConfig.getRecordKeyFieldProp(), - tableConfig.getPartitionFieldProp(), parquetSchema.getFieldIndex(tableConfig.getRecordKeyFieldProp()), - parquetSchema.getFieldIndex(tableConfig.getPartitionFieldProp()))); - } catch (Exception exception) { - throw new HoodieException("Fetching table schema failed with exception ", exception); - } - } - // Return parquet file with a list of log files in the same file group. public static List, List>> groupLogsByBaseFile(Configuration conf, List partitionPaths) { Set partitionSet = new HashSet<>(partitionPaths); diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index 5aa37a6195330..a66cdc7798e42 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -771,7 +771,9 @@ public void testLogOnlyReader() throws Exception { RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus( new FileStatus(writer.getLogFile().getFileSize(), false, 1, 1, 0, writer.getLogFile().getPath()), basePath.toString(), - fileSlice.getLogFiles().collect(Collectors.toList())); + fileSlice.getLogFiles().collect(Collectors.toList()), + false, + Option.empty()); realtimeFileStatus.setMaxCommitTime(instantTime); HoodieRealtimePath realtimePath = (HoodieRealtimePath) realtimeFileStatus.getPath(); HoodieRealtimeFileSplit split = From 4a68cff6b5960e71552276b840d652b4c539a77a Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Thu, 10 Feb 2022 20:42:28 -0800 Subject: [PATCH 21/30] Tidying up --- .../org/apache/hudi/hadoop/RealtimeFileStatus.java | 8 -------- .../hadoop/realtime/HoodieRealtimeFileSplit.java | 14 +++++++------- 2 files changed, 7 insertions(+), 15 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java index abd3cb4e85a61..641aa2759ff20 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/RealtimeFileStatus.java @@ -73,10 +73,6 @@ public RealtimeFileStatus(FileStatus fileStatus, this.virtualKeyInfo = virtualKeyInfo; } - public RealtimeFileStatus(FileStatus fileStatus, String basePath, List deltaLogFiles) throws IOException { - this(fileStatus, basePath, deltaLogFiles, false, Option.empty()); - } - @Override public Path getPath() { Path path = super.getPath(); @@ -91,10 +87,6 @@ public Path getPath() { return realtimePath; } - public String getBasePath() { - return basePath; - } - public List getDeltaLogFiles() { return deltaLogFiles; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java index 895ffb209ef95..a424f021c2d20 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimeFileSplit.java @@ -41,23 +41,23 @@ * in {@link #readFromInput(DataInput)} and {@link #writeToOutput(DataOutput)} */ public class HoodieRealtimeFileSplit extends FileSplit implements RealtimeSplit { - /** - * Marks whether this path produced as part of Incremental Query - */ - private boolean belongsToIncrementalQuery = false; /** * List of delta log-files holding updated records for this base-file */ private List deltaLogFiles = new ArrayList<>(); + /** + * Base path of the table this path belongs to + */ + private String basePath; /** * Latest commit instant available at the time of the query in which all of the files * pertaining to this split are represented */ private String maxCommitTime; /** - * Base path of the table this path belongs to + * Marks whether this path produced as part of Incremental Query */ - private String basePath; + private boolean belongsToIncrementalQuery = false; /** * Virtual key configuration of the table this split belongs to */ @@ -88,8 +88,8 @@ public HoodieRealtimeFileSplit(FileSplit baseSplit, throws IOException { super(baseSplit.getPath(), baseSplit.getStart(), baseSplit.getLength(), baseSplit.getLocations()); this.deltaLogFiles = deltaLogFiles; - this.maxCommitTime = maxCommitTime; this.basePath = basePath; + this.maxCommitTime = maxCommitTime; this.belongsToIncrementalQuery = belongsToIncrementalQuery; this.virtualKeyInfo = virtualKeyInfo; } From d357ea2d8e7519490bc2a91d48b978c3768d8a07 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 11 Feb 2022 09:07:47 -0800 Subject: [PATCH 22/30] Reverting inadvertent change --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 58cc2d00355d1..8f2152dfa0430 100644 --- a/pom.xml +++ b/pom.xml @@ -126,7 +126,7 @@ 2.11.12 2.12.10 ${scala11.version} - 2.12 + 2.11 0.12 3.3.1 3.0.1 From a1246304af3e587cac37779f8b2ca9e6dec497c1 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 11 Feb 2022 15:00:58 -0800 Subject: [PATCH 23/30] Fixed incorrect `blockSize` being set in `FileStatus` object when listing from `HoodieCommitMetadata` --- .../common/model/HoodieCommitMetadata.java | 14 ++++++++++---- .../partitioner/profile/WriteProfiles.java | 19 ++++++++++--------- .../HoodieMergeOnReadTableInputFormat.java | 2 +- .../hadoop/utils/HoodieInputFormatUtils.java | 4 ++-- .../TestHoodieRealtimeRecordReader.java | 2 +- .../hudi/MergeOnReadIncrementalRelation.scala | 17 +++++------------ 6 files changed, 29 insertions(+), 29 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index d693d91f676fc..f2679a41c8bc0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.PropertyAccessor; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.hudi.common.fs.FSUtils; @@ -147,10 +148,12 @@ public Map getFileGroupIdAndFullPaths(String basePath * been touched multiple times in the given commits, the return value will keep the one * from the latest commit. * + * + * @param hadoopConf * @param basePath The base path * @return the file full path to file status mapping */ - public Map getFullPathToFileStatus(String basePath) { + public Map getFullPathToFileStatus(Configuration hadoopConf, String basePath) { Map fullPathToFileStatus = new HashMap<>(); for (List stats : getPartitionToWriteStats().values()) { // Iterate through all the written files. @@ -158,7 +161,8 @@ public Map getFullPathToFileStatus(String basePath) { String relativeFilePath = stat.getPath(); Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null; if (fullPath != null) { - FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, 0, + long blockSize = FSUtils.getFs(fullPath.toString(), hadoopConf).getDefaultBlockSize(fullPath); + FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, blockSize, 0, fullPath); fullPathToFileStatus.put(fullPath.getName(), fileStatus); } @@ -172,14 +176,16 @@ public Map getFullPathToFileStatus(String basePath) { * been touched multiple times in the given commits, the return value will keep the one * from the latest commit by file group ID. * - *

    Note: different with {@link #getFullPathToFileStatus(String)}, + *

    Note: different with {@link #getFullPathToFileStatus(Configuration, String)}, * only the latest commit file for a file group is returned, * this is an optimization for COPY_ON_WRITE table to eliminate legacy files for filesystem view. * + * + * @param hadoopConf * @param basePath The base path * @return the file ID to file status mapping */ - public Map getFileIdToFileStatus(String basePath) { + public Map getFileIdToFileStatus(Configuration hadoopConf, String basePath) { Map fileIdToFileStatus = new HashMap<>(); for (List stats : getPartitionToWriteStats().values()) { // Iterate through all the written files. diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java index e8aafd830f10f..405522802c368 100644 --- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java +++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java @@ -18,6 +18,10 @@ package org.apache.hudi.sink.partitioner.profile; +import org.apache.flink.core.fs.Path; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -29,11 +33,6 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.util.StreamerUtil; - -import org.apache.flink.core.fs.Path; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -117,7 +116,7 @@ private static Map getFilesToReadOfInstant( HoodieCommitMetadata metadata, FileSystem fs, HoodieTableType tableType) { - return getFilesToRead(metadata, basePath.toString(), tableType).entrySet().stream() + return getFilesToRead(fs.getConf(), metadata, basePath.toString(), tableType).entrySet().stream() // filter out the file paths that does not exist, some files may be cleaned by // the cleaner. .filter(entry -> { @@ -133,14 +132,16 @@ private static Map getFilesToReadOfInstant( } private static Map getFilesToRead( + Configuration hadoopConf, HoodieCommitMetadata metadata, String basePath, - HoodieTableType tableType) { + HoodieTableType tableType + ) { switch (tableType) { case COPY_ON_WRITE: - return metadata.getFileIdToFileStatus(basePath); + return metadata.getFileIdToFileStatus(hadoopConf, basePath); case MERGE_ON_READ: - return metadata.getFullPathToFileStatus(basePath); + return metadata.getFullPathToFileStatus(hadoopConf, basePath); default: throw new AssertionError(); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 42a94aefdeb4d..8260312c96f9f 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -152,7 +152,7 @@ protected List listStatusForIncrementalMode(JobConf job, // build fileGroup from fsView List affectedFileStatus = Arrays.asList(HoodieInputFormatUtils - .listAffectedFilesForCommits(new Path(tableMetaClient.getBasePath()), metadataList)); + .listAffectedFilesForCommits(job, new Path(tableMetaClient.getBasePath()), metadataList)); // step3 HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(tableMetaClient, commitsTimelineToReturn, affectedFileStatus.toArray(new FileStatus[0])); // build fileGroup from fsView diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 07cc554502db3..368e1fe8a9d85 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -532,12 +532,12 @@ private static HoodieBaseFile refreshFileStatus(Configuration conf, HoodieBaseFi * * @return the affected file status array */ - public static FileStatus[] listAffectedFilesForCommits(Path basePath, List metadataList) { + public static FileStatus[] listAffectedFilesForCommits(Configuration hadoopConf, Path basePath, List metadataList) { // TODO: Use HoodieMetaTable to extract affected file directly. HashMap fullPathToFileStatus = new HashMap<>(); // Iterate through the given commits. for (HoodieCommitMetadata metadata: metadataList) { - fullPathToFileStatus.putAll(metadata.getFullPathToFileStatus(basePath.toString())); + fullPathToFileStatus.putAll(metadata.getFullPathToFileStatus(hadoopConf, basePath.toString())); } return fullPathToFileStatus.values().toArray(new FileStatus[0]); } diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index a66cdc7798e42..fc4eb7ce2c042 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -614,7 +614,7 @@ public void testIncrementalWithOnlylog() throws Exception { HoodieParquetRealtimeInputFormat inputFormat = new HoodieParquetRealtimeInputFormat(); inputFormat.setConf(baseJobConf); InputSplit[] splits = inputFormat.getSplits(baseJobConf, 1); - assertTrue(splits.length == 1); + assertEquals(1, splits.length); JobConf newJobConf = new JobConf(baseJobConf); List fields = schema.getFields(); setHiveColumnNameProps(fields, newJobConf, false); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala index db9ebeff4d7b6..cda7b81f02e0a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala @@ -17,24 +17,17 @@ package org.apache.hudi +import org.apache.hadoop.fs.{GlobPattern, Path} +import org.apache.hadoop.mapred.JobConf import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.view.HoodieTableFileSystemView -import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.exception.HoodieException -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listAffectedFilesForCommits -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getCommitMetadata -import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getWritePartitionPaths +import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.{getCommitMetadata, getWritePartitionPaths, listAffectedFilesForCommits} import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes - -import org.apache.hadoop.fs.{GlobPattern, Path} -import org.apache.hadoop.mapred.JobConf - -import org.apache.log4j.LogManager - import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SQLContext} @@ -167,7 +160,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext, def buildFileIndex(): List[HoodieMergeOnReadFileSplit] = { val metadataList = commitsToReturn.map(instant => getCommitMetadata(instant, commitsTimelineToReturn)) - val affectedFileStatus = listAffectedFilesForCommits(new Path(metaClient.getBasePath), metadataList) + val affectedFileStatus = listAffectedFilesForCommits(conf, new Path(metaClient.getBasePath), metadataList) val fsView = new HoodieTableFileSystemView(metaClient, commitsTimelineToReturn, affectedFileStatus) // Iterate partitions to create splits From 5d842accaf5d504e2b851d05a3f83777cc8d034a Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 11 Feb 2022 16:47:06 -0800 Subject: [PATCH 24/30] Cleaned up deprecated code --- .../HoodieCopyOnWriteTableInputFormat.java | 12 ---- .../hudi/hadoop/HoodieHFileInputFormat.java | 5 -- .../hadoop/HoodieParquetInputFormatBase.java | 2 +- .../HoodieMergeOnReadTableInputFormat.java | 5 -- .../hadoop/utils/HoodieInputFormatUtils.java | 69 ------------------- 5 files changed, 1 insertion(+), 92 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index ade9517a5075e..229a086abb4ec 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -162,10 +162,6 @@ public RecordReader getRecordReader(InputSplit spli throw new UnsupportedEncodingException("not implemented"); } - protected boolean includeLogFilesForSnapshotView() { - return false; - } - /** * 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 @@ -225,11 +221,6 @@ private BootstrapBaseFileSplit makeExternalFileSplit(PathWithBootstrapFileStatus } } - @Nonnull - private List listStatusForSnapshotModeLegacy(JobConf job, Map tableMetaClientMap, List snapshotPaths) throws IOException { - return HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapshotView()); - } - @Nonnull private List listStatusForSnapshotMode(JobConf job, Map tableMetaClientMap, @@ -277,9 +268,6 @@ private List listStatusForSnapshotMode(JobConf job, ); } - // TODO(HUDI-3280) cleanup - validate(targetFiles, listStatusForSnapshotModeLegacy(job, tableMetaClientMap, snapshotPaths)); - return targetFiles; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java index 0f0f736243c33..6eb1663a0d12c 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java @@ -41,11 +41,6 @@ protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline tim return HoodieInputFormatUtils.filterInstantsTimeline(timeline); } - @Override - protected boolean includeLogFilesForSnapshotView() { - return false; - } - @Override public RecordReader getRecordReader(final InputSplit split, final JobConf job, final Reporter reporter) throws IOException { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormatBase.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormatBase.java index 4f8d07bf8f5c6..ed88acacb4d2f 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormatBase.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormatBase.java @@ -47,7 +47,7 @@ */ public abstract class HoodieParquetInputFormatBase extends MapredParquetInputFormat implements Configurable { - private final HoodieCopyOnWriteTableInputFormat inputFormatDelegate; + private final HoodieTableInputFormat inputFormatDelegate; protected HoodieParquetInputFormatBase(HoodieCopyOnWriteTableInputFormat inputFormatDelegate) { this.inputFormatDelegate = inputFormatDelegate; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java index 8260312c96f9f..982d52b0d4807 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieMergeOnReadTableInputFormat.java @@ -185,11 +185,6 @@ protected List listStatusForIncrementalMode(JobConf job, return result; } - @Override - protected boolean includeLogFilesForSnapshotView() { - return true; - } - @Override protected boolean isSplitable(FileSystem fs, Path filename) { if (filename instanceof HoodieRealtimePath) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java index 368e1fe8a9d85..7fec1fb63f6fa 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java @@ -29,23 +29,18 @@ import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileFormat; -import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.Option; @@ -55,7 +50,6 @@ import org.apache.hudi.hadoop.HoodieHFileInputFormat; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile; -import org.apache.hudi.hadoop.RealtimeFileStatus; import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat; import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; import org.apache.log4j.LogManager; @@ -436,69 +430,6 @@ public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) { .build(); } - /** - * @deprecated - */ - public static List filterFileStatusForSnapshotMode(JobConf job, Map tableMetaClientMap, - List snapshotPaths, boolean includeLogFiles) throws IOException { - HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job); - List returns = new ArrayList<>(); - - Map> groupedPaths = - HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths); - - Map fsViewCache = new HashMap<>(); - - LOG.info("Found a total of " + groupedPaths.size() + " groups"); - - try { - for (Map.Entry> entry : groupedPaths.entrySet()) { - HoodieTableMetaClient metaClient = entry.getKey(); - if (LOG.isDebugEnabled()) { - LOG.debug("Hoodie Metadata initialized with completed commit instant as :" + metaClient); - } - - HoodieTimeline timeline = HoodieHiveUtils.getTableTimeline(metaClient.getTableConfig().getTableName(), job, metaClient); - - HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient -> - FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline)); - List filteredBaseFiles = new ArrayList<>(); - Map> filteredLogs = new HashMap<>(); - for (Path p : entry.getValue()) { - String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p); - List matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList()); - filteredBaseFiles.addAll(matched); - if (includeLogFiles) { - List logMatched = fsView.getLatestFileSlices(relativePartitionPath) - .filter(f -> !f.getBaseFile().isPresent() && f.getLatestLogFile().isPresent()) - .collect(Collectors.toList()); - logMatched.forEach(f -> { - List logPathSizePairs = f.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList()); - filteredLogs.put(f.getLatestLogFile().get().getFileStatus(), logPathSizePairs); - }); - } - } - - LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size()); - for (HoodieBaseFile filteredFile : filteredBaseFiles) { - if (LOG.isDebugEnabled()) { - LOG.debug("Processing latest hoodie file - " + filteredFile.getPath()); - } - filteredFile = refreshFileStatus(job, filteredFile); - returns.add(getFileStatus(filteredFile)); - } - - for (Map.Entry> filterLogEntry : filteredLogs.entrySet()) { - RealtimeFileStatus rs = new RealtimeFileStatus(filterLogEntry.getKey(), "", filterLogEntry.getValue(), false, Option.empty()); - returns.add(rs); - } - } - } finally { - fsViewCache.forEach(((metaClient, fsView) -> fsView.close())); - } - return returns; - } - /** * Checks the file status for a race condition which can set the file size to 0. 1. HiveInputFormat does * super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed. From d8beebf4cc51cc6ec6a6342c35091d1c3a3ca755 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Fri, 11 Feb 2022 16:48:11 -0800 Subject: [PATCH 25/30] Extracted base `HoodieTableInputFormat` --- .../HoodieCopyOnWriteTableInputFormat.java | 17 +----- .../hudi/hadoop/HoodieTableInputFormat.java | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+), 16 deletions(-) create mode 100644 hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 229a086abb4ec..2b8dae255e3c4 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -18,8 +18,6 @@ package org.apache.hudi.hadoop; -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; @@ -76,20 +74,7 @@ * * NOTE: This class is invariant of the underlying file-format of the files being read */ -public class HoodieCopyOnWriteTableInputFormat extends FileInputFormat - implements Configurable { - - protected Configuration conf; - - @Override - public final Configuration getConf() { - return conf; - } - - @Override - public final void setConf(Configuration conf) { - this.conf = conf; - } +public class HoodieCopyOnWriteTableInputFormat extends HoodieTableInputFormat { @Override protected boolean isSplitable(FileSystem fs, Path filename) { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java new file mode 100644 index 0000000000000..05156ac38eac7 --- /dev/null +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java @@ -0,0 +1,54 @@ +package org.apache.hudi.hadoop; + +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 java.io.IOException; + +/** + * Abstract base class of the Hive's {@link FileInputFormat} implementations allowing for reading of Hudi's + * Copy-on-Write (COW) and Merge-on-Read (MOR) tables + */ +public abstract class HoodieTableInputFormat extends FileInputFormat + implements Configurable { + + protected Configuration conf; + + @Override + public final Configuration getConf() { + return conf; + } + + @Override + public final void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + protected boolean isSplitable(FileSystem fs, Path filename) { + return super.isSplitable(fs, filename); + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, String[] hosts) { + return super.makeSplit(file, start, length, hosts); + } + + @Override + protected FileSplit makeSplit(Path file, long start, long length, String[] hosts, String[] inMemoryHosts) { + return super.makeSplit(file, start, length, hosts, inMemoryHosts); + } + + @Override + protected FileStatus[] listStatus(JobConf job) throws IOException { + return super.listStatus(job); + } +} From b3a526497dc10d9e39b0bc7a580a0e534e6b35bd Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Mon, 14 Feb 2022 12:14:43 -0800 Subject: [PATCH 26/30] Misisng license --- .../hudi/hadoop/HoodieTableInputFormat.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java index 05156ac38eac7..d18cb7895ad00 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieTableInputFormat.java @@ -1,3 +1,21 @@ +/* + * 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.hadoop; import org.apache.hadoop.conf.Configurable; From d1f40f1717b24df882025c364122d26b63922ba0 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Tue, 15 Feb 2022 18:38:28 -0800 Subject: [PATCH 27/30] Added missing validation for whether instant is present in the timeline --- .../java/org/apache/hudi/BaseHoodieTableFileIndex.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index f63f8ea420038..4b12f3140df1e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -271,6 +272,8 @@ private void doRefresh() { Option queryInstant = specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::getTimestamp)); + validate(activeTimeline, queryInstant); + if (tableType.equals(HoodieTableType.MERGE_ON_READ) && queryType.equals(HoodieTableQueryType.SNAPSHOT)) { cachedAllInputFileSlices = partitionFiles.keySet().stream() .collect(Collectors.toMap( @@ -318,6 +321,12 @@ private static long fileSliceSize(FileSlice fileSlice) { return fileSlice.getBaseFile().map(BaseFile::getFileLen).orElse(0L) + logFileSize; } + private static void validate(HoodieTimeline activeTimeline, Option queryInstant) { + if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) { + throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get())); + } + } + protected static final class PartitionPath { final String path; final Object[] values; From 3ebaec5751fb91eb761150b7dd433f1c76ba2af8 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Tue, 15 Feb 2022 18:39:15 -0800 Subject: [PATCH 28/30] Killing dead-code --- .../hudi/hadoop/utils/HoodieHiveUtils.java | 41 +------------------ 1 file changed, 2 insertions(+), 39 deletions(-) diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java index b4f7e336335d4..fa2bce4875379 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieHiveUtils.java @@ -19,15 +19,11 @@ package org.apache.hudi.hadoop.utils; import org.apache.hadoop.conf.Configuration; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.exception.HoodieIOException; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -148,39 +144,6 @@ public static List getIncrementalTableNames(JobContext job) { return result; } - /** - * Depending on the configs hoodie.%s.consume.pending.commits and hoodie.%s.consume.commit of job - * - * (hoodie..consume.pending.commits, hoodie..consume.commit) -> - * (true, validCommit) -> returns activeTimeline filtered until validCommit - * (true, InValidCommit) -> Raises HoodieIOException - * (true, notSet) -> Raises HoodieIOException - * (false, validCommit) -> returns completedTimeline filtered until validCommit - * (false, InValidCommit) -> Raises HoodieIOException - * (false or notSet, notSet) -> returns completedTimeline unfiltered - * - * validCommit is one which exists in the timeline being checked and vice versa - */ - public static HoodieTimeline getTableTimeline(final String tableName, final JobConf job, final HoodieTableMetaClient metaClient) { - HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline(); - - boolean includePendingCommits = shouldIncludePendingCommits(job, tableName); - Option maxCommit = getMaxCommit(job, tableName); - - HoodieTimeline finalizedTimeline = includePendingCommits ? timeline : timeline.filterCompletedInstants(); - - return !maxCommit.isPresent() ? finalizedTimeline : filterIfInstantExists(tableName, finalizedTimeline, maxCommit.get()); - - } - - private static HoodieTimeline filterIfInstantExists(String tableName, HoodieTimeline timeline, String maxCommit) { - if (maxCommit == null || !timeline.containsInstant(maxCommit)) { - LOG.info("Timestamp " + maxCommit + " doesn't exist in the commits timeline:" + timeline + " table: " + tableName); - throw new HoodieIOException("Valid timestamp is required for " + HOODIE_CONSUME_COMMIT + " in snapshot mode"); - } - return timeline.findInstantsBeforeOrEquals(maxCommit); - } - public static boolean isIncrementalUseDatabase(Configuration conf) { return conf.getBoolean(HOODIE_INCREMENTAL_USE_DATABASE, false); } From 91262ed944ae04d5e4df7315d1c7cc6bd1af6b1d Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Tue, 15 Feb 2022 18:40:34 -0800 Subject: [PATCH 29/30] Fixed tests --- .../hadoop/TestHoodieParquetInputFormat.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java index b7cb72e38a8ef..2ae7c36d98e7e 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java @@ -18,6 +18,15 @@ package org.apache.hudi.hadoop; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -34,16 +43,6 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hadoop.testutils.InputFormatTestUtil; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; - -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.io.ArrayWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RecordReader; -import org.apache.hadoop.mapreduce.Job; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; @@ -203,11 +202,11 @@ public void testSnapshotWithInvalidCommitShouldThrowException() throws IOExcepti FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); InputFormatTestUtil.setupSnapshotIncludePendingCommits(jobConf, "1"); Exception exception = assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf)); - assertEquals("Valid timestamp is required for hoodie.%s.consume.commit in snapshot mode", exception.getMessage()); + assertEquals("Query instant (1) not found in the timeline", exception.getMessage()); InputFormatTestUtil.setupSnapshotMaxCommitTimeQueryMode(jobConf, "1"); exception = assertThrows(HoodieIOException.class, () -> inputFormat.listStatus(jobConf)); - assertEquals("Valid timestamp is required for hoodie.%s.consume.commit in snapshot mode", exception.getMessage()); + assertEquals("Query instant (1) not found in the timeline", exception.getMessage()); } @Test From cb19f8424b20855153867418ec6edb44525230d7 Mon Sep 17 00:00:00 2001 From: Alexey Kudinkin Date: Wed, 16 Feb 2022 12:34:37 -0800 Subject: [PATCH 30/30] Make `BaseHoodieTableFileIndex` only validate instants when required (true for Hive, false for Spark) --- .../apache/hudi/BaseHoodieTableFileIndex.java | 28 +++++++++++-------- .../hudi/hadoop/HiveHoodieTableFileIndex.java | 1 + .../hudi/SparkHoodieTableFileIndex.scala | 1 + 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index 4b12f3140df1e..428da925c49ea 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -75,6 +75,7 @@ public abstract class BaseHoodieTableFileIndex { protected final List queryPaths; private final boolean shouldIncludePendingCommits; + private final boolean shouldValidateInstant; private final HoodieTableType tableType; protected final String basePath; @@ -99,6 +100,7 @@ public abstract class BaseHoodieTableFileIndex { * @param queryPaths target DFS paths being queried * @param specifiedQueryInstant instant as of which table is being queried * @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations + * @param shouldValidateInstant flags to validate whether query instant is present in the timeline * @param fileStatusCache transient cache of fetched [[FileStatus]]es */ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, @@ -108,6 +110,7 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, List queryPaths, Option specifiedQueryInstant, boolean shouldIncludePendingCommits, + boolean shouldValidateInstant, FileStatusCache fileStatusCache) { this.partitionColumns = metaClient.getTableConfig().getPartitionFields() .orElse(new String[0]); @@ -123,6 +126,7 @@ public BaseHoodieTableFileIndex(HoodieEngineContext engineContext, this.queryPaths = queryPaths; this.specifiedQueryInstant = specifiedQueryInstant; this.shouldIncludePendingCommits = shouldIncludePendingCommits; + this.shouldValidateInstant = shouldValidateInstant; this.tableType = metaClient.getTableType(); this.basePath = metaClient.getBasePath(); @@ -287,15 +291,15 @@ private void doRefresh() { ) ); } else { - // TODO re-align with the branch (MOR, snapshot) branch cachedAllInputFileSlices = partitionFiles.keySet().stream() .collect(Collectors.toMap( Function.identity(), partitionPath -> - specifiedQueryInstant.map(instant -> - fileSystemView.getLatestFileSlicesBeforeOrOn(partitionPath.path, instant, true)) - .orElse(fileSystemView.getLatestFileSlices(partitionPath.path)) - .collect(Collectors.toList()) + queryInstant.map(instant -> + fileSystemView.getLatestFileSlicesBeforeOrOn(partitionPath.path, instant, true) + ) + .orElse(fileSystemView.getLatestFileSlices(partitionPath.path)) + .collect(Collectors.toList()) ) ); } @@ -313,6 +317,14 @@ private void doRefresh() { LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration)); } + private void validate(HoodieTimeline activeTimeline, Option queryInstant) { + if (shouldValidateInstant) { + if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) { + throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get())); + } + } + } + private static long fileSliceSize(FileSlice fileSlice) { long logFileSize = fileSlice.getLogFiles().map(HoodieLogFile::getFileSize) .filter(s -> s > 0) @@ -321,12 +333,6 @@ private static long fileSliceSize(FileSlice fileSlice) { return fileSlice.getBaseFile().map(BaseFile::getFileLen).orElse(0L) + logFileSize; } - private static void validate(HoodieTimeline activeTimeline, Option queryInstant) { - if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) { - throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get())); - } - } - protected static final class PartitionPath { final String path; final Object[] values; diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java index 176c1c5e5ca56..000fce5e8fbff 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HiveHoodieTableFileIndex.java @@ -53,6 +53,7 @@ public HiveHoodieTableFileIndex(HoodieEngineContext engineContext, queryPaths, specifiedQueryInstant, shouldIncludePendingCommits, + true, new NoopCache()); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index be3247104c423..46201c4132078 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -65,6 +65,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession, queryPaths.asJava, toJavaOption(specifiedQueryInstant), false, + false, SparkHoodieTableFileIndex.adapt(fileStatusCache) ) with SparkAdapterSupport