1
0

Implement Compaction policy abstraction. Implement LogSizeBased Bounded IO Compaction as the default strategy

This commit is contained in:
Prasanna Rajaperumal
2017-04-04 12:37:28 -07:00
committed by prazanna
parent 82b211d2e6
commit 91b088f29f
18 changed files with 585 additions and 68 deletions

View File

@@ -19,6 +19,8 @@ package com.uber.hoodie.config;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
@@ -83,6 +85,13 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
// 500GB of target IO per compaction (both read and write)
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
// 200GB of target IO per compaction
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
private HoodieCompactionConfig(Properties props) {
super(props);
@@ -167,6 +176,16 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
return this;
}
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
return this;
}
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
return this;
}
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
@@ -195,6 +214,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
Preconditions.checkArgument(

View File

@@ -19,9 +19,9 @@ package com.uber.hoodie.config;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.config.HoodieCompactionConfig.Builder;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import com.uber.hoodie.metrics.MetricsReporterType;
import org.apache.spark.storage.StorageLevel;
@@ -164,6 +164,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
}
public CompactionStrategy getCompactionStrategy() {
return ReflectionUtils.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
}
public Long getTargetIOPerCompactionInMB() {
return Long.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
}
/**
* index properties
**/
@@ -238,7 +246,9 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return new Builder();
}
public static class Builder {
public static class Builder {
private final Properties props = new Properties();
private boolean isIndexConfigSet = false;
private boolean isStorageConfigSet = false;

View File

@@ -19,61 +19,72 @@ package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Encapsulates all the needed information about a compaction
* and make a decision whether this compaction is effective or not
*
* @see CompactionFilter
* @see CompactionStrategy
*/
public class CompactionOperation implements Serializable {
private String dataFileCommitTime;
private long dataFileSize;
private List<String> deltaFilePaths;
private String dataFilePath;
private String fileId;
private String partitionPath;
//Only for serialization/de-serialization
@Deprecated
public CompactionOperation() {
}
private String dataFileCommitTime;
private long dataFileSize;
private List<String> deltaFilePaths;
private String dataFilePath;
private String fileId;
private String partitionPath;
private Map<String, Object> metrics;
public CompactionOperation(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> value) {
this.dataFilePath = dataFile.getPath();
this.fileId = dataFile.getFileId();
this.partitionPath = partitionPath;
this.dataFileCommitTime = dataFile.getCommitTime();
this.dataFileSize = dataFile.getFileStatus().getLen();
this.deltaFilePaths = value.stream().map(s -> s.getPath().toString()).collect(
Collectors.toList());
}
//Only for serialization/de-serialization
@Deprecated
public CompactionOperation() {
}
public String getDataFileCommitTime() {
return dataFileCommitTime;
}
public CompactionOperation(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles, HoodieWriteConfig writeConfig) {
this.dataFilePath = dataFile.getPath();
this.fileId = dataFile.getFileId();
this.partitionPath = partitionPath;
this.dataFileCommitTime = dataFile.getCommitTime();
this.dataFileSize = dataFile.getFileSize();
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString()).collect(
Collectors.toList());
this.metrics = writeConfig.getCompactionStrategy()
.captureMetrics(dataFile, partitionPath, logFiles);
}
public long getDataFileSize() {
return dataFileSize;
}
public String getDataFileCommitTime() {
return dataFileCommitTime;
}
public List<String> getDeltaFilePaths() {
return deltaFilePaths;
}
public long getDataFileSize() {
return dataFileSize;
}
public String getDataFilePath() {
return dataFilePath;
}
public List<String> getDeltaFilePaths() {
return deltaFilePaths;
}
public String getFileId() {
return fileId;
}
public String getDataFilePath() {
return dataFilePath;
}
public String getPartitionPath() {
return partitionPath;
}
public String getFileId() {
return fileId;
}
public String getPartitionPath() {
return partitionPath;
}
public Map<String, Object> getMetrics() {
return metrics;
}
}

View File

@@ -36,7 +36,7 @@ public interface HoodieCompactor extends Serializable {
* @throws Exception
*/
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable, CompactionFilter compactionFilter) throws Exception;
HoodieTable hoodieTable) throws Exception;
// Helper methods

