1
0

Async Compaction Main API changes

This commit is contained in:
Balaji Varadarajan
2018-05-23 23:09:25 -07:00
committed by vinoth chandar
parent 9b78523d62
commit 2f8ce93030
18 changed files with 878 additions and 267 deletions

View File

@@ -1,42 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.io.compact;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
/**
* Contains Hoodie Compaction instant along with workload
*/
public class HoodieCompactionInstantWithPlan {
private final String compactionInstantTime;
private final HoodieCompactionPlan compactionPlan;
public HoodieCompactionInstantWithPlan(String compactionInstantTime,
HoodieCompactionPlan compactionPlan) {
this.compactionInstantTime = compactionInstantTime;
this.compactionPlan = compactionPlan;
}
public String getCompactionInstantTime() {
return compactionInstantTime;
}
public HoodieCompactionPlan getCompactionPlan() {
return compactionPlan;
}
}

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.io.Serializable;
import java.util.Set;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -30,15 +31,6 @@ import org.apache.spark.api.java.JavaSparkContext;
*/
public interface HoodieCompactor extends Serializable {
/**
* Compact the delta files with the data files
*
* @deprecated : Will be removed in next PR
*/
@Deprecated
JavaRDD<WriteStatus> compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
/**
* Generate a new compaction plan for scheduling
*
@@ -50,7 +42,8 @@ public interface HoodieCompactor extends Serializable {
* @throws IOException when encountering errors
*/
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime)
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
Set<String> fileIdsWithPendingCompactions)
throws IOException;
/**
@@ -58,5 +51,5 @@ public interface HoodieCompactor extends Serializable {
*/
JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionCommitTime) throws IOException;
String compactionInstantTime) throws IOException;
}

View File

@@ -45,6 +45,7 @@ import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
@@ -73,31 +74,22 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Accumulator to keep track of total log file slices for a dataset
private AccumulatorV2<Long, Long> totalFileSlices;
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws IOException {
HoodieCompactionPlan compactionPlan = generateCompactionPlan(jsc, hoodieTable, config,
compactionCommitTime);
List<HoodieCompactionOperation> operations = compactionPlan.getOperations();
if ((operations == null) || (operations.isEmpty())) {
return jsc.emptyRDD();
}
return compact(jsc, compactionPlan, hoodieTable, config, compactionCommitTime);
}
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionCommitTime) throws IOException {
String compactionInstantTime) throws IOException {
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|| (compactionPlan.getOperations().isEmpty())) {
return jsc.emptyRDD();
}
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
List<CompactionOperation> operations = compactionPlan.getOperations().stream().map(
CompactionOperation::convertFromAvroRecordInstance).collect(toList());
log.info("Compactor compacting " + operations + " files");
return jsc.parallelize(operations, operations.size())
.map(s -> compact(table, metaClient, config, s, compactionCommitTime))
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
}
@@ -164,8 +156,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
@Override
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime)
throws IOException {
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
Set<String> fileIdsWithPendingCompactions) throws IOException {
totalLogFiles = new LongAccumulator();
totalFileSlices = new LongAccumulator();
@@ -191,7 +183,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
List<HoodieCompactionOperation> operations =
jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
.getLatestFileSlices(partitionPath).map(
.getLatestFileSlices(partitionPath)
.filter(slice -> !fileIdsWithPendingCompactions.contains(slice.getFileId()))
.map(
s -> {
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
.getBaseInstantAndLogVersionComparator().reversed()).collect(Collectors.toList());
@@ -211,13 +205,16 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
log.info("Total number of latest files slices " + totalFileSlices.value());
log.info("Total number of log files " + totalLogFiles.value());
log.info("Total number of file slices " + totalFileSlices.value());
// Filter the compactions with the passed in filter. This lets us choose most effective
// compactions only
// 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,
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
Preconditions.checkArgument(compactionPlan.getOperations().stream()
.filter(op -> fileIdsWithPendingCompactions.contains(op.getFileId())).count() == 0,
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
+ "Please fix your strategy implementation."
+ "FileIdsWithPendingCompactions :" + fileIdsWithPendingCompactions
+ ", Selected workload :" + compactionPlan);
if (compactionPlan.getOperations().isEmpty()) {
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
}