1
0

[HUDI-3206] Unify Hive's MOR implementations to avoid duplication (#4559)

Unify Hive's MOR implementations to avoid duplication to avoid duplication across implementations for different file-formats (Parquet, HFile, etc)

- Extracted HoodieRealtimeFileInputFormatBase (extending COW HoodieFileInputFormatBase base)
- Rebased Parquet, HFile implementations onto HoodieRealtimeFileInputFormatBase
- Tidying up
This commit is contained in:
Alexey Kudinkin
2022-02-07 11:06:28 -08:00
committed by GitHub
parent 773b317983
commit 3f263b82ce
10 changed files with 460 additions and 415 deletions

View File

@@ -44,6 +44,7 @@ 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;
import org.junit.jupiter.api.Test;
@@ -115,7 +116,7 @@ public class TestHoodieParquetInputFormat {
timeline.setInstants(instants);
// Verify getCommitsTimelineBeforePendingCompaction does not return instants after first compaction instant
HoodieTimeline filteredTimeline = inputFormat.filterInstantsTimeline(timeline);
HoodieTimeline filteredTimeline = HoodieInputFormatUtils.filterInstantsTimeline(timeline);
assertTrue(filteredTimeline.containsInstant(t1));
assertTrue(filteredTimeline.containsInstant(t2));
assertFalse(filteredTimeline.containsInstant(t3));
@@ -126,7 +127,7 @@ public class TestHoodieParquetInputFormat {
instants.remove(t3);
timeline = new HoodieActiveTimeline(metaClient);
timeline.setInstants(instants);
filteredTimeline = inputFormat.filterInstantsTimeline(timeline);
filteredTimeline = HoodieInputFormatUtils.filterInstantsTimeline(timeline);
// verify all remaining instants are returned.
assertTrue(filteredTimeline.containsInstant(t1));
@@ -140,7 +141,7 @@ public class TestHoodieParquetInputFormat {
instants.remove(t5);
timeline = new HoodieActiveTimeline(metaClient);
timeline.setInstants(instants);
filteredTimeline = inputFormat.filterInstantsTimeline(timeline);
filteredTimeline = HoodieInputFormatUtils.filterInstantsTimeline(timeline);
// verify all remaining instants are returned.
assertTrue(filteredTimeline.containsInstant(t1));