1
0

Spark Stage retry handling

This commit is contained in:
Balaji Varadarajan
2019-03-08 15:05:33 -08:00
committed by vinoth chandar
parent 3fd2fd6e9d
commit 145034c5fa
53 changed files with 1664 additions and 967 deletions

View File

@@ -17,14 +17,17 @@
package com.uber.hoodie.io;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.io.storage.HoodieWrapperFileSystem;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieTimer;
import com.uber.hoodie.common.util.NoOpConsistencyGuard;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
@@ -32,16 +35,19 @@ import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
protected final String commitTime;
protected final String fileId;
protected final String writeToken;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTable<T> hoodieTable;
@@ -50,10 +56,13 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
protected HoodieTimer timer;
protected final WriteStatus writeStatus;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable) {
public HoodieIOHandle(HoodieWriteConfig config, String commitTime, String fileId,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.fileId = fileId;
this.writeToken = makeSparkWriteToken();
this.config = config;
this.fs = hoodieTable.getMetaClient().getFs();
this.fs = getFileSystem(hoodieTable, config);
this.hoodieTable = hoodieTable;
this.originalSchema = new Schema.Parser().parse(config.getSchema());
this.writerSchema = createHoodieWriteSchema(originalSchema);
@@ -63,33 +72,26 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
config.getWriteStatusFailureFraction());
}
private static FileSystem getFileSystem(HoodieTable hoodieTable, HoodieWriteConfig config) {
return new HoodieWrapperFileSystem(hoodieTable.getMetaClient().getFs(), config.isConsistencyCheckEnabled()
? new FailSafeConsistencyGuard(hoodieTable.getMetaClient().getFs(),
config.getMaxConsistencyChecks(), config.getInitialConsistencyCheckIntervalMs(),
config.getMaxConsistencyCheckIntervalMs()) : new NoOpConsistencyGuard());
}
/**
* Deletes any new tmp files written during the current commit, into the partition
* Generate a write token based on the currently running spark task and its place in the spark dag.
*/
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config, String commitTime,
String partitionPath, int taskPartitionId, HoodieTable hoodieTable) {
FileSystem fs = hoodieTable.getMetaClient().getFs();
try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
if (prevFailedFiles != null) {
logger.info(
"Deleting " + prevFailedFiles.length + " files generated by previous failed attempts.");
for (FileStatus status : prevFailedFiles) {
fs.delete(status.getPath(), false);
}
}
} catch (IOException e) {
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime, e);
}
private static String makeSparkWriteToken() {
return FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
TaskContext.get().taskAttemptId());
}
public static Schema createHoodieWriteSchema(Schema originalSchema) {
return HoodieAvroUtils.addMetadataFields(originalSchema);
}
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
public Path makeNewPath(String partitionPath) {
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
try {
fs.mkdirs(path); // create a new partition as needed.
@@ -97,16 +99,37 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
throw new HoodieIOException("Failed to make dir " + path, e);
}
return new Path(path.toString(),
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
return new Path(path.toString(), FSUtils.makeDataFileName(commitTime, writeToken, fileId));
}
public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId,
long taskAttemptId) {
Path path = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME);
return new Path(path.toString(),
FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId,
taskAttemptId));
/**
* Creates an empty marker file corresponding to storage writer path
* @param partitionPath Partition path
*/
protected void createMarkerFile(String partitionPath) {
Path markerPath = makeNewMarkerPath(partitionPath);
try {
logger.info("Creating Marker Path=" + markerPath);
fs.create(markerPath, false).close();
} catch (IOException e) {
throw new HoodieException("Failed to create marker file " + markerPath, e);
}
}
/**
* THe marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename
* @param partitionPath
* @return
*/
private Path makeNewMarkerPath(String partitionPath) {
Path markerRootPath = new Path(hoodieTable.getMetaClient().getMarkerFolderPath(commitTime));
Path path = FSUtils.getPartitionPath(markerRootPath, partitionPath);
try {
fs.mkdirs(path); // create a new partition as needed.
} catch (IOException e) {
throw new HoodieIOException("Failed to make dir " + path, e);
}
return new Path(path.toString(), FSUtils.makeMarkerFile(commitTime, writeToken, fileId));
}
public Schema getWriterSchema() {