1
0

[HUDI-687] Stop incremental reader on RO table when there is a pending compaction (#1396)

This commit is contained in:
satishkotha
2020-04-10 10:45:41 -07:00
committed by GitHub
parent 8c7cef3e50
commit c0f96e0726
11 changed files with 540 additions and 125 deletions

View File

@@ -38,6 +38,23 @@ public class HoodieHiveUtil {
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
/*
* Boolean property to stop incremental reader when there is a pending compaction.
* This is needed to prevent certain race conditions with RO views of MOR tables. only applicable for RO views.
*
* example timeline:
*
* t0 -> create bucket1.parquet
* t1 -> create and append updates bucket1.log
* t2 -> request compaction
* t3 -> create bucket2.parquet
*
* if compaction at t2 takes a long time, incremental readers on RO tables can move to t3 and would skip updates in t1
*
* To workaround this problem, we want to stop returning data belonging to commits > t2.
* After compaction is complete, incremental reader would see updates in t2, t3, so on.
*/
public static final String HOODIE_STOP_AT_COMPACTION_PATTERN = "hoodie.%s.ro.stop.at.compaction";
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
public static final String SNAPSHOT_SCAN_MODE = "SNAPSHOT";
public static final String DEFAULT_SCAN_MODE = SNAPSHOT_SCAN_MODE;
@@ -46,6 +63,13 @@ public class HoodieHiveUtil {
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode");
public static boolean stopAtCompaction(JobContext job, String tableName) {
String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName);
boolean stopAtCompaction = job.getConfiguration().getBoolean(compactionPropName, true);
LOG.info("Read stop at compaction - " + stopAtCompaction);
return stopAtCompaction;
}
public static Integer readMaxCommits(JobContext job, String tableName) {
String maxCommitName = String.format(HOODIE_MAX_COMMIT_PATTERN, tableName);
int maxCommits = job.getConfiguration().getInt(maxCommitName, DEFAULT_MAX_COMMITS);

View File

@@ -18,17 +18,6 @@
package org.apache.hudi.hadoop;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -42,6 +31,18 @@ import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
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.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -118,6 +119,36 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
return returns.toArray(new FileStatus[returns.size()]);
}
/**
* Filter any specific instants that we do not want to process.
* example timeline:
*
* t0 -> create bucket1.parquet
* t1 -> create and append updates bucket1.log
* t2 -> request compaction
* t3 -> create bucket2.parquet
*
* if compaction at t2 takes a long time, incremental readers on RO tables can move to t3 and would skip updates in t1
*
* To workaround this problem, we want to stop returning data belonging to commits > t2.
* After compaction is complete, incremental reader would see updates in t2, t3, so on.
*/
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getCommitsAndCompactionTimeline();
Option<HoodieInstant> pendingCompactionInstant = commitsAndCompactionTimeline.filterPendingCompactionTimeline().firstInstant();
if (pendingCompactionInstant.isPresent()) {
HoodieDefaultTimeline instantsTimeline = commitsAndCompactionTimeline.findInstantsBefore(pendingCompactionInstant.get().getTimestamp());
int numCommitsFilteredByCompaction = commitsAndCompactionTimeline.getCommitsTimeline().countInstants()
- instantsTimeline.getCommitsTimeline().countInstants();
LOG.info("Earliest pending compaction instant is: " + pendingCompactionInstant.get().getTimestamp()
+ " skipping " + numCommitsFilteredByCompaction + " commits");
return instantsTimeline;
} else {
return timeline;
}
}
/**
* Achieves listStatus functionality for an incrementally queried table. Instead of listing all
* partitions and then filtering based on the commits of interest, this logic first extracts the
@@ -126,10 +157,18 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
private List<FileStatus> listStatusForIncrementalMode(
JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
String tableName = tableMetaClient.getTableConfig().getTableName();
HoodieTimeline timeline = tableMetaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
Job jobContext = Job.getInstance(job);
HoodieDefaultTimeline baseTimeline;
if (HoodieHiveUtil.stopAtCompaction(jobContext, tableName)) {
baseTimeline = filterInstantsTimeline(tableMetaClient.getActiveTimeline());
} else {
baseTimeline = tableMetaClient.getActiveTimeline();
}
HoodieTimeline timeline = baseTimeline.getCommitsTimeline().filterCompletedInstants();
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(jobContext, tableName);
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
Integer maxCommits = HoodieHiveUtil.readMaxCommits(jobContext, tableName);
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
List<HoodieInstant> commitsToCheck = timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
.getInstants().collect(Collectors.toList());

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
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.HoodieTableFileSystemView;
@@ -159,6 +160,12 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
return super.listStatus(job);
}
@Override
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
// no specific filtering for Realtime format
return timeline;
}
/**
* Add a field to the existing fields projected.
*/