Perform consistency checks during write finalize
- Check to ensure written files are listable on storage - Docs reflected to capture how this helps with s3 storage - Unit tests added, corrections to existing tests - Fix DeltaStreamer to manage archived commits in a separate folder
This commit is contained in:
committed by
vinoth chandar
parent
4c74dd4cad
commit
9ca6f91e97
@@ -29,7 +29,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.model.HoodieRollingStatMetadata;
|
||||
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;
|
||||
@@ -382,44 +381,40 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
||||
if (!config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
return Optional.empty();
|
||||
public void finalizeWrite(JavaSparkContext jsc, List<WriteStatus> writeStatuses)
|
||||
throws HoodieIOException {
|
||||
|
||||
super.finalizeWrite(jsc, writeStatuses);
|
||||
|
||||
if (config.shouldUseTempFolderForCopyOnWrite()) {
|
||||
// This is to rename each data file from temporary path to its final location
|
||||
jsc.parallelize(writeStatuses, config.getFinalizeWriteParallelism())
|
||||
.map(status -> status.getStat())
|
||||
.foreach(writeStat -> {
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path finalPath = new Path(config.getBasePath(), writeStat.getPath());
|
||||
|
||||
if (writeStat.getTempPath() != null) {
|
||||
final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath());
|
||||
boolean success;
|
||||
try {
|
||||
logger.info("Renaming temporary file: " + tempPath + " to " + finalPath);
|
||||
success = fs.rename(tempPath, finalPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
}
|
||||
|
||||
// This is to rename each data file from temporary path to its final location
|
||||
List<Tuple2<String, Boolean>> results = jsc
|
||||
.parallelize(writeStatuses, config.getFinalizeWriteParallelism()).map(writeStatus -> {
|
||||
Tuple2<String, HoodieWriteStat> writeStatTuple2 = (Tuple2<String, HoodieWriteStat>)
|
||||
writeStatus;
|
||||
HoodieWriteStat writeStat = writeStatTuple2._2();
|
||||
final FileSystem fs = getMetaClient().getFs();
|
||||
final Path finalPath = new Path(config.getBasePath(), writeStat.getPath());
|
||||
|
||||
if (writeStat.getTempPath() != null) {
|
||||
final Path tempPath = new Path(config.getBasePath(), writeStat.getTempPath());
|
||||
boolean success;
|
||||
try {
|
||||
logger.info("Renaming temporary file: " + tempPath + " to " + finalPath);
|
||||
success = fs.rename(tempPath, finalPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to rename file: " + tempPath + " to " + finalPath);
|
||||
}
|
||||
}
|
||||
|
||||
return new Tuple2<>(writeStat.getPath(), true);
|
||||
}).collect();
|
||||
|
||||
// clean temporary data files
|
||||
cleanTemporaryDataFiles(jsc);
|
||||
|
||||
return Optional.of(results.size());
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -43,6 +43,7 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.MergeOnReadLazyInsertIterable;
|
||||
@@ -294,9 +295,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<Integer> finalizeWrite(JavaSparkContext jsc, List writeStatuses) {
|
||||
// do nothing for MOR tables
|
||||
return Optional.empty();
|
||||
public void finalizeWrite(JavaSparkContext jsc, List<WriteStatus> writeStatuses)
|
||||
throws HoodieIOException {
|
||||
// delegate to base class for MOR tables
|
||||
super.finalizeWrite(jsc, writeStatuses);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -23,7 +23,6 @@ import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
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.TableFileSystemView;
|
||||
@@ -33,26 +32,31 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieSavepointException;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.ConsistencyCheck;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Abstract implementation of a HoodieTable
|
||||
*/
|
||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
// time between successive attempts to ensure written data's metadata is consistent on storage
|
||||
private static long INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
|
||||
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
|
||||
private static int MAX_CONSISTENCY_CHECKS = 7;
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T> index;
|
||||
@@ -245,11 +249,26 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Finalize the written data files
|
||||
* Finalize the written data onto storage. Perform any final cleanups
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param writeStatuses List of WriteStatus
|
||||
* @return number of files finalized
|
||||
* @throws HoodieIOException if some paths can't be finalized on storage
|
||||
*/
|
||||
public abstract Optional<Integer> finalizeWrite(JavaSparkContext jsc,
|
||||
List<Tuple2<String, HoodieWriteStat>> writeStatuses);
|
||||
public void finalizeWrite(JavaSparkContext jsc, List<WriteStatus> writeStatuses)
|
||||
throws HoodieIOException {
|
||||
if (config.isConsistencyCheckEnabled()) {
|
||||
List<String> pathsToCheck = writeStatuses.stream()
|
||||
.map(ws -> ws.getStat().getTempPath() != null
|
||||
? ws.getStat().getTempPath() : ws.getStat().getPath())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<String> failingPaths = new ConsistencyCheck(config.getBasePath(), pathsToCheck, jsc,
|
||||
config.getFinalizeWriteParallelism())
|
||||
.check(MAX_CONSISTENCY_CHECKS, INITIAL_CONSISTENCY_CHECK_INTERVAL_MS);
|
||||
if (failingPaths.size() > 0) {
|
||||
throw new HoodieIOException("Could not verify consistency of paths : " + failingPaths);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user