1
0

Introduce getCommitsAndCompactionsTimeline() explicitly & adjust usage across code base

This commit is contained in:
Vinoth Chandar
2017-04-26 13:36:49 -07:00
committed by prazanna
parent bae0528013
commit da17c5c607
15 changed files with 42 additions and 48 deletions

View File

@@ -99,8 +99,8 @@ public class HoodieInputFormat extends MapredParquetInputFormat
}
String tableName = metadata.getTableConfig().getTableName();
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
// FIXME(VC): This is incorrect and needs to change to include commits, delta commits, compactions, as all of them produce a base parquet file today
HoodieTimeline timeline = metadata.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants();
// Get all commits, delta commits, compactions, as all of them produce a base parquet file today
HoodieTimeline timeline = metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
TableFileSystemView fsView = new HoodieTableFileSystemView(metadata, timeline);
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {

View File

@@ -21,12 +21,10 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.InvalidDatasetException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.fs.PathFilter;
@@ -36,7 +34,6 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Given a path is a part of