View File

@@ -64,7 +64,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
@Override
public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
HoodieTable hoodieTable, CompactionFilter compactionFilter) throws IOException {
HoodieTable hoodieTable) throws IOException {
Preconditions.checkArgument(
hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"HoodieRealtimeTableCompactor can only compact table of type "
@@ -86,12 +86,12 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getFileSystemView()
.groupLatestDataFileWithLogFiles(partitionPath).entrySet()
.stream()
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue()))
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue(), config))
.collect(toList()).iterator()).collect();
log.info("Total of " + operations.size() + " compactions are retrieved");
// Filter the compactions with the passed in filter. This lets us choose most effective compactions only
operations = compactionFilter.filter(operations);
operations = config.getCompactionStrategy().orderAndFilter(config, operations);
if (operations.isEmpty()) {
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
return null;

View File

@@ -0,0 +1,80 @@
/*
* 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.strategy;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.util.List;
import java.util.Map;
import java.util.Optional;
/**
* CompactionStrategy which looks at total IO to be done for the compaction (read + write)
* and limits the list of compactions to be under a configured limit on the IO
*
* @see CompactionStrategy
*/
public class BoundedIOCompactionStrategy implements CompactionStrategy {
public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB";
public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB";
public static final String TOTAL_IO_MB = "TOTAL_IO_MB";
@Override
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = Maps.newHashMap();
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
Optional::isPresent).map(Optional::get).reduce(
(size1, size2) -> size1 + size2).orElse(0L);
// Total read will be the base file + all the log files
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
// Total write will be similar to the size of the base file
Long totalIOWrite = FSUtils.getSizeInMB(dataFile.getFileSize());
// Total IO will the the IO for read + write
Long totalIO = totalIORead + totalIOWrite;
// Save these metrics and we will use during the filter
metrics.put(TOTAL_IO_READ_MB, totalIORead);
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite);
metrics.put(TOTAL_IO_MB, totalIO);
return metrics;
}
@Override
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
// Iterate through the operations in order and accept operations as long as we are within the IO limit
// Preserves the original ordering of compactions
List<CompactionOperation> finalOperations = Lists.newArrayList();
long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB();
for (CompactionOperation op : operations) {
long opIo = (Long) op.getMetrics().get(TOTAL_IO_MB);
targetIORemaining -= opIo;
finalOperations.add(op);
if (targetIORemaining <= 0) {
return finalOperations;
}
}
return finalOperations;
}
}

View File

@@ -0,0 +1,62 @@
/*
* 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.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
/**
* Strategy for compaction. Pluggable implementation of define how compaction should be done.
* The implementations of this interface can capture the relevant metrics to order and filter
* the final list of compaction operation to run in a single compaction.
*
* Implementation of CompactionStrategy cannot hold any state.
* Difference instantiations can be passed in every time
*
* @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor
* @see CompactionOperation
*/
public interface CompactionStrategy extends Serializable {
/**
* Callback hook when a CompactionOperation is created. Individual strategies can
* capture the metrics they need to decide on the priority.
*
* @param dataFile - Base file to compact
* @param partitionPath - Partition path
* @param logFiles - List of log files to compact with the base file
* @return Map[String, Object] - metrics captured
*/
Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles);
/**
* Order and Filter the list of compactions. Use the metrics captured with the
* captureMetrics to order and filter out compactions
*
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
* @param operations - list of compactions collected
* @return list of compactions to perform in this run
*/
List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<CompactionOperation> operations);
}

View File

