1
0

Track fileIds with pending compaction in FileSystemView to provide correct API semantics

This commit is contained in:
Balaji Varadarajan
2018-05-24 11:19:40 -07:00
committed by vinoth chandar
parent 1b61f04e05
commit 9d99942564
9 changed files with 448 additions and 40 deletions

View File

@@ -108,7 +108,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
+ ".partitions";
// 500GB of target IO per compaction (both read and write)
public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
public static final String DEFAULT_COMPACTOR_ID = "default";
private HoodieCompactionConfig(Properties props) {
super(props);

View File

@@ -41,7 +41,6 @@ import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
@@ -49,6 +48,7 @@ import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -95,7 +95,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
log.info("Compactor " + compactionPlan.getCompactorId() + " running, Compacting " + operations + " files");
log.info("Compactor compacting " + operations + " files");
return jsc.parallelize(operations, operations.size())
.map(s -> compact(table, metaClient, config, s, compactionCommitTime))
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
@@ -217,7 +217,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// TODO: In subsequent PRs, pending Compaction plans will be wired in. Strategy can look at pending compaction
// plans to schedule next compaction plan
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
new ArrayList<>());
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
if (compactionPlan.getOperations().isEmpty()) {
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
}

View File

@@ -22,7 +22,6 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.Serializable;
import java.util.List;
@@ -92,7 +91,7 @@ public abstract class CompactionStrategy implements Serializable {
public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig,
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
// Strategy implementation can overload this method to set specific compactor-id
return HoodieCompactionPlan.newBuilder().setCompactorId(HoodieCompactionConfig.DEFAULT_COMPACTOR_ID)
return HoodieCompactionPlan.newBuilder()
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
.build();
}