Skip to content

Commit

Permalink
Improve Google Batch executor stability and error handling (#5690)
Browse files Browse the repository at this point in the history

Signed-off-by: jorgee <jorge.ejarque@seqera.io>
Signed-off-by: Paolo Di Tommaso <paolo.ditommaso@gmail.com>
Signed-off-by: Ben Sherman <bentshermann@gmail.com>
Signed-off-by: Jorge Ejarque <jorgee@users.noreply.github.com>
Co-authored-by: Paolo Di Tommaso <paolo.ditommaso@gmail.com>
Co-authored-by: Ben Sherman <bentshermann@gmail.com>
Co-authored-by: Chris Hakkaart <chris.hakkaart@seqera.io>
  • Loading branch information
4 people authored Feb 12, 2025
1 parent db4a903 commit b64087f
Show file tree
Hide file tree
Showing 12 changed files with 319 additions and 65 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,13 @@ import groovy.transform.InheritConstructors
* Exception thrown when an error is raised during the process file staging phase
*
* @author Paolo Di Tommaso <paolo.ditommaso@gmail.com>
*
* Note: This exception extends ProcessUrecoverableException to force the execution to fail instead of
* retrying the task execution because the file staging process has its own retry strategy, and
* therefore it's likely to be a permanent error.
*
* See also https://github.com/nextflow-io/nextflow/issues/5727
*/
@InheritConstructors
class ProcessStageException extends ProcessException implements ShowOnlyExceptionMessage {
class ProcessStageException extends ProcessUnrecoverableException implements ShowOnlyExceptionMessage {
}
11 changes: 10 additions & 1 deletion modules/nextflow/src/main/groovy/nextflow/file/FilePorter.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ class FilePorter {
// remove the target file that could be have partially downloaded
cleanup(stagePath)
// check if a stage/download retry is allowed
if( count++ < maxRetries && e !instanceof NoSuchFileException && e !instanceof InterruptedIOException && !Thread.currentThread().isInterrupted() ) {
if( count++ < maxRetries && recoverableError(e) && !Thread.currentThread().isInterrupted() ) {
def message = "Unable to stage foreign file: ${filePath.toUriString()} (try ${count} of ${maxRetries}) -- Cause: $e.message"
log.isDebugEnabled() ? log.warn(message, e) : log.warn(message)

Expand All @@ -344,6 +344,15 @@ class FilePorter {
}
}

private boolean recoverableError(IOException e){
final result =
e !instanceof NoSuchFileException
&& (e instanceof SocketTimeoutException || e !instanceof InterruptedIOException)
&& e !instanceof SocketException
log.debug "Stage foreign file exception: recoverable=$result; type=${e.class.name}; message=${e.message}"
return result
}

private String fmtError(Path filePath, Exception e) {
def message = "Can't stage file ${FilesEx.toUriString(filePath)}"
if( e instanceof NoSuchFileException )
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ class TaskArrayCollector {
try {
// submit task directly if the collector is closed
// or if the task is retried (since it might have dynamic resources)
if( closed || task.config.getAttempt() > 1 ) {
if( closed ) {
executor.submit(task)
return
}
Expand Down Expand Up @@ -138,7 +138,7 @@ class TaskArrayCollector {
*/
protected TaskArrayRun createTaskArray(List<TaskRun> tasks) {
// prepare child job launcher scripts
final handlers = tasks.collect( t -> executor.createTaskHandler(t) )
final handlers = tasks.collect( t -> executor.createTaskHandler(t).withArrayChild(true) )
for( TaskHandler handler : handlers ) {
handler.prepareLauncher()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,16 @@ abstract class TaskHandler {
*/
TaskRun getTask() { task }

/**
* Whenever this handle reference a job array task child
*/
boolean isArrayChild

TaskHandler withArrayChild(boolean child) {
this.isArrayChild = child
return this
}

/**
* Task current status
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2185,15 +2185,13 @@ class TaskProcessor {
session.filePorter.transfer(batch)
}

final protected void makeTaskContextStage3( TaskRun task, HashCode hash, Path folder ) {

protected void makeTaskContextStage3( TaskRun task, HashCode hash, Path folder ) {
// set hash-code & working directory
task.hash = hash
task.workDir = folder
task.config.workDir = folder
task.config.hash = hash.toString()
task.config.name = task.getName()

}

final protected HashCode createTaskHashKey(TaskRun task) {
Expand Down Expand Up @@ -2340,12 +2338,12 @@ class TaskProcessor {

makeTaskContextStage3(task, hash, folder)

// when no collector is define OR it's a task retry, then submit directly for execution
if( !arrayCollector || task.config.getAttempt() > 1 )
executor.submit(task)
// add the task to the collection of running tasks
if( arrayCollector )
arrayCollector.collect(task)
else
executor.submit(task)

arrayCollector.collect(task)
}

protected boolean checkWhenGuard(TaskRun task) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,23 +87,6 @@ class TaskArrayCollectorTest extends Specification {
1 * executor.submit(task)
}

def 'should submit retried tasks directly' () {
given:
def executor = Mock(DummyExecutor)
def collector = Spy(new TaskArrayCollector(null, executor, 5))
and:
def task = Mock(TaskRun) {
getConfig() >> Mock(TaskConfig) {
getAttempt() >> 2
}
}

when:
collector.collect(task)
then:
1 * executor.submit(task)
}

def 'should create task array' () {
given:
def exec = Mock(DummyExecutor) {
Expand Down Expand Up @@ -137,6 +120,7 @@ class TaskArrayCollectorTest extends Specification {
when:
def taskArray = collector.createTaskArray([task, task, task])
then:
3 * handler.withArrayChild(true) >> handler
3 * exec.createTaskHandler(task) >> handler
3 * handler.prepareLauncher()
1 * collector.createArrayTaskScript([handler, handler, handler]) >> 'the-task-array-script'
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -277,4 +277,20 @@ class TaskHandlerTest extends Specification {
then:
0 * handler.killTask()
}

@Unroll
def 'should set isChildArray flag'() {
given:
def handler = Spy(TaskHandler)

expect:
!handler.isArrayChild
and:
handler.withArrayChild(VALUE).isArrayChild == VALUE

where:
VALUE | _
false | _
true | _
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.nio.file.Path
import java.nio.file.Paths
import java.util.concurrent.ExecutorService

import com.google.common.hash.HashCode
import groovyx.gpars.agent.Agent
import nextflow.Global
import nextflow.ISession
Expand Down Expand Up @@ -401,9 +402,7 @@ class TaskProcessorTest extends Specification {

}


def 'should update agent state'() {

when:
def state = new Agent<StateObj>(new StateObj())
int i = 0
Expand Down Expand Up @@ -1172,4 +1171,51 @@ class TaskProcessorTest extends Specification {

}

def 'should submit a task' () {
given:
def exec = Mock(Executor)
def proc = Spy(new TaskProcessor(executor: exec))
and:
def task = Mock(TaskRun)
def hash = Mock(HashCode)
def path = Mock(Path)

when:
proc.submitTask(task, hash, path)
then:
1 * proc.makeTaskContextStage3(task, hash, path) >> null
and:
1 * exec.submit(task)
}

def 'should collect a task' () {
given:
def exec = Mock(Executor)
def collector = Mock(TaskArrayCollector)
def proc = Spy(new TaskProcessor(executor: exec, arrayCollector: collector))
and:
def task = Mock(TaskRun)
def hash = Mock(HashCode)
def path = Mock(Path)

when:
proc.submitTask(task, hash, path)
then:
task.getConfig()>>Mock(TaskConfig) { getAttempt()>>1 }
and:
1 * proc.makeTaskContextStage3(task, hash, path) >> null
and:
1 * collector.collect(task)
0 * exec.submit(task)

when:
proc.submitTask(task, hash, path)
then:
task.getConfig()>>Mock(TaskConfig) { getAttempt()>>2 }
and:
1 * proc.makeTaskContextStage3(task, hash, path) >> null
and:
0 * collector.collect(task)
1 * exec.submit(task)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import com.google.cloud.batch.v1.AllocationPolicy
import com.google.cloud.batch.v1.ComputeResource
import com.google.cloud.batch.v1.Environment
import com.google.cloud.batch.v1.Job
import com.google.cloud.batch.v1.JobStatus
import com.google.cloud.batch.v1.LifecyclePolicy
import com.google.cloud.batch.v1.LogsPolicy
import com.google.cloud.batch.v1.Runnable
Expand Down Expand Up @@ -452,49 +453,73 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask {
* @return Retrieve the submitted task state
*/
protected String getTaskState() {
final tasks = client.listTasks(jobId)
if( !tasks.iterator().hasNext() ) {
// if there are no tasks checks the job status
return checkJobStatus()
}
return isArrayChild
? getStateFromTaskStatus()
: getStateFromJobStatus()
}

protected String getStateFromTaskStatus() {
final now = System.currentTimeMillis()
final delta = now - timestamp;
if( !taskState || delta >= 1_000) {
final status = client.getTaskStatus(jobId, taskId)
final newState = status?.state as String
if( newState ) {
log.trace "[GOOGLE BATCH] Get job=$jobId task=$taskId state=$newState"
taskState = newState
timestamp = now
}
if( newState == 'PENDING' ) {
final eventsCount = status.getStatusEventsCount()
final lastEvent = eventsCount > 0 ? status.getStatusEvents(eventsCount - 1) : null
if( lastEvent?.getDescription()?.contains('CODE_GCE_QUOTA_EXCEEDED') )
log.warn1 "Batch job cannot be run: ${lastEvent.getDescription()}"
final status = client.getTaskInArrayStatus(jobId, taskId)
if( status ) {
inspectTaskStatus(status)
} else {
// If no task status retrieved check job status
final jobStatus = client.getJobStatus(jobId)
inspectJobStatus(jobStatus)
}
}
return taskState
}

protected String checkJobStatus() {
final jobStatus = client.getJobStatus(jobId)
final newState = jobStatus?.state as String
protected String getStateFromJobStatus() {
final now = System.currentTimeMillis()
final delta = now - timestamp;
if( !taskState || delta >= 1_000) {
final status = client.getJobStatus(jobId)
inspectJobStatus(status)
}
return taskState
}

private void inspectTaskStatus(com.google.cloud.batch.v1.TaskStatus status) {
final newState = status?.state as String
if (newState) {
log.trace "[GOOGLE BATCH] Get job=$jobId task=$taskId state=$newState"
taskState = newState
timestamp = System.currentTimeMillis()
}
if (newState == 'PENDING') {
final eventsCount = status.getStatusEventsCount()
final lastEvent = eventsCount > 0 ? status.getStatusEvents(eventsCount - 1) : null
if (lastEvent?.getDescription()?.contains('CODE_GCE_QUOTA_EXCEEDED'))
log.warn1 "Batch job cannot be run: ${lastEvent.getDescription()}"
}
}

protected String inspectJobStatus(JobStatus status) {
final newState = status?.state as String
if (newState) {
log.trace "[GOOGLE BATCH] Get job=$jobId state=$newState"
taskState = newState
timestamp = System.currentTimeMillis()
if (newState == "FAILED") {
noTaskJobfailure = true
}
return taskState
} else {
return "PENDING"
}
if (newState == 'SCHEDULED') {
final eventsCount = status.getStatusEventsCount()
final lastEvent = eventsCount > 0 ? status.getStatusEvents(eventsCount - 1) : null
if (lastEvent?.getDescription()?.contains('CODE_GCE_QUOTA_EXCEEDED'))
log.warn1 "Batch job cannot be run: ${lastEvent.getDescription()}"
}
}

static private final List<String> RUNNING_OR_COMPLETED = ['RUNNING', 'SUCCEEDED', 'FAILED']
static private final List<String> RUNNING_OR_COMPLETED = ['RUNNING', 'SUCCEEDED', 'FAILED', 'DELETION_IN_PROGRESS']

static private final List<String> COMPLETED = ['SUCCEEDED', 'FAILED']
static private final List<String> COMPLETED = ['SUCCEEDED', 'FAILED', 'DELETION_IN_PROGRESS']

@Override
boolean checkIfRunning() {
Expand Down Expand Up @@ -526,6 +551,8 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask {
task.stderr = errorFile
}
status = TaskStatus.COMPLETED
if( isArrayChild )
client.removeFromArrayTasks(jobId, taskId)
return true
}

Expand Down
Loading

0 comments on commit b64087f

Please sign in to comment.