@@ -0,0 +1,70 @@
/*
* 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.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size
* and limits the compactions within a configured IO bound
*
* @see BoundedIOCompactionStrategy
* @see CompactionStrategy
*/
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements
Comparator<CompactionOperation> {
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@Override
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles) {
Map<String, Object> metrics = super.captureMetrics(dataFile, partitionPath, logFiles);
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
Optional::isPresent).map(Optional::get).reduce(
(size1, size2) -> size1 + size2).orElse(0L);
// save the metrics needed during the order
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize);
return metrics;
}
@Override
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<CompactionOperation> operations) {
// Order the operations based on the reverse size of the logs and limit them by the IO
return super
.orderAndFilter(writeConfig, operations.stream().sorted(this).collect(Collectors.toList()));
}
@Override
public int compare(CompactionOperation op1, CompactionOperation op2) {
Long totalLogSize1 = (Long) op1.getMetrics().get(TOTAL_LOG_FILE_SIZE);
Long totalLogSize2 = (Long) op2.getMetrics().get(TOTAL_LOG_FILE_SIZE);
// Reverse the comparison order - so that larger log file size is compacted first
return totalLogSize2.compareTo(totalLogSize1);
}
}

View File

@@ -0,0 +1,47 @@
/*
* 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.strategy;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import java.util.List;
import java.util.Map;
/**
* UnBoundedCompactionStrategy will not change ordering or filter any compaction.
* It is a pass-through and will compact all the base files which has a log file.
* This usually means no-intelligence on compaction.
*
* @see CompactionStrategy
*/
public class UnBoundedCompactionStrategy implements CompactionStrategy {
@Override
public Map<String, Object> captureMetrics(HoodieDataFile dataFile, String partitionPath,
List<HoodieLogFile> logFiles) {
return Maps.newHashMap();
}
@Override
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig config,
List<CompactionOperation> operations) {
return operations;
}
}

View File

@@ -25,7 +25,6 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.CompactionFilter;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import java.util.Optional;
import org.apache.log4j.LogManager;
@@ -89,7 +88,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
logger.info("Compacting merge on read table " + config.getBasePath());
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
try {
return Optional.of(compactor.compact(jsc, config, this, CompactionFilter.allowAll()));
return Optional.of(compactor.compact(jsc, config, this));
} catch (IOException e) {
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
}

View File

