-
Notifications
You must be signed in to change notification settings - Fork 751
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[GOBBLIN-1947] Send workUnitChange event when helix task consistently fail #3832
Changes from all commits
a9de744
9c1e510
46145fb
d31d3e0
744dafb
dfd0d76
9ad98bd
978f989
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,10 +17,14 @@ | |
|
||
package org.apache.gobblin.cluster; | ||
|
||
import com.google.common.eventbus.Subscribe; | ||
import java.io.IOException; | ||
import java.lang.reflect.InvocationTargetException; | ||
import java.net.URI; | ||
import java.time.Duration; | ||
import java.util.ArrayList; | ||
import java.util.HashMap; | ||
import java.util.LinkedList; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
|
@@ -30,6 +34,14 @@ | |
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.TimeoutException; | ||
|
||
import java.util.stream.Collectors; | ||
import org.apache.gobblin.runtime.SourceDecorator; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaPartition; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaSource; | ||
import org.apache.gobblin.source.extractor.extract.kafka.KafkaUtils; | ||
import org.apache.gobblin.source.workunit.BasicWorkUnitStream; | ||
import org.apache.gobblin.source.workunit.WorkUnitStream; | ||
import org.apache.gobblin.stream.WorkUnitChangeEvent; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
|
@@ -78,6 +90,8 @@ | |
import org.apache.gobblin.util.PropertiesUtils; | ||
import org.apache.gobblin.util.SerializationUtils; | ||
|
||
import static org.apache.gobblin.util.JobLauncherUtils.*; | ||
|
||
|
||
/** | ||
* An implementation of {@link JobLauncher} that launches a Gobblin job using the Helix task framework. | ||
|
@@ -131,7 +145,7 @@ public class GobblinHelixJobLauncher extends AbstractJobLauncher { | |
private final long workFlowExpiryTimeSeconds; | ||
private final long helixJobStopTimeoutSeconds; | ||
private final long helixWorkflowSubmissionTimeoutSeconds; | ||
private Map<String, TaskConfig> workUnitToHelixConfig; | ||
private Map<String, TaskConfig> helixIdTaskConfigMap; | ||
private Retryer<Boolean> taskRetryer; | ||
|
||
public GobblinHelixJobLauncher(Properties jobProps, final HelixManager helixManager, Path appWorkDir, | ||
|
@@ -185,7 +199,7 @@ public GobblinHelixJobLauncher(Properties jobProps, final HelixManager helixMana | |
this.stateStores.getTaskStateStore(), this.outputTaskStateDir, this.getIssueRepository()); | ||
|
||
this.helixMetrics = helixMetrics; | ||
this.workUnitToHelixConfig = new HashMap<>(); | ||
this.helixIdTaskConfigMap = new HashMap<>(); | ||
this.taskRetryer = RetryerBuilder.<Boolean>newBuilder() | ||
.retryIfException() | ||
.withStopStrategy(StopStrategies.stopAfterAttempt(3)).build(); | ||
|
@@ -254,6 +268,76 @@ protected void runWorkUnits(List<WorkUnit> workUnits) throws Exception { | |
} | ||
} | ||
|
||
/** | ||
* The implementation of this method has the assumption that work unit change should never delete without adding new | ||
* work units, which will cause starvation. Thus, process {@link WorkUnitChangeEvent} for two scenario: | ||
* 1. workUnitChangeEvent only contains old tasks and no new tasks given: recalculate new work unit through kafka | ||
* source and pack with a min container setting. | ||
* 2. workUnitChangeEvent contains both valid old and new work unit: respect the information and directly remove | ||
* old tasks and start new work units | ||
* | ||
* @param workUnitChangeEvent Event post by EventBus to specify old tasks to be removed and new tasks to be run | ||
* @throws InvocationTargetException | ||
*/ | ||
@Override | ||
@Subscribe | ||
public void handleWorkUnitChangeEvent(WorkUnitChangeEvent workUnitChangeEvent) | ||
throws InvocationTargetException { | ||
log.info("Received WorkUnitChangeEvent with old Task {} and new WU {}", | ||
workUnitChangeEvent.getOldTaskIds(), workUnitChangeEvent.getNewWorkUnits()); | ||
final JobState jobState = this.jobContext.getJobState(); | ||
List<WorkUnit> workUnits = workUnitChangeEvent.getNewWorkUnits(); | ||
// Use old task Id to recalculate new work units | ||
if(workUnits == null || workUnits.isEmpty()) { | ||
workUnits = recalculateWorkUnit(workUnitChangeEvent.getOldTaskIds()); | ||
// If no new valid work units can be generated, dismiss the WorkUnitChangeEvent | ||
if(workUnits == null || workUnits.isEmpty()) { | ||
log.info("Not able to update work unit meaningfully, dismiss the WorkUnitChangeEvent"); | ||
return; | ||
} | ||
} | ||
|
||
// Follow how AbstractJobLauncher handles work units to make sure consistent behaviour | ||
WorkUnitStream workUnitStream = new BasicWorkUnitStream.Builder(workUnits).build(); | ||
// For streaming use case, this might be a necessary step to find dataset specific namespace so that each workUnit | ||
// can create staging and temp directories with the correct determination of shard-path | ||
workUnitStream = this.executeHandlers(workUnitStream, this.destDatasetHandlerService); | ||
workUnitStream = this.processWorkUnitStream(workUnitStream, jobState); | ||
workUnits = materializeWorkUnitList(workUnitStream); | ||
try { | ||
this.removeTasksFromCurrentJob(workUnitChangeEvent.getOldTaskIds()); | ||
this.addTasksToCurrentJob(workUnits); | ||
} catch (Exception e) { | ||
//todo: emit some event to indicate there is an error handling this event that may cause starvation | ||
log.error("Failed to process WorkUnitChangeEvent with old tasks {} and new workunits {}.", | ||
workUnitChangeEvent.getOldTaskIds(), workUnits, e); | ||
throw new InvocationTargetException(e); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. did we test what's the behavior for throw this exception? are we able to catch it and fail the whole application and restart directly? Or it will finally fail silently and starve? Also curious why do we throw InvocationTargetException? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Throwing error actually won't fail the application, so we rely on the Retryer and replanner(if Retryer also failed) here. I've run the test for a week and didn't see any error throwing, but I do agree this may be hard to debug. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For the InvocationTargetException, it's actually inherited from the super class which written by you :) |
||
} | ||
} | ||
|
||
private List<WorkUnit> recalculateWorkUnit(List<String> oldHelixTaskIDs) { | ||
JobState jobState = this.jobContext.getJobState(); | ||
Map<String, List<Integer>> filteredTopicPartition = new HashMap<>(); | ||
for(String id : oldHelixTaskIDs) { | ||
WorkUnit workUnit = getWorkUnitFromStateStoreByHelixId(id); | ||
String topicName = workUnit.getProp(KafkaSource.TOPIC_NAME); | ||
List<Integer> partitions = filteredTopicPartition.getOrDefault(topicName, | ||
new LinkedList<>()); | ||
partitions.addAll(KafkaUtils.getPartitions(workUnit).stream().map(KafkaPartition::getId).collect(Collectors.toList())); | ||
filteredTopicPartition.put(topicName, partitions); | ||
} | ||
// If a topic contains less than 2 filtered partition, it can't be further split so remove from map | ||
filteredTopicPartition.values().removeIf(list -> list == null || list.size() < 2); | ||
if(filteredTopicPartition.isEmpty()) { | ||
return new ArrayList<>(); | ||
} | ||
KafkaSource<?, ?> source = (KafkaSource<?, ?>) ((SourceDecorator<?, ?>) this.jobContext.getSource()).getSource(); | ||
//TODO: having a smarter way to calculate the new work unit size to replace the current static approach to simply double | ||
int newWorkUnitSize = oldHelixTaskIDs.size() * 2; | ||
return source.getWorkunitsForFilteredPartitions(jobState, | ||
com.google.common.base.Optional.of(filteredTopicPartition), com.google.common.base.Optional.of(newWorkUnitSize)); | ||
} | ||
|
||
@Override | ||
protected void executeCancellation() { | ||
if (this.jobSubmitted) { | ||
|
@@ -275,31 +359,31 @@ protected void executeCancellation() { | |
} | ||
} | ||
|
||
protected void removeTasksFromCurrentJob(List<String> workUnitIdsToRemove) throws IOException, ExecutionException, | ||
protected void removeTasksFromCurrentJob(List<String> helixTaskIdsToRemove) throws IOException, ExecutionException, | ||
RetryException { | ||
String jobName = this.jobContext.getJobId(); | ||
try (ParallelRunner stateSerDeRunner = new ParallelRunner(this.stateSerDeRunnerThreads, this.fs)) { | ||
for (String workUnitId : workUnitIdsToRemove) { | ||
for (String helixTaskId : helixTaskIdsToRemove) { | ||
taskRetryer.call(new Callable<Boolean>() { | ||
@Override | ||
public Boolean call() throws Exception { | ||
String taskId = workUnitToHelixConfig.get(workUnitId).getId(); | ||
String workUnitId = helixIdTaskConfigMap.get(helixTaskId).getConfigMap().get(ConfigurationKeys.TASK_ID_KEY); | ||
boolean remove = | ||
HelixUtils.deleteTaskFromHelixJob(helixWorkFlowName, jobName, taskId, helixTaskDriver); | ||
HelixUtils.deleteTaskFromHelixJob(helixWorkFlowName, jobName, helixTaskId, helixTaskDriver); | ||
if (remove) { | ||
log.info(String.format("Removed helix task %s with gobblin task id %s from helix job %s:%s ", taskId, | ||
log.info(String.format("Removed helix task %s with gobblin task id %s from helix job %s:%s ", helixTaskId, | ||
workUnitId, helixWorkFlowName, jobName)); | ||
} else { | ||
throw new IOException( | ||
String.format("Cannot remove task %s from helix job %s:%s", workUnitId, | ||
String.format("Cannot remove Helix task %s from helix job %s:%s", helixTaskId, | ||
helixWorkFlowName, jobName)); | ||
} | ||
return true; | ||
} | ||
}); | ||
deleteWorkUnitFromStateStore(workUnitId, stateSerDeRunner); | ||
log.info(String.format("remove task state for %s in state store", workUnitId)); | ||
this.workUnitToHelixConfig.remove(workUnitId); | ||
deleteWorkUnitFromStateStoreByHelixId(helixTaskId, stateSerDeRunner); | ||
log.info(String.format("Removed task state for Helix task %s in state store", helixTaskId)); | ||
this.helixIdTaskConfigMap.remove(helixTaskId); | ||
} | ||
} | ||
} | ||
|
@@ -513,7 +597,7 @@ private TaskConfig getTaskConfig(WorkUnit workUnit, ParallelRunner stateSerDeRun | |
rawConfigMap.put(ConfigurationKeys.TASK_ID_KEY, workUnit.getId()); | ||
rawConfigMap.put(GobblinClusterConfigurationKeys.TASK_SUCCESS_OPTIONAL_KEY, "true"); | ||
TaskConfig taskConfig = TaskConfig.Builder.from(rawConfigMap); | ||
workUnitToHelixConfig.put(workUnit.getId(), taskConfig); | ||
helixIdTaskConfigMap.put(taskConfig.getId(), taskConfig); | ||
return taskConfig; | ||
} | ||
|
||
|
@@ -526,12 +610,36 @@ private void addWorkUnit(WorkUnit workUnit, ParallelRunner stateSerDeRunner, Map | |
taskConfigMap.put(workUnit.getId(), getTaskConfig(workUnit, stateSerDeRunner)); | ||
} | ||
|
||
/** | ||
* get a single {@link WorkUnit} (flattened) from state store. | ||
*/ | ||
private WorkUnit getWorkUnitFromStateStoreByHelixId(String helixTaskId) { | ||
String workUnitFilePath = | ||
helixIdTaskConfigMap.get(helixTaskId).getConfigMap().get(GobblinClusterConfigurationKeys.WORK_UNIT_FILE_PATH); | ||
final StateStore stateStore; | ||
Path workUnitFile = new Path(workUnitFilePath); | ||
String workUnitId = helixIdTaskConfigMap.get(helixTaskId).getConfigMap().get(ConfigurationKeys.TASK_ID_KEY); | ||
final String fileName = workUnitFile.getName(); | ||
final String storeName = workUnitFile.getParent().getName(); | ||
if (fileName.endsWith(MULTI_WORK_UNIT_FILE_EXTENSION)) { | ||
stateStore = stateStores.getMwuStateStore(); | ||
} else { | ||
stateStore = stateStores.getWuStateStore(); | ||
} | ||
try { | ||
return (WorkUnit) stateStore.get(storeName, fileName, workUnitId); | ||
} catch (IOException ioException) { | ||
log.error("Failed to fetch workUnit for helix task {} from path {}", helixTaskId, workUnitFilePath); | ||
} | ||
return null; | ||
} | ||
|
||
/** | ||
* Delete a single {@link WorkUnit} (flattened) from state store. | ||
*/ | ||
private void deleteWorkUnitFromStateStore(String workUnitId, ParallelRunner stateSerDeRunner) { | ||
private void deleteWorkUnitFromStateStoreByHelixId(String helixTaskId, ParallelRunner stateSerDeRunner) { | ||
String workUnitFilePath = | ||
workUnitToHelixConfig.get(workUnitId).getConfigMap().get(GobblinClusterConfigurationKeys.WORK_UNIT_FILE_PATH); | ||
helixIdTaskConfigMap.get(helixTaskId).getConfigMap().get(GobblinClusterConfigurationKeys.WORK_UNIT_FILE_PATH); | ||
Path workUnitFile = new Path(workUnitFilePath); | ||
final String fileName = workUnitFile.getName(); | ||
final String storeName = workUnitFile.getParent().getName(); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In which scenario, we will have no new work units here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
When we send workUnitChangeEvent from YarnAutoScalingManager. The yarn service only has the information of helix, so may not easy to pre-calculate the new workUnit as this process need KafkaSource, which yarn isn't aware of.