Introduce RealtimeTableView and Implement HoodieRealtimeTableCompactor (#73)
This commit is contained in:
committed by
Prasanna Rajaperumal
parent
48fbb0f425
commit
11d2fd3428
@@ -64,7 +64,6 @@ import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
@@ -94,8 +93,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
private transient final HoodieCommitArchiveLog archiveLog;
|
||||
private transient Timer.Context writeContext = null;
|
||||
|
||||
private final SimpleDateFormat FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
/**
|
||||
* @param jsc
|
||||
* @param clientConfig
|
||||
@@ -331,7 +328,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
clean();
|
||||
if (writeContext != null) {
|
||||
long durationInMs = metrics.getDurationInMs(writeContext.stop());
|
||||
metrics.updateCommitMetrics(FORMATTER.parse(commitTime).getTime(), durationInMs,
|
||||
metrics.updateCommitMetrics(
|
||||
HoodieActiveTimeline.COMMIT_FORMATTER.parse(commitTime).getTime(), durationInMs,
|
||||
metadata);
|
||||
writeContext = null;
|
||||
}
|
||||
@@ -495,7 +493,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Provides a new commit time for a write operation (insert/update)
|
||||
*/
|
||||
public String startCommit() {
|
||||
String commitTime = FORMATTER.format(new Date());
|
||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
startCommitWithTime(commitTime);
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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 java.util.List;
|
||||
|
||||
/**
|
||||
* 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);
|
||||
|
||||
// Default implementation - do not filter anything
|
||||
static CompactionFilter allowAll() {
|
||||
return s -> s;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
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
|
||||
*/
|
||||
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() {
|
||||
}
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
public String getDataFileCommitTime() {
|
||||
return dataFileCommitTime;
|
||||
}
|
||||
|
||||
public long getDataFileSize() {
|
||||
return dataFileSize;
|
||||
}
|
||||
|
||||
public List<String> getDeltaFilePaths() {
|
||||
return deltaFilePaths;
|
||||
}
|
||||
|
||||
public String getDataFilePath() {
|
||||
return dataFilePath;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,26 @@
|
||||
/*
|
||||
* 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.common.model.HoodieCommitMetadata;
|
||||
|
||||
/**
|
||||
* Place holder for the compaction specific meta-data, uses all the details used in a normal HoodieCommitMetadata
|
||||
*/
|
||||
public class HoodieCompactionMetadata extends HoodieCommitMetadata {
|
||||
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* 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.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.RealtimeTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Date;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
*/
|
||||
public interface HoodieCompactor extends Serializable {
|
||||
/**
|
||||
* Compact the delta files with the data files
|
||||
* @throws Exception
|
||||
*/
|
||||
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
|
||||
CompactionFilter compactionFilter) throws Exception;
|
||||
|
||||
|
||||
// Helper methods
|
||||
default String startCompactionCommit(HoodieTableMetaClient metaClient) {
|
||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
activeTimeline
|
||||
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
||||
return commitTime;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,169 @@
|
||||
/*
|
||||
* 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.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.RealtimeTableView;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.commons.collections.IteratorUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage.
|
||||
* Computes all possible compactions, passes it through a CompactionFilter and executes
|
||||
* all the compactions and writes a new version of base files and make a normal commit
|
||||
*
|
||||
* @see HoodieCompactor
|
||||
*/
|
||||
public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
|
||||
|
||||
@Override
|
||||
public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
|
||||
CompactionFilter compactionFilter) throws Exception {
|
||||
// TODO - rollback any compactions in flight
|
||||
|
||||
String compactionCommit = startCompactionCommit(metaClient);
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath());
|
||||
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<CompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> {
|
||||
FileSystem fileSystem = FSUtils.getFs();
|
||||
return fsView.groupLatestDataFileWithLogFiles(fileSystem, partitionPath)
|
||||
.entrySet().stream()
|
||||
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}).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);
|
||||
if(operations.isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
return null;
|
||||
}
|
||||
|
||||
log.info("After filtering, Compacting " + operations + " files");
|
||||
List<Tuple2<String, HoodieWriteStat>> updateStatusMap =
|
||||
jsc.parallelize(operations, operations.size()).map(
|
||||
(Function<CompactionOperation, Iterator<List<WriteStatus>>>) compactionOperation -> executeCompaction(
|
||||
metaClient, config, compactionOperation, compactionCommit)).flatMap(
|
||||
(FlatMapFunction<Iterator<List<WriteStatus>>, WriteStatus>) listIterator -> {
|
||||
List<List<WriteStatus>> collected = IteratorUtils.toList(listIterator);
|
||||
return collected.stream().flatMap(List::stream).collect(Collectors.toList());
|
||||
}).mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
|
||||
@Override
|
||||
public Tuple2<String, HoodieWriteStat> call(WriteStatus writeStatus)
|
||||
throws Exception {
|
||||
return new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat());
|
||||
}
|
||||
}).collect();
|
||||
|
||||
HoodieCompactionMetadata metadata = new HoodieCompactionMetadata();
|
||||
for (Tuple2<String, HoodieWriteStat> stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat._1(), stat._2());
|
||||
}
|
||||
log.info("Compaction finished with result " + metadata);
|
||||
|
||||
//noinspection ConstantConditions
|
||||
if (isCompactionSucceeded(metadata)) {
|
||||
log.info("Compaction succeeded " + compactionCommit);
|
||||
commitCompaction(compactionCommit, metaClient, metadata);
|
||||
} else {
|
||||
log.info("Compaction failed " + compactionCommit);
|
||||
}
|
||||
return metadata;
|
||||
}
|
||||
|
||||
private boolean isCompactionSucceeded(HoodieCompactionMetadata result) {
|
||||
//TODO figure out a success factor for a compaction
|
||||
return true;
|
||||
}
|
||||
|
||||
private Iterator<List<WriteStatus>> executeCompaction(HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
|
||||
throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
Schema schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
|
||||
.getDeltaFilePaths() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// 1. Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
|
||||
// 2. naively loads all the delta records in memory to merge it,
|
||||
// since we only need a iterator, we could implement a lazy iterator to load from one delta file at a time
|
||||
List<HoodieRecord<HoodieAvroPayload>> readDeltaFilesInMemory =
|
||||
AvroUtils.loadFromFiles(fs, operation.getDeltaFilePaths(), schema);
|
||||
|
||||
HoodieCopyOnWriteTable<HoodieAvroPayload> table =
|
||||
new HoodieCopyOnWriteTable<>(commitTime, config, metaClient);
|
||||
return table.handleUpdate(operation.getFileId(), readDeltaFilesInMemory.iterator());
|
||||
}
|
||||
|
||||
public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient,
|
||||
HoodieCompactionMetadata metadata) {
|
||||
log.info("Comitting " + commitTime);
|
||||
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
|
||||
|
||||
try {
|
||||
activeTimeline.saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException(
|
||||
"Failed to commit " + metaClient.getBasePath() + " at time " + commitTime, e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -30,7 +30,6 @@ import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.LazyInsertIterable;
|
||||
import com.uber.hoodie.io.HoodieUpdateHandle;
|
||||
@@ -390,7 +389,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -399,7 +398,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr)
|
||||
throws IOException {
|
||||
// these are updates
|
||||
HoodieUpdateHandle upsertHandle =
|
||||
new HoodieUpdateHandle<>(config, commitTime, metaClient, recordItr, fileLoc);
|
||||
@@ -462,11 +463,4 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(Integer partition,
|
||||
Iterator recordItr,
|
||||
Partitioner partitioner) {
|
||||
return handleUpsertPartition(partition, recordItr, partitioner);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user