@@ -32,7 +32,6 @@ import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.CompactionFilter;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
@@ -165,7 +164,7 @@ public class TestMergeOnReadTable {
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
compactor.compact(jsc, getConfig(), table);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
allFiles = HoodieTestUtils.listAllDataFilesInPath(fs, cfg.getBasePath());

View File

@@ -36,7 +36,6 @@ import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieBloomIndex;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.CompactionFilter;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import com.uber.hoodie.table.HoodieTable;
@@ -113,7 +112,7 @@ public class TestHoodieCompactor {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
compactor.compact(jsc, getConfig(), table);
}
@Test
@@ -129,7 +128,7 @@ public class TestHoodieCompactor {
writeClient.insert(recordsRDD, newCommitTime).collect();
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
compactor.compact(jsc, getConfig(), table);
assertTrue("If there is nothing to compact, result will be empty",
result.getFileIdAndFullPaths().isEmpty());
}
@@ -177,7 +176,7 @@ public class TestHoodieCompactor {
table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCompactionMetadata result =
compactor.compact(jsc, getConfig(), table, CompactionFilter.allowAll());
compactor.compact(jsc, getConfig(), table);
// Verify that recently written compacted data file has no log file
metaClient = new HoodieTableMetaClient(fs, basePath);

View File

@@ -0,0 +1,123 @@
/*
* 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.strategy;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import com.beust.jcommander.internal.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.compact.CompactionOperation;
import com.uber.hoodie.io.compact.strategy.BoundedIOCompactionStrategy;
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.junit.Test;
public class TestHoodieCompactionStrategy {
private static final long MB = 1024 * 1024L;
@Test
public void testUnBounded() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp")
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build();
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
assertEquals("UnBounded should not re-order or filter", operations, returned);
}
@Test
public void testBoundedIOSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
BoundedIOCompactionStrategy strategy = new BoundedIOCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp")
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
.withTargetIOPerCompactionInMB(400).build()).build();
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
assertTrue("BoundedIOCompaction should have resulted in fewer compactions",
returned.size() < operations.size());
assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen",
2, returned.size());
// Total size of all the log files
Long returnedSize = returned.stream()
.map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s)
.reduce((size1, size2) -> size1 + size2).orElse(0L);
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB",
610, (long) returnedSize);
}
@Test
public void testLogFileSizeCompactionSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp")
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
.withTargetIOPerCompactionInMB(400).build()).build();
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions",
returned.size() < operations.size());
assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction",
1, returned.size());
// Total size of all the log files
Long returnedSize = returned.stream()
.map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB)).map(s -> (Long) s)
.reduce((size1, size2) -> size1 + size2).orElse(0L);
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB",
1204, (long) returnedSize);
}
private List<CompactionOperation> createCompactionOperations(HoodieWriteConfig config,
Map<Long, List<Long>> sizesMap) {
List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size());
sizesMap.forEach((k, v) -> {
operations.add(new CompactionOperation(TestHoodieDataFile.newDataFile(k), "",
v.stream().map(TestHoodieLogFile::newLogFile).collect(
Collectors.toList()), config));
});
return operations;
}
}

View File

@@ -0,0 +1,57 @@
/*
* 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.strategy;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.util.FSUtils;
import java.util.UUID;
import org.apache.hadoop.fs.FileStatus;
public class TestHoodieDataFile extends HoodieDataFile {
private final long size;
public TestHoodieDataFile(long size) {
super(null);
this.size = size;
}
@Override
public String getPath() {
return "/tmp/test";
}
@Override
public String getFileId() {
return UUID.randomUUID().toString();
}
@Override
public String getCommitTime() {
return "100";
}
@Override
public long getFileSize() {
return size;
}
public static HoodieDataFile newDataFile(long size) {
return new TestHoodieDataFile(size);
}
}

View File

@@ -14,21 +14,33 @@
* limitations under the License.
*/
package com.uber.hoodie.io.compact;
package com.uber.hoodie.io.strategy;
import java.util.List;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import java.util.Optional;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
/**
* Implementations of CompactionFilter allows prioritizing and filtering certain type of
* compactions over other compactions.
*
* e.g. Filter in-efficient compaction like compacting a very large old parquet file with a small avro file
*/
public interface CompactionFilter {
List<CompactionOperation> filter(List<CompactionOperation> input);
public class TestHoodieLogFile extends HoodieLogFile {
// Default implementation - do not filter anything
static CompactionFilter allowAll() {
return s -> s;
}
private final long size;
public TestHoodieLogFile(long size) {
super((Path) null);
this.size = size;
}
@Override
public Path getPath() {
return new Path("/tmp/test-log");
}
@Override
public Optional<Long> getFileSize() {
return Optional.of(size);
}
public static HoodieLogFile newLogFile(long size) {
return new TestHoodieLogFile(size);
}
}

View File

@@ -75,6 +75,9 @@ public class HoodieLogFile {
return fileStatus;
}
public Optional<Long> getFileSize() {
return fileStatus.map(FileStatus::getLen);
}
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
String fileId = getFileId();

View File

@@ -351,4 +351,8 @@ public class FSUtils {
fs.mkdirs(partitionPath);
}
}
public static Long getSizeInMB(long sizeInBytes) {
return sizeInBytes / (1024 * 1024);
}
}

View File

@@ -18,6 +18,7 @@ package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.exception.HoodieException;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
@@ -40,4 +41,21 @@ public class ReflectionUtils {
throw new IOException("Could not load payload class " + recordPayloadClass, e);
}
}
public static <T> T loadClass(String fqcn) {
try {
if(clazzCache.get(fqcn) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(fqcn);
clazzCache.put(fqcn, clazz);
}
return (T) clazzCache.get(fqcn).newInstance();
} catch (ClassNotFoundException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (InstantiationException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (IllegalAccessException e) {
throw new HoodieException("Could not load class " + fqcn, e);
}
}
}