Re-factoring Compaction as first level API in WriteClient similar to upsert/insert
This commit is contained in:
committed by
vinoth chandar
parent
5405a6287b
commit
0eaa21111a
@@ -35,6 +35,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.LazyInsertIterable;
|
||||
import com.uber.hoodie.io.HoodieCleanHelper;
|
||||
@@ -64,6 +65,7 @@ import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
@@ -414,6 +416,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String commitTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileLoc,
|
||||
Iterator<HoodieRecord<T>> recordItr)
|
||||
@@ -513,12 +520,6 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc, String commitCompactionTime) {
|
||||
logger.info("Nothing to compact in COW storage format");
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of
|
||||
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
|
||||
|
||||
@@ -41,14 +41,6 @@ import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.ArrayList;
|
||||
@@ -60,6 +52,14 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where
|
||||
@@ -167,7 +167,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc, String compactionCommitTime) {
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionCommitTime) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
@@ -182,13 +182,13 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
|
||||
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
|
||||
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
|
||||
return Optional.empty();
|
||||
return jsc.emptyRDD();
|
||||
}
|
||||
|
||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return Optional.of(compactor.compact(jsc, config, this, compactionCommitTime));
|
||||
return compactor.compact(jsc, config, this, compactionCommitTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
|
||||
@@ -20,7 +20,6 @@ import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
@@ -43,9 +42,8 @@ import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
@@ -57,8 +55,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
@@ -240,7 +236,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
@@ -254,11 +249,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
}
|
||||
|
||||
/**
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
* Run Compaction on the table.
|
||||
* Compaction arranges the data so that it is optimized for data access
|
||||
*/
|
||||
public abstract Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc,
|
||||
String commitCompactionTime);
|
||||
public abstract JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String commitTime);
|
||||
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
@@ -279,5 +273,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* @param writeStatuses List of WriteStatus
|
||||
* @return number of files finalized
|
||||
*/
|
||||
public abstract Optional<Integer> finalizeWrite(JavaSparkContext jsc, List<Tuple2<String, HoodieWriteStat>> writeStatuses);
|
||||
public abstract Optional<Integer> finalizeWrite(JavaSparkContext jsc,
|
||||
List<Tuple2<String, HoodieWriteStat>> writeStatuses);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user