From 41ac662aeb326c8a957852e5ce3f75bb369fb4a8 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 10:05:49 -0500 Subject: [PATCH 01/97] Add initial array executor Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/Session.groovy | 14 ++ .../nextflow/executor/ArrayExecutor.groovy | 124 +++++++++++++++ .../nextflow/executor/ArrayTaskAware.groovy | 35 +++++ .../nextflow/executor/ArrayTaskHandler.groovy | 123 +++++++++++++++ .../groovy/nextflow/executor/Executor.groovy | 4 +- .../nextflow/executor/ExecutorFactory.groovy | 38 +++-- .../executor/local/LocalExecutor.groovy | 11 +- .../processor/ArrayTaskPollingMonitor.groovy | 144 ++++++++++++++++++ .../nextflow/processor/TaskHandler.groovy | 4 +- .../processor/TaskPollingMonitor.groovy | 30 +++- .../nextflow/processor/TaskProcessor.groovy | 15 ++ .../nextflow/trace/TraceObserver.groovy | 13 +- .../executor/ArrayExecutorTest.groovy | 46 ++++++ 13 files changed, 572 insertions(+), 29 deletions(-) create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy create mode 100644 modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 5e8550f15f..c400e66a19 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -424,6 +424,9 @@ class Session implements ISession { return result } + void registerObserver(TraceObserver observer) { + observers << observer + } /* * intercepts interruption signal i.e. CTRL+C @@ -935,6 +938,17 @@ class Session implements ISession { } } + void notifyProcessClose(String process) { + observers.each { observer -> + try { + observer.onProcessClose(process) + } + catch( Exception e ) { + log.debug(e.getMessage(), e) + } + } + } + void notifyProcessTerminate(TaskProcessor process) { for( int i=0; i + */ +@Slf4j +@CompileStatic +class ArrayExecutor extends Executor implements TraceObserver { + + private ArrayTaskAware target + + private Integer arraySize + + private Map> queues = new ConcurrentHashMap<>() + + /** + * Initialize the executor class + */ + @Override + protected void register() { + super.register() + + session.registerObserver(this) + + final targetName = session.getExecConfigProp('array', 'target', 'local') as String + try { + target = (ArrayTaskAware)session.executorFactory.getExecutor(targetName, session) + } + catch( GroovyCastException e ) { + throw new IllegalArgumentException("Executor '${targetName}' does not support array jobs") + } + + arraySize = session.getExecConfigProp('array', 'arraySize', 100) as Integer + + log.debug "Creating 'array' executor > target executor: '${targetName}', array size: ${arraySize}" + } + + @Override + TaskMonitor createTaskMonitor() { + return ArrayTaskPollingMonitor.create(session, name, 100, Duration.of('5 sec')) + } + + /** + * Add submitted tasks to the queue, and schedule an array job when + * the queue reaches the desired size. + * + * @param task + */ + @Override + synchronized void submit( TaskRun task ) { + log.trace "Scheduling process: ${task}" + + if( session.isTerminated() ) + new IllegalStateException("Session terminated - Cannot add process to execution array: ${task}") + + // add task to the corresponding process queue + final process = task.processor.name + if( process !in queues ) + queues[process] = [] + + final array = queues[process] + array << task + + // schedule array job when the process queue is full + if( array.size() == arraySize ) { + log.debug "[ARRAY] Submitting array job for process '${process}'" + + monitor.schedule(target.createArrayTaskHandler(array)) + queues[process] = [] + } + } + + @Override + TaskHandler createTaskHandler(TaskRun task) { + throw new UnsupportedOperationException() + } + + /** + * Submit any remaining tasks as a partial batch when a process is closed. + * + * @param process + */ + @Override + void onProcessClose(String process) { + final array = queues[process] + + if( array != null && array.size() > 0 ) { + log.debug "[ARRAY] Submitting remainder array job for process '${process}'" + + monitor.schedule(target.createArrayTaskHandler(array)) + } + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy new file mode 100644 index 0000000000..82476c6ef4 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy @@ -0,0 +1,35 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.executor + +import nextflow.processor.TaskRun + +/** + * Interface for executors that support array jobs. + * + * @author Ben Sherman + */ +interface ArrayTaskAware { + + /** + * Create a task handler for an array job. + * + * @param array + */ + ArrayTaskHandler createArrayTaskHandler( List array ) + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy new file mode 100644 index 0000000000..8505d144e0 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy @@ -0,0 +1,123 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.executor + + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.processor.TaskHandler +import nextflow.processor.TaskStatus +import nextflow.trace.TraceRecord +/** + * Generic handler for an array task, which simply launches + * each task and waits for it to finish. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class ArrayTaskHandler extends TaskHandler { + + final List array + + ArrayTaskHandler(List array) { + // use the first task to provide common properties (e.g. configuration) + super(array.first().getTask()) + + this.array = array + } + + @Override + void submit() { + for( TaskHandler handler : array ) + handler.submit() + + status = TaskStatus.SUBMITTED + } + + @Override + boolean checkIfRunning() { + for( TaskHandler handler : array ) + if( !handler.checkIfRunning() ) + return false + + status = TaskStatus.RUNNING + return true + } + + @Override + boolean checkIfCompleted() { + for( TaskHandler handler : array ) + if( !handler.checkIfCompleted() ) + return false + + status = TaskStatus.COMPLETED + return true + } + + @Override + void kill() { + for( TaskHandler handler : array ) + handler.kill() + } + + @Override + protected StringBuilder toStringBuilder( StringBuilder builder ) { + builder << '\n' + for( TaskHandler handler : array ) + builder << ' ' << handler.toString() << '\n' + + return builder + } + + @Override + String getStatusString() { + if( array.any { h -> h.task.failed } ) return 'FAILED' + if( array.any { h -> h.task.aborted } ) return 'ABORTED' + return this.status.toString() + } + + @Override + TraceRecord safeTraceRecord() { + throw new UnsupportedOperationException() + } + + @Override + TraceRecord getTraceRecord() { + throw new UnsupportedOperationException() + } + + @Override + boolean canForkProcess() { + final max = task.processor.maxForks + return !max + ? true + : task.processor.forksCount + array.size() <= max + } + + @Override + void incProcessForks() { + task.processor.forksCount?.add(array.size()) + } + + @Override + void decProcessForks() { + task.processor.forksCount?.add(-array.size()) + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy index 5222e13c1b..632a066666 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy @@ -49,7 +49,7 @@ abstract class Executor { /** * The queue holder that keep track of all tasks for this executor. */ - private TaskMonitor monitor + protected TaskMonitor monitor TaskMonitor getMonitor() { monitor } @@ -79,7 +79,7 @@ abstract class Executor { * * @param task A {@code TaskRun} instance */ - final void submit( TaskRun task ) { + void submit( TaskRun task ) { log.trace "Scheduling process: ${task}" if( session.isTerminated() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy index 6b72deab11..c6b00f9241 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy @@ -43,25 +43,26 @@ class ExecutorFactory { * Map the executor class to its 'friendly' name */ final static Map> BUILT_IN_EXECUTORS = [ - 'nope': NopeExecutor, - 'local': LocalExecutor, - 'flux': FluxExecutor, - 'sge': SgeExecutor, - 'oge': SgeExecutor, - 'uge': SgeExecutor, - 'lsf': LsfExecutor, - 'pbs': PbsExecutor, - 'pbspro': PbsProExecutor, - 'slurm': SlurmExecutor, + 'array': ArrayExecutor, 'bridge': BridgeExecutor, - 'crg': CrgExecutor, 'bsc': LsfExecutor, 'condor': CondorExecutor, + 'crg': CrgExecutor, + 'flux': FluxExecutor, + 'hq': HyperQueueExecutor, 'k8s': K8sExecutor, - 'nqsii': NqsiiExecutor, + 'local': LocalExecutor, + 'lsf': LsfExecutor, 'moab': MoabExecutor, + 'nope': NopeExecutor, + 'nqsii': NqsiiExecutor, 'oar': OarExecutor, - 'hq': HyperQueueExecutor + 'oge': SgeExecutor, + 'pbs': PbsExecutor, + 'pbspro': PbsProExecutor, + 'sge': SgeExecutor, + 'slurm': SlurmExecutor, + 'uge': SgeExecutor ] @PackageScope Map> executorsMap @@ -193,6 +194,17 @@ class ExecutorFactory { clazz = LocalExecutor.class } + return getExecutor0(name, clazz, session) + } + + Executor getExecutor(String name, Session session) { + // -- load the executor to be used + def clazz = getExecutorClass(name) + + return getExecutor0(name, clazz, session) + } + + protected Executor getExecutor0(String name, Class clazz, Session session) { // this code is not supposed to be executed parallel def result = executors.get(clazz) if( result ) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy index aa0f1f4f5c..bb1a1b7650 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy @@ -19,6 +19,9 @@ package nextflow.executor.local import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import nextflow.executor.ArrayTaskAware +import nextflow.executor.ArrayTaskHandler +import nextflow.executor.Executor import nextflow.executor.Executor import nextflow.executor.SupportedScriptTypes import nextflow.fusion.FusionHelper @@ -35,7 +38,7 @@ import nextflow.script.ScriptType @Slf4j @CompileStatic @SupportedScriptTypes( [ScriptType.SCRIPTLET, ScriptType.GROOVY] ) -class LocalExecutor extends Executor { +class LocalExecutor extends Executor implements ArrayTaskAware { @Override protected TaskMonitor createTaskMonitor() { @@ -62,5 +65,11 @@ class LocalExecutor extends Executor { boolean isFusionEnabled() { return FusionHelper.isFusionEnabled(session) } + + @Override + ArrayTaskHandler createArrayTaskHandler(List array) { + final handlers = array.collect { task -> createTaskHandler(task) } + new ArrayTaskHandler(handlers) + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy new file mode 100644 index 0000000000..949097e3cf --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy @@ -0,0 +1,144 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.processor + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.Session +import nextflow.executor.ArrayTaskHandler +import nextflow.util.Duration + +/** + * Extension of the polling monitor for array jobs. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class ArrayTaskPollingMonitor extends TaskPollingMonitor { + + protected ArrayTaskPollingMonitor(Map params) { + super(params) + } + + static ArrayTaskPollingMonitor create( Session session, String name, int defQueueSize, Duration defPollInterval ) { + final capacity = session.getQueueSize(name, defQueueSize) + final pollInterval = session.getPollInterval(name, defPollInterval) + final dumpInterval = session.getMonitorDumpInterval(name) + + log.debug "Creating task monitor for executor '$name' > capacity: $capacity; pollInterval: $pollInterval; dumpInterval: $dumpInterval " + new ArrayTaskPollingMonitor(name: name, session: session, capacity: capacity, pollInterval: pollInterval, dumpInterval: dumpInterval) + } + + protected ArrayTaskHandler toArrayHandler(TaskHandler handler) { + if( handler !instanceof ArrayTaskHandler ) + throw new IllegalStateException() + + return (ArrayTaskHandler)handler + } + + @Override + protected void handleException(TaskHandler handler, Throwable error) { + handler = toArrayHandler(handler) + + def fault = null + def faultHandler = null + try { + // remove the array task from the processing queue + if( evict(handler) ) + handler.decProcessForks() + + // attempt to retry each task in the array + handler.array.each { h -> + def fault0 = h.task.processor.resumeOrDie(h.task, error) + if( fault == null && fault0 ) { + fault = fault0 + faultHandler = h + } + } + + log.trace "Task fault (1): $fault" + } + finally { + // abort the session if a task fault was returned + if( fault instanceof TaskFault ) + session.fault(fault, handler) + } + } + + @Override + protected void checkTaskStatus( TaskHandler handler ) { + handler = toArrayHandler(handler) + + // check if the array task is started + if( handler.checkIfRunning() ) { + log.trace "Task started > $handler" + notifyTaskStart(handler) + } + + // check if the array task is completed + if( handler.checkIfCompleted() ) { + log.debug "Task completed > $handler" + + // decrement forks count + handler.decProcessForks() + + // remove the array task from the processing queue + evict(handler) + + // attempt to finalize each task in the array + def fault = null + def faultHandler = null + handler.array.each { h -> + final fault0 = h.task.processor.finalizeTask(h.task) + if( fault == null && fault0 ) { + fault = fault0 + faultHandler = h + } + } + + // notify task completion + notifyTaskComplete(handler) + + // abort the session if a task fault was returned + if( fault instanceof TaskFault ) + session.fault(fault, faultHandler) + } + } + + @Override + protected void notifyTaskSubmit(TaskHandler handler) { + toArrayHandler(handler).array.each { h -> session.notifyTaskSubmit(h) } + } + + @Override + protected void notifyTaskPending(TaskHandler handler) { + toArrayHandler(handler).array.each { h -> session.notifyTaskPending(h) } + } + + @Override + protected void notifyTaskStart(TaskHandler handler) { + toArrayHandler(handler).array.each { h -> session.notifyTaskStart(h) } + } + + @Override + protected void notifyTaskComplete(TaskHandler handler) { + toArrayHandler(handler).array.each { h -> session.notifyTaskComplete(h) } + } + +} + diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index a94ded210f..ca285afe71 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -231,14 +231,14 @@ abstract class TaskHandler { /** * Increment the number of current forked processes */ - final void incProcessForks() { + void incProcessForks() { task.processor.forksCount?.increment() } /** * Decrement the number of current forked processes */ - final void decProcessForks() { + void decProcessForks() { task.processor.forksCount?.decrement() } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 84c37bb327..3ea4b9abca 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -197,7 +197,7 @@ class TaskPollingMonitor implements TaskMonitor { // this guarantees that in the queue are only jobs successfully submitted runningQueue.add(handler) // notify task submission - session.notifyTaskSubmit(handler) + notifyTaskSubmit(handler) } /** @@ -225,7 +225,7 @@ class TaskPollingMonitor implements TaskMonitor { try{ pendingQueue << handler taskAvail.signal() // signal that a new task is available for execution - session.notifyTaskPending(handler) + notifyTaskPending(handler) log.trace "Scheduled task > $handler" } finally { @@ -562,7 +562,7 @@ class TaskPollingMonitor implements TaskMonitor { } catch ( Throwable e ) { handleException(handler, e) - session.notifyTaskComplete(handler) + notifyTaskComplete(handler) } // remove processed handler either on successful submit or failed one (managed by catch section) // when `canSubmit` return false the handler should be retained to be tried in a following iteration @@ -573,7 +573,7 @@ class TaskPollingMonitor implements TaskMonitor { } - final protected void handleException( TaskHandler handler, Throwable error ) { + protected void handleException( TaskHandler handler, Throwable error ) { def fault = null try { if (evict(handler)) { @@ -603,7 +603,7 @@ class TaskPollingMonitor implements TaskMonitor { // check if it is started if( handler.checkIfRunning() ) { log.trace "Task started > $handler" - session.notifyTaskStart(handler) + notifyTaskStart(handler) } // check if it is terminated @@ -619,7 +619,7 @@ class TaskPollingMonitor implements TaskMonitor { final fault = handler.task.processor.finalizeTask(handler.task) // notify task completion - session.notifyTaskComplete(handler) + notifyTaskComplete(handler) // abort the execution in case of task failure if (fault instanceof TaskFault) { @@ -658,7 +658,7 @@ class TaskPollingMonitor implements TaskMonitor { // notify task completion handler.task.aborted = true - session.notifyTaskComplete(handler) + notifyTaskComplete(handler) } try { @@ -676,5 +676,21 @@ class TaskPollingMonitor implements TaskMonitor { return pendingQueue } + protected void notifyTaskSubmit(TaskHandler handler) { + session.notifyTaskSubmit(handler) + } + + protected void notifyTaskPending(TaskHandler handler) { + session.notifyTaskPending(handler) + } + + protected void notifyTaskStart(TaskHandler handler) { + session.notifyTaskStart(handler) + } + + protected void notifyTaskComplete(TaskHandler handler) { + session.notifyTaskComplete(handler) + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 87b4834a8b..a084836759 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -2426,6 +2426,21 @@ class TaskProcessor { log.trace "<${name}> Poison pill arrived; port: $index" openPorts.set(index, 0) // mark the port as closed state.update { StateObj it -> it.poison() } + + // check whether all input channels are closed + def closed = true + for( int i = 0; i < openPorts.length() - 1; i++ ) { + if( openPorts.get(i) != 0 ) { + closed = false + break + } + } + + // notify session that the process is closed + if( closed ) { + log.trace "<${name}> All ports are closed, closing process" + session.notifyProcessClose(name) + } } return message diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy index 45e414764e..f96cd0985a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy @@ -45,14 +45,19 @@ trait TraceObserver { */ void onFlowComplete() {} - /* + /** * Invoked when the process is created. */ void onProcessCreate( TaskProcessor process ){} - /* - * Invoked when all tak have been executed and process ends. - */ + /** + * Invoked when all tasks for a process have been created. + */ + void onProcessClose( String process ){} + + /** + * Invoked when all tasks have been executed and process ends. + */ void onProcessTerminate( TaskProcessor process ){} /** diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy new file mode 100644 index 0000000000..66b0e62b01 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy @@ -0,0 +1,46 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.executor + +import nextflow.Session +import spock.lang.Specification +/** + * + * @author Ben Sherman + */ +class ArrayExecutorTest extends Specification { + + def 'should throw error if target executor does not support array jobs' () { + + setup: + def executorFactory = Spy(ExecutorFactory) + def session = Mock(Session) { + getExecutorFactory() >> executorFactory + } + def executor = new ArrayExecutor() + executor.session = session + + when: + executor.register() + then: + session.getExecConfigProp('array', 'target', 'local') >> 'nope' + executorFactory.getExecutor('nope', session) >> new NopeExecutor() + IllegalArgumentException e = thrown() + e.message == "Executor 'nope' does not support array jobs" + } + +} From a0900ae814dce30e29a4a793c354453199af95f9 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 10:13:54 -0500 Subject: [PATCH 02/97] Add support for SLURM array jobs Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 43 +++++- .../executor/ArrayGridTaskHandler.groovy | 141 ++++++++++++++++++ .../nextflow/executor/GridTaskHandler.groovy | 69 +-------- .../nextflow/executor/SlurmExecutor.groovy | 13 ++ .../nextflow/executor/SubmitRetryAware.groovy | 106 +++++++++++++ 5 files changed, 303 insertions(+), 69 deletions(-) create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index cfad0b5246..6a958f90e1 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -21,6 +21,7 @@ import java.nio.file.Path import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.util.logging.Slf4j +import nextflow.processor.TaskHandler import nextflow.processor.TaskMonitor import nextflow.processor.TaskPollingMonitor import nextflow.processor.TaskProcessor @@ -36,7 +37,7 @@ import org.apache.commons.lang.StringUtils */ @Slf4j @CompileStatic -abstract class AbstractGridExecutor extends Executor { +abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { protected Duration queueInterval @@ -65,7 +66,7 @@ abstract class AbstractGridExecutor extends Executor { /* * Prepare and launch the task in the underlying execution platform */ - GridTaskHandler createTaskHandler(TaskRun task) { + TaskHandler createTaskHandler(TaskRun task) { assert task assert task.workDir @@ -407,5 +408,43 @@ abstract class AbstractGridExecutor extends Executor { // Instead, it is the command wrapper script that is launched run within a container process. return isFusionEnabled() } + + @Override + ArrayTaskHandler createArrayTaskHandler(List array) { + final handlers = array.collect { task -> createTaskHandler(task) } + new ArrayGridTaskHandler(handlers, this) + } + + String createArrayTaskWrapper(ArrayGridTaskHandler handler) { + final array = handler.array + final task = array.first().getTask() + + final arrayHeader = getArrayDirective(array.size()) + final taskHeaders = getHeaders(task) + final files = array + .collect { h -> ((GridTaskHandler)h).wrapperFile } + .join(' ') + + final builder = new StringBuilder() + << '#!/bin/bash\n' + << "${headerToken} ${arrayHeader}\n" + << taskHeaders + << "declare -a array=( ${files} )\n" + << "bash \${array[\$${arrayIndexName}]}\n" + + return builder.toString() + } + + protected String getArrayDirective(int arraySize) { + throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + } + + protected String getArrayIndexName() { + throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + } + + protected List getArraySubmitCommandLine() { + throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy new file mode 100644 index 0000000000..3fd24be38d --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy @@ -0,0 +1,141 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.executor + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.exception.ProcessFailedException +import nextflow.exception.ProcessNonZeroExitStatusException +import nextflow.processor.TaskHandler +import nextflow.processor.TaskStatus +import nextflow.util.CmdLineHelper +/** + * Handles the execution of an array job for any grid executor. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware { + + final AbstractGridExecutor executor + + private jobId + + ArrayGridTaskHandler(List array, AbstractGridExecutor executor) { + super(array) + + this.executor = executor + } + + @Override + void submit() { + ProcessBuilder builder = null + try { + // -- create the array job script + final arrayScript = executor.createArrayTaskWrapper(this) + // -- create the submit command + builder = createProcessBuilder() + // -- submit the array job with a retryable strategy + final result = safeExecute( () -> processStart(builder, arrayScript) ) + // -- save the job id + this.jobId = executor.parseJobId(result) + this.status = TaskStatus.SUBMITTED + log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" + } + catch( Exception e ) { + // update task exit status and message + if( e instanceof ProcessNonZeroExitStatusException ) { + for( TaskHandler handler : array ) { + handler.task.exitStatus = e.getExitStatus() + handler.task.stdout = e.getReason() + handler.task.script = e.getCommand() + } + } + else { + for( TaskHandler handler : array ) + handler.task.script = builder ? CmdLineHelper.toLine(builder.command()) : null + } + this.status = TaskStatus.COMPLETED + throw new ProcessFailedException("Error submitting array job for execution", e) + } + } + + protected ProcessBuilder createProcessBuilder() { + + // -- log the submit command + final cli = executor.getArraySubmitCommandLine() + log.trace "submit array job > cli: ${cli}" + + // -- launch array job script + new ProcessBuilder() + .command( cli as String[] ) + .redirectErrorStream(true) + } + + protected String processStart(ProcessBuilder builder, String arrayScript) { + final process = builder.start() + + try { + // -- pipe the array job script to the command stdin + log.trace "[${executor.name.toUpperCase()}] Submit array job >\n${arrayScript.indent()}" + process.out << arrayScript + process.out.close() + + // -- wait for the submission to complete + final result = process.text + final exitStatus = process.waitFor() + final cmd = launchCmd0(builder, arrayScript) + + if( exitStatus ) + throw new ProcessNonZeroExitStatusException("Failed to submit array job to grid scheduler for execution", result, exitStatus, cmd) + + // -- return the process stdout + return result + } + finally { + // make sure to release all resources + process.in.closeQuietly() + process.out.closeQuietly() + process.err.closeQuietly() + process.destroy() + } + } + + protected String launchCmd0(ProcessBuilder builder, String arrayScript) { + final cmd = CmdLineHelper.toLine(builder.command()) + + new StringBuilder() + .append("cat << 'LAUNCH_COMMAND_EOF' | ${cmd}\n") + .append(arrayScript.trim()) + .append('\nLAUNCH_COMMAND_EOF\n') + .toString() + } + + @Override + void kill() { + executor.killTask(jobId) + } + + @Override + protected StringBuilder toStringBuilder(StringBuilder builder) { + builder << "\n array jobId: $jobId; " + + return super.toStringBuilder(builder) + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 8de788918d..eac2fd6134 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -20,16 +20,7 @@ import static nextflow.processor.TaskStatus.* import java.nio.file.Path import java.nio.file.attribute.BasicFileAttributes -import java.time.temporal.ChronoUnit -import java.util.function.Predicate -import java.util.regex.Pattern - -import dev.failsafe.Failsafe -import dev.failsafe.RetryPolicy -import dev.failsafe.event.EventListener -import dev.failsafe.event.ExecutionAttemptedEvent -import dev.failsafe.function.CheckedSupplier -import groovy.transform.Memoized + import groovy.util.logging.Slf4j import nextflow.exception.ProcessException import nextflow.exception.ProcessFailedException @@ -47,7 +38,7 @@ import nextflow.util.Throttle * Handles a job execution in the underlying grid platform */ @Slf4j -class GridTaskHandler extends TaskHandler implements FusionAwareTask { +class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetryAware { /** The target executor platform */ final AbstractGridExecutor executor @@ -116,62 +107,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return builder } - @Memoized - protected Predicate retryCondition(String reasonPattern) { - final pattern = Pattern.compile(reasonPattern) - return new Predicate() { - @Override - boolean test(Throwable failure) { - if( failure instanceof ProcessNonZeroExitStatusException ) { - final reason = failure.reason - return reason ? pattern.matcher(reason).find() : false - } - return false - } - } - } - - protected RetryPolicy retryPolicy() { - - final delay = executor.session.getConfigAttribute("executor.retry.delay", '500ms') as Duration - final maxDelay = executor.session.getConfigAttribute("executor.retry.maxDelay", '30s') as Duration - final jitter = executor.session.getConfigAttribute("executor.retry.jitter", '0.25') as double - final maxAttempts = executor.session.getConfigAttribute("executor.retry.maxAttempts", '3') as int - final reason = executor.session.getConfigAttribute("executor.submit.retry.reason", 'Socket timed out') as String - - final listener = new EventListener() { - @Override - void accept(ExecutionAttemptedEvent event) throws Throwable { - final failure = event.getLastFailure() - if( failure instanceof ProcessNonZeroExitStatusException ) { - final msg = """\ - Failed to submit process '${task.name}' - - attempt : ${event.attemptCount} - - command : ${failure.command} - - reason : ${failure.reason} - """.stripIndent(true) - log.warn msg - - } else { - log.debug("Unexpected retry failure: ${failure?.message}", failure) - } - } - } - - return RetryPolicy.builder() - .handleIf(retryCondition(reason)) - .withBackoff(delay.toMillis(), maxDelay.toMillis(), ChronoUnit.MILLIS) - .withMaxAttempts(maxAttempts) - .withJitter(jitter) - .onFailedAttempt(listener) - .build() - } - - protected T safeExecute(CheckedSupplier action) { - final policy = retryPolicy() - return Failsafe.with(policy).get(action) - } - protected String processStart(ProcessBuilder builder, String pipeScript) { final process = builder.start() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index c4d785e696..9f57f3ad34 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -48,6 +48,7 @@ class SlurmExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ + @Override protected List getDirectives(TaskRun task, List result) { result << '-J' << getJobNameFor(task) @@ -90,6 +91,7 @@ class SlurmExecutor extends AbstractGridExecutor { return result } + @Override String getHeaderToken() { '#SBATCH' } /** @@ -197,4 +199,15 @@ class SlurmExecutor extends AbstractGridExecutor { boolean isFusionEnabled() { return FusionHelper.isFusionEnabled(session) } + + @Override + protected String getArrayDirective(int arraySize) { + "--array 0-${arraySize - 1}" + } + + @Override + protected String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } + + @Override + List getArraySubmitCommandLine() { List.of('sbatch') } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy new file mode 100644 index 0000000000..95adeeba16 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy @@ -0,0 +1,106 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.executor + +import java.time.temporal.ChronoUnit +import java.util.function.Predicate +import java.util.regex.Pattern + +import dev.failsafe.Failsafe +import dev.failsafe.RetryPolicy +import dev.failsafe.event.EventListener +import dev.failsafe.event.ExecutionAttemptedEvent +import dev.failsafe.function.CheckedSupplier +import groovy.transform.Memoized +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.exception.ProcessNonZeroExitStatusException +import nextflow.processor.TaskRun +import nextflow.util.Duration +/** + * Generic retry-able submit implementation for executors. + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +trait SubmitRetryAware { + + abstract Executor getExecutor() + + abstract TaskRun getTask() + + T safeExecute(CheckedSupplier action) { + Failsafe.with(retryPolicy()).get(action) + } + + private RetryPolicy retryPolicy() { + + final delay = executor.session.getConfigAttribute('executor.retry.delay', '500ms') as Duration + final maxDelay = executor.session.getConfigAttribute('executor.retry.maxDelay', '30s') as Duration + final jitter = executor.session.getConfigAttribute('executor.retry.jitter', '0.25') as double + final maxAttempts = executor.session.getConfigAttribute('executor.retry.maxAttempts', '3') as int + final reason = executor.session.getConfigAttribute('executor.submit.retry.reason', 'Socket timed out') as String + + RetryPolicy.builder() + .handleIf(retryCondition(reason)) + .withBackoff(delay.toMillis(), maxDelay.toMillis(), ChronoUnit.MILLIS) + .withMaxAttempts(maxAttempts) + .withJitter(jitter) + .onFailedAttempt(new RetryListener(task: task)) + .build() + } + + static private class RetryListener extends EventListener { + TaskRun task + + @Override + void accept(ExecutionAttemptedEvent event) throws Throwable { + final failure = event.getLastFailure() + if( failure instanceof ProcessNonZeroExitStatusException ) { + final msg = """\ + Failed to submit process '${task.name}' + - attempt : ${event.attemptCount} + - command : ${failure.command} + - reason : ${failure.reason} + """.stripIndent(true) + log.warn msg + } else { + log.debug("Unexpected retry failure: ${failure?.message}", failure) + } + } + } + + @Memoized + private Predicate retryCondition(String reasonPattern) { + new RetryPredicate(pattern: Pattern.compile(reasonPattern)) + } + + static private class RetryPredicate extends Predicate { + Pattern pattern + + @Override + boolean test(Throwable failure) { + if( failure instanceof ProcessNonZeroExitStatusException ) { + final reason = failure.reason + return reason ? pattern.matcher(reason).find() : false + } + return false + } + } + +} From b5bad002426457827fe43181451d93e3556d41b3 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 11:04:47 -0500 Subject: [PATCH 03/97] Fix failing test Signed-off-by: Ben Sherman --- .../nextflow/executor/SubmitRetryAware.groovy | 4 +- .../executor/GridTaskHandlerTest.groovy | 15 ------- .../executor/SubmitRetryAwareTest.groovy | 43 +++++++++++++++++++ 3 files changed, 45 insertions(+), 17 deletions(-) create mode 100644 modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy index 95adeeba16..c209fa5d99 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy @@ -48,7 +48,7 @@ trait SubmitRetryAware { Failsafe.with(retryPolicy()).get(action) } - private RetryPolicy retryPolicy() { + RetryPolicy retryPolicy() { final delay = executor.session.getConfigAttribute('executor.retry.delay', '500ms') as Duration final maxDelay = executor.session.getConfigAttribute('executor.retry.maxDelay', '30s') as Duration @@ -86,7 +86,7 @@ trait SubmitRetryAware { } @Memoized - private Predicate retryCondition(String reasonPattern) { + Predicate retryCondition(String reasonPattern) { new RetryPredicate(pattern: Pattern.compile(reasonPattern)) } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 01a028e05a..96dc2b485f 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -35,21 +35,6 @@ import test.TestHelper */ class GridTaskHandlerTest extends Specification { - def 'should check retry predicate' () { - given: - def handler = new GridTaskHandler() - - when: - def predicate = handler.retryCondition("Socket timed out") - then: - predicate.test(new ProcessNonZeroExitStatusException('Error', 'Socket timed out', 1, null)) - and: - predicate.test(new ProcessNonZeroExitStatusException('Error', 'error\nBatch job submission failed\nSocket timed out on send/recv operation', 1, null)) - and: - !predicate.test(new ProcessNonZeroExitStatusException('Error', 'OK', 0, null)) - - } - def 'should capture error cause' () { given: def task = new TaskRun(name: 'foo', workDir: Paths.get('/some/work')) diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy new file mode 100644 index 0000000000..7c220c4c19 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy @@ -0,0 +1,43 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.executor + +import nextflow.exception.ProcessNonZeroExitStatusException +import spock.lang.Specification +/** + * + * @author Paolo Di Tommaso + */ +class SubmitRetryAwareTest extends Specification { + + def 'should check retry predicate' () { + given: + def retry = [:] as SubmitRetryAware + + when: + def predicate = retry.retryCondition("Socket timed out") + then: + predicate.test(new ProcessNonZeroExitStatusException('Error', 'Socket timed out', 1, null)) + and: + predicate.test(new ProcessNonZeroExitStatusException('Error', 'error\nBatch job submission failed\nSocket timed out on send/recv operation', 1, null)) + and: + !predicate.test(new ProcessNonZeroExitStatusException('Error', 'OK', 0, null)) + + } + +} From 713e33e607af8eeeed29df2c8a9666412ed64d08 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 14:54:14 -0500 Subject: [PATCH 04/97] Document array executor Signed-off-by: Ben Sherman --- docs/executor.rst | 58 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/docs/executor.rst b/docs/executor.rst index d015bced4b..b3047d8300 100644 --- a/docs/executor.rst +++ b/docs/executor.rst @@ -14,6 +14,64 @@ In other words, you can write your pipeline script once and have it running on y or the cloud — simply change the executor definition in the Nextflow configuration file. +.. _array-executor: + +Array +===== + +.. note:: This feature requires Nextflow version ``23.05.0-edge`` or later. + +.. warning:: This feature is experimental and may change in a future release. + +Many execution platforms support "array jobs", that is, a collection of jobs with the same resource requirements and script +parameterized by an index. An array job incurs significantly less scheduling overhead compared to submitting each task separately, +and they are a best practice in HPC environments. Nextflow supports array jobs through the Array executor. + +The Array executor takes two config options: an array size (default: ``100``) and a "target" executor (default: ``'local'``). +These options can be specified in the Nextflow configuration as follows:: + + process { + executor = 'array' + } + + executor { + $array { + arraySize = 100 + target = 'slurm' + } + } + +The target executor can be any Nextflow executor that supports array jobs, which currently includes the following: + +* :ref:`local-executor` +* :ref:`slurm-executor` + +The Array executor submits tasks in batches to the target executor, as soon as a full batch is ready. Batches are separated +by process. Any "leftover" tasks are submitted as a partial batch at the end. If any tasks in an array job fail and can be retried, +they will be retried in another array job without interfering with the tasks that succeeded. Because the order in which tasks are +executed varies across workflow runs, so too will the grouping of tasks into array jobs. + +Aside from the batching, tasks in an array job are executed in the same way, i.e. each task is executed in its own work directory with +its own script. However, certain configuration properties must be the same for all tasks created by a process. + +The following process directives msut be the same for all tasks when using array jobs, because they are specified once for the array job: + +* :ref:`process-accelerator` +* :ref:`process-clusterOptions` +* :ref:`process-cpus` +* :ref:`process-disk` +* :ref:`process-machineType` +* :ref:`process-memory` +* :ref:`process-queue` +* :ref:`process-resourcelabels` +* :ref:`process-time` + +For cloud-based executors like AWS Batch, the following additional directives must be uniform: + +* :ref:`process-container` +* :ref:`process-containerOptions` + + .. _awsbatch-executor: AWS Batch From 76de0d447b90eaf635f1fc66afa08687f46f17ea Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 14:55:30 -0500 Subject: [PATCH 05/97] Use concurrent queues in array executor, add fallback for leftover tasks Signed-off-by: Ben Sherman --- .../nextflow/executor/ArrayExecutor.groovy | 49 ++++++++++++++----- 1 file changed, 36 insertions(+), 13 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy index e4b43e5abd..29e1436dbf 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy @@ -17,6 +17,7 @@ package nextflow.executor import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.ConcurrentLinkedQueue import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -42,7 +43,9 @@ class ArrayExecutor extends Executor implements TraceObserver { private Integer arraySize - private Map> queues = new ConcurrentHashMap<>() + private Map> queues = new ConcurrentHashMap<>() + + private Map closed = new ConcurrentHashMap<>() /** * Initialize the executor class @@ -84,21 +87,39 @@ class ArrayExecutor extends Executor implements TraceObserver { if( session.isTerminated() ) new IllegalStateException("Session terminated - Cannot add process to execution array: ${task}") - // add task to the corresponding process queue final process = task.processor.name + + // submit task directly if process has already closed + if( closed[process] ) { + ((Executor)target).submit(task) + return + } + + // initialize process queue if( process !in queues ) - queues[process] = [] + queues[process] = new ConcurrentLinkedQueue<>() - final array = queues[process] - array << task + // add task to the process queue + final queue = queues[process] + queue.add(task) - // schedule array job when the process queue is full - if( array.size() == arraySize ) { + // schedule array job when a batch is ready + if( queue.size() >= arraySize ) { log.debug "[ARRAY] Submitting array job for process '${process}'" + submit0(queue, arraySize) + } + } - monitor.schedule(target.createArrayTaskHandler(array)) - queues[process] = [] + synchronized private void submit0( Queue queue, int size ) { + def array = new ArrayList() + def iter = queue.iterator() + + for( int i : 1..size ) { + array << iter.next() + iter.remove() } + + monitor.schedule(target.createArrayTaskHandler(array)) } @Override @@ -113,12 +134,14 @@ class ArrayExecutor extends Executor implements TraceObserver { */ @Override void onProcessClose(String process) { - final array = queues[process] + final queue = queues[process] - if( array != null && array.size() > 0 ) { + if( queue != null && queue.size() > 0 ) { log.debug "[ARRAY] Submitting remainder array job for process '${process}'" - - monitor.schedule(target.createArrayTaskHandler(array)) + submit0(queue, queue.size()) } + + closed[process] = true } + } From 1ef005cde0c948776be57e7268f4663aeed4db3e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 16:38:28 -0500 Subject: [PATCH 06/97] Move batching logic to array task polling monitor Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 7 +- .../nextflow/executor/ArrayExecutor.groovy | 109 ++--------------- .../nextflow/executor/ArrayTaskAware.groovy | 8 +- .../groovy/nextflow/executor/Executor.groovy | 4 +- .../executor/local/LocalExecutor.groovy | 8 -- .../processor/ArrayTaskPollingMonitor.groovy | 112 ++++++++++++++++-- .../executor/ArrayExecutorTest.groovy | 2 +- 7 files changed, 120 insertions(+), 130 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 6a958f90e1..8512adc6ef 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -66,7 +66,7 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { /* * Prepare and launch the task in the underlying execution platform */ - TaskHandler createTaskHandler(TaskRun task) { + GridTaskHandler createTaskHandler(TaskRun task) { assert task assert task.workDir @@ -410,9 +410,8 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { } @Override - ArrayTaskHandler createArrayTaskHandler(List array) { - final handlers = array.collect { task -> createTaskHandler(task) } - new ArrayGridTaskHandler(handlers, this) + ArrayTaskHandler createArrayTaskHandler(List array) { + new ArrayGridTaskHandler(array, this) } String createArrayTaskWrapper(ArrayGridTaskHandler handler) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy index 29e1436dbf..f4dd10707e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy @@ -16,19 +16,13 @@ package nextflow.executor -import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.ConcurrentLinkedQueue - import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.processor.ArrayTaskPollingMonitor import nextflow.processor.TaskHandler import nextflow.processor.TaskMonitor -import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun -import nextflow.trace.TraceObserver import nextflow.util.Duration -import org.codehaus.groovy.runtime.typehandling.GroovyCastException /** * Executor that submits tasks in batches to a target executor * that supports array jobs. @@ -37,111 +31,24 @@ import org.codehaus.groovy.runtime.typehandling.GroovyCastException */ @Slf4j @CompileStatic -class ArrayExecutor extends Executor implements TraceObserver { - - private ArrayTaskAware target - - private Integer arraySize - - private Map> queues = new ConcurrentHashMap<>() - - private Map closed = new ConcurrentHashMap<>() - - /** - * Initialize the executor class - */ - @Override - protected void register() { - super.register() - - session.registerObserver(this) +class ArrayExecutor extends Executor { - final targetName = session.getExecConfigProp('array', 'target', 'local') as String - try { - target = (ArrayTaskAware)session.executorFactory.getExecutor(targetName, session) - } - catch( GroovyCastException e ) { - throw new IllegalArgumentException("Executor '${targetName}' does not support array jobs") - } - - arraySize = session.getExecConfigProp('array', 'arraySize', 100) as Integer - - log.debug "Creating 'array' executor > target executor: '${targetName}', array size: ${arraySize}" - } + private Executor target @Override TaskMonitor createTaskMonitor() { - return ArrayTaskPollingMonitor.create(session, name, 100, Duration.of('5 sec')) - } - - /** - * Add submitted tasks to the queue, and schedule an array job when - * the queue reaches the desired size. - * - * @param task - */ - @Override - synchronized void submit( TaskRun task ) { - log.trace "Scheduling process: ${task}" - - if( session.isTerminated() ) - new IllegalStateException("Session terminated - Cannot add process to execution array: ${task}") - - final process = task.processor.name - - // submit task directly if process has already closed - if( closed[process] ) { - ((Executor)target).submit(task) - return - } - - // initialize process queue - if( process !in queues ) - queues[process] = new ConcurrentLinkedQueue<>() - - // add task to the process queue - final queue = queues[process] - queue.add(task) - - // schedule array job when a batch is ready - if( queue.size() >= arraySize ) { - log.debug "[ARRAY] Submitting array job for process '${process}'" - submit0(queue, arraySize) - } - } - - synchronized private void submit0( Queue queue, int size ) { - def array = new ArrayList() - def iter = queue.iterator() + final targetName = session.getExecConfigProp('array', 'target', 'local') as String + target = session.executorFactory.getExecutor(targetName, session) - for( int i : 1..size ) { - array << iter.next() - iter.remove() - } + if( target !instanceof ArrayTaskAware ) + throw new IllegalArgumentException("Executor '${targetName}' does not support array jobs") - monitor.schedule(target.createArrayTaskHandler(array)) + return ArrayTaskPollingMonitor.create(session, target, 100, Duration.of('5 sec'), 100) } @Override TaskHandler createTaskHandler(TaskRun task) { - throw new UnsupportedOperationException() - } - - /** - * Submit any remaining tasks as a partial batch when a process is closed. - * - * @param process - */ - @Override - void onProcessClose(String process) { - final queue = queues[process] - - if( queue != null && queue.size() > 0 ) { - log.debug "[ARRAY] Submitting remainder array job for process '${process}'" - submit0(queue, queue.size()) - } - - closed[process] = true + target.createTaskHandler(task) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy index 82476c6ef4..fe63d39748 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy @@ -16,20 +16,22 @@ package nextflow.executor -import nextflow.processor.TaskRun +import nextflow.processor.TaskHandler /** * Interface for executors that support array jobs. * * @author Ben Sherman */ -interface ArrayTaskAware { +trait ArrayTaskAware { /** * Create a task handler for an array job. * * @param array */ - ArrayTaskHandler createArrayTaskHandler( List array ) + ArrayTaskHandler createArrayTaskHandler(List array) { + new ArrayTaskHandler(array) + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy index 632a066666..5222e13c1b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy @@ -49,7 +49,7 @@ abstract class Executor { /** * The queue holder that keep track of all tasks for this executor. */ - protected TaskMonitor monitor + private TaskMonitor monitor TaskMonitor getMonitor() { monitor } @@ -79,7 +79,7 @@ abstract class Executor { * * @param task A {@code TaskRun} instance */ - void submit( TaskRun task ) { + final void submit( TaskRun task ) { log.trace "Scheduling process: ${task}" if( session.isTerminated() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy index bb1a1b7650..d499a1d60d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy @@ -20,8 +20,6 @@ package nextflow.executor.local import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.executor.ArrayTaskAware -import nextflow.executor.ArrayTaskHandler -import nextflow.executor.Executor import nextflow.executor.Executor import nextflow.executor.SupportedScriptTypes import nextflow.fusion.FusionHelper @@ -65,11 +63,5 @@ class LocalExecutor extends Executor implements ArrayTaskAware { boolean isFusionEnabled() { return FusionHelper.isFusionEnabled(session) } - - @Override - ArrayTaskHandler createArrayTaskHandler(List array) { - final handlers = array.collect { task -> createTaskHandler(task) } - new ArrayTaskHandler(handlers) - } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy index 949097e3cf..b9c19f6315 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy @@ -16,38 +16,129 @@ package nextflow.processor +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.ConcurrentLinkedQueue + import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.Session +import nextflow.executor.ArrayTaskAware import nextflow.executor.ArrayTaskHandler +import nextflow.executor.Executor +import nextflow.trace.TraceObserver import nextflow.util.Duration /** - * Extension of the polling monitor for array jobs. + * Polling monitor that submits tasks in batches. * * @author Ben Sherman */ @Slf4j @CompileStatic -class ArrayTaskPollingMonitor extends TaskPollingMonitor { +class ArrayTaskPollingMonitor extends TaskPollingMonitor implements TraceObserver { + + private Executor executor + + private int arraySize + + private Map> arrayQueues = new ConcurrentHashMap<>() + + private Map closedProcesses = new ConcurrentHashMap<>() protected ArrayTaskPollingMonitor(Map params) { super(params) + this.executor = params.executor as Executor + this.arraySize = params.arraySize as int + + session.registerObserver(this) } - static ArrayTaskPollingMonitor create( Session session, String name, int defQueueSize, Duration defPollInterval ) { + static ArrayTaskPollingMonitor create( Session session, Executor executor, int defQueueSize, Duration defPollInterval, int defArraySize ) { + final name = 'array' final capacity = session.getQueueSize(name, defQueueSize) final pollInterval = session.getPollInterval(name, defPollInterval) final dumpInterval = session.getMonitorDumpInterval(name) + final arraySize = session.getExecConfigProp(name, 'arraySize', defArraySize) as Integer + + log.debug "Creating array task monitor for exector '$executor.name' > capacity: $capacity; pollInterval: $pollInterval; dumpInterval: $dumpInterval; arraySize: $arraySize" + + new ArrayTaskPollingMonitor( + name: name, + session: session, + executor: executor, + capacity: capacity, + pollInterval: pollInterval, + dumpInterval: dumpInterval, + arraySize: arraySize ) + } + + /** + * Add scheduled tasks to a queue, and schedule an array job when + * the queue reaches the desired size. + * + * @param handler + */ + @Override + synchronized void schedule(TaskHandler handler) { + final process = handler.task.processor.name + + // schedule task directly if process has already closed + if( process in closedProcesses ) { + executor.monitor.schedule(handler) + return + } + + // initialize array queue + if( process !in arrayQueues ) + arrayQueues[process] = new ConcurrentLinkedQueue<>() + + // add task to the array queue + final queue = arrayQueues[process] + queue.add(handler) + + // schedule array job when a batch is ready + if( queue.size() >= arraySize ) { + log.debug "Submitting array job for process '${process}'" + schedule0(queue, arraySize) + } + } + + /** + * Submit any remaining tasks as an array job when a process is closed. + * + * @param process + */ + @Override + void onProcessClose(String process) { + final queue = arrayQueues[process] - log.debug "Creating task monitor for executor '$name' > capacity: $capacity; pollInterval: $pollInterval; dumpInterval: $dumpInterval " - new ArrayTaskPollingMonitor(name: name, session: session, capacity: capacity, pollInterval: pollInterval, dumpInterval: dumpInterval) + if( queue != null && queue.size() > 0 ) { + log.debug "Submitting remainder array job for process '${process}'" + schedule0(queue, queue.size()) + } + + closedProcesses[process] = true } - protected ArrayTaskHandler toArrayHandler(TaskHandler handler) { - if( handler !instanceof ArrayTaskHandler ) - throw new IllegalStateException() + synchronized protected void schedule0( Queue queue, int size ) { + // remove tasks from the queue + def array = new ArrayList() + def iter = queue.iterator() + while( iter.hasNext() && array.size() < size ) { + array << iter.next() + iter.remove() + } + + if( array.size() < size ) + log.warn "Array job expected ${size} tasks but received only ${array.size()}" + + // create array task handler and schedule it + super.schedule(((ArrayTaskAware)executor).createArrayTaskHandler(array)) + } + + protected ArrayTaskHandler toArrayHandler(TaskHandler handler) { + assert handler instanceof ArrayTaskHandler return (ArrayTaskHandler)handler } @@ -63,7 +154,7 @@ class ArrayTaskPollingMonitor extends TaskPollingMonitor { handler.decProcessForks() // attempt to retry each task in the array - handler.array.each { h -> + for( TaskHandler h : handler.array ) { def fault0 = h.task.processor.resumeOrDie(h.task, error) if( fault == null && fault0 ) { fault = fault0 @@ -103,7 +194,7 @@ class ArrayTaskPollingMonitor extends TaskPollingMonitor { // attempt to finalize each task in the array def fault = null def faultHandler = null - handler.array.each { h -> + for( TaskHandler h : handler.array ) { final fault0 = h.task.processor.finalizeTask(h.task) if( fault == null && fault0 ) { fault = fault0 @@ -141,4 +232,3 @@ class ArrayTaskPollingMonitor extends TaskPollingMonitor { } } - diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy index 66b0e62b01..8817d69b86 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy @@ -35,7 +35,7 @@ class ArrayExecutorTest extends Specification { executor.session = session when: - executor.register() + executor.createTaskMonitor() then: session.getExecConfigProp('array', 'target', 'local') >> 'nope' executorFactory.getExecutor('nope', session) >> new NopeExecutor() From 27728ebcaca6cb8cbe087ac56abbf868db2e9342 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 21 Apr 2023 18:12:52 -0500 Subject: [PATCH 07/97] Cache status checks in array task handler Signed-off-by: Ben Sherman --- .../nextflow/executor/ArrayTaskHandler.groovy | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy index 8505d144e0..192e734fdf 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy @@ -50,20 +50,32 @@ class ArrayTaskHandler extends TaskHandler { status = TaskStatus.SUBMITTED } + protected Set runningCache = [] as Set + @Override boolean checkIfRunning() { for( TaskHandler handler : array ) - if( !handler.checkIfRunning() ) + if( handler in runningCache ) + continue + else if( handler.checkIfRunning() ) + runningCache << handler + else return false status = TaskStatus.RUNNING return true } + protected Set completedCache = [] as Set + @Override boolean checkIfCompleted() { for( TaskHandler handler : array ) - if( !handler.checkIfCompleted() ) + if( handler in completedCache ) + continue + else if( handler.checkIfCompleted() ) + completedCache << handler + else return false status = TaskStatus.COMPLETED From 343b331d4b1f65d0c63b8a8de2c3235992f2898a Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 24 Apr 2023 14:27:22 -0500 Subject: [PATCH 08/97] Finalize support for SLURM array jobs Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 6 ++++- .../executor/ArrayGridTaskHandler.groovy | 17 +++++++++++--- .../nextflow/executor/ArrayTaskHandler.groovy | 23 ++++++++++++++++--- .../nextflow/executor/BridgeExecutor.groovy | 2 +- .../nextflow/executor/CondorExecutor.groovy | 2 +- .../nextflow/executor/FluxExecutor.groovy | 2 +- .../nextflow/executor/GridTaskHandler.groovy | 12 ++++++++-- .../executor/HyperQueueExecutor.groovy | 2 +- .../nextflow/executor/LsfExecutor.groovy | 2 +- .../nextflow/executor/MoabExecutor.groovy | 2 +- .../nextflow/executor/NqsiiExecutor.groovy | 6 ++--- .../nextflow/executor/OarExecutor.groovy | 10 ++++---- .../nextflow/executor/PbsExecutor.groovy | 2 +- .../nextflow/executor/SgeExecutor.groovy | 2 +- .../nextflow/executor/SlurmExecutor.groovy | 7 ++++-- .../nextflow/processor/TaskHandler.groovy | 9 ++++++++ .../processor/TaskPollingMonitor.groovy | 1 + 17 files changed, 80 insertions(+), 27 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 8512adc6ef..36855c925f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -206,7 +206,7 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { /** * Given the string returned the by grid submit command, extract the process handle i.e. the grid jobId */ - abstract parseJobId( String text ); + abstract String parseJobId( String text ); /** * Kill a grid job @@ -445,5 +445,9 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { protected List getArraySubmitCommandLine() { throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") } + + protected String getArrayTaskId(String jobId, int index) { + throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy index 3fd24be38d..df505ab47d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy @@ -48,13 +48,17 @@ class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware try { // -- create the array job script final arrayScript = executor.createArrayTaskWrapper(this) + // -- create the submit command builder = createProcessBuilder() + // -- submit the array job with a retryable strategy final result = safeExecute( () -> processStart(builder, arrayScript) ) + // -- save the job id - this.jobId = executor.parseJobId(result) - this.status = TaskStatus.SUBMITTED + this.setJobId(executor.parseJobId(result)) + this.setStatus(TaskStatus.SUBMITTED) + log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" } catch( Exception e ) { @@ -70,11 +74,18 @@ class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware for( TaskHandler handler : array ) handler.task.script = builder ? CmdLineHelper.toLine(builder.command()) : null } - this.status = TaskStatus.COMPLETED + this.setStatus(TaskStatus.COMPLETED) throw new ProcessFailedException("Error submitting array job for execution", e) } } + protected void setJobId(String jobId) { + this.jobId = jobId + array.eachWithIndex { handler, i -> + ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) + } + } + protected ProcessBuilder createProcessBuilder() { // -- log the submit command diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy index 192e734fdf..27a3d182aa 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy @@ -17,6 +17,7 @@ package nextflow.executor +import java.nio.file.Path import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -42,12 +43,28 @@ class ArrayTaskHandler extends TaskHandler { this.array = array } + @Override + Path prepareLauncher() { + for( TaskHandler handler : array ) + handler.prepareLauncher() + + return null + } + @Override void submit() { for( TaskHandler handler : array ) handler.submit() - status = TaskStatus.SUBMITTED + setStatus(TaskStatus.SUBMITTED) + } + + @Override + void setStatus(TaskStatus status) { + super.setStatus(status) + + for( TaskHandler handler : array ) + handler.setStatus(status) } protected Set runningCache = [] as Set @@ -62,7 +79,7 @@ class ArrayTaskHandler extends TaskHandler { else return false - status = TaskStatus.RUNNING + setStatus(TaskStatus.RUNNING) return true } @@ -78,7 +95,7 @@ class ArrayTaskHandler extends TaskHandler { else return false - status = TaskStatus.COMPLETED + setStatus(TaskStatus.COMPLETED) return true } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy index 920f20afce..59896feef9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy @@ -106,7 +106,7 @@ class BridgeExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId(String text) { + String parseJobId(String text) { for( String line : text.readLines() ) { def m = SUBMIT_REGEX.matcher(line) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy index 03a161bb3f..9d83d0d842 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy @@ -93,7 +93,7 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - def parseJobId(String text) { + String parseJobId(String text) { text.tokenize(' -')[0] } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy index 37d7a4e1bd..b08246c59e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy @@ -109,7 +109,7 @@ class FluxExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId(String text) { + String parseJobId(String text) { // Parse the special "f" first for( String line : text.readLines() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index eac2fd6134..44bdad9ea3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -139,6 +139,12 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr } } + @Override + Path prepareLauncher() { + // -- create the wrapper script + createTaskWrapper(task).build() + } + protected BashWrapperBuilder createTaskWrapper(TaskRun task) { return fusionEnabled() ? fusionLauncher() @@ -185,8 +191,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr void submit() { ProcessBuilder builder = null try { - // -- create the wrapper script - createTaskWrapper(task).build() // -- start the execution and notify the event to the monitor builder = createProcessBuilder() // -- forward the job launcher script to the command stdin if required @@ -215,6 +219,10 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr } + void setJobId(jobId) { + this.jobId = jobId + } + private long startedMillis diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index 0805c16609..198ad334ee 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -86,7 +86,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - def parseJobId(String text) { + String parseJobId(String text) { try { text as Integer as String } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index a4216b59bb..9be44c9b45 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -136,7 +136,7 @@ class LsfExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId(String text) { + String parseJobId(String text) { def pattern = ~/Job <(\d+)> is submitted/ for( String line : text.readLines() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy index 21c47af000..2483d228a2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy @@ -99,7 +99,7 @@ class MoabExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId( String text ) { + String parseJobId( String text ) { String result try { result = new XmlSlurper() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy index 0a60e17485..83d5ccc0e6 100755 --- a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy @@ -98,9 +98,9 @@ class NqsiiExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId( String text ) { - def pattern = ~/Request (\d+).+ submitted to queue.+/ - for( String line : text.readLines() ) { + String parseJobId( String text ) { + def pattern = ~/Request (\d+).+ submitted to queue.+/ + for( String line : text.readLines() ) { def m = pattern.matcher(line) if( m.find() ) { return m.group(1) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy index e8a9820a3b..47e1cd9c48 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy @@ -113,11 +113,11 @@ class OarExecutor extends AbstractGridExecutor { static private Pattern SUBMIT_REGEX = ~/OAR_JOB_ID=(\d+)/ @Override - def parseJobId(String text) { - for( String line : text.readLines() ) { - def m = SUBMIT_REGEX.matcher(line) - if( m.matches() ) { - return m.group(1).toString() + String parseJobId(String text) { + for( String line : text.readLines() ) { + def m = SUBMIT_REGEX.matcher(line) + if( m.matches() ) { + return m.group(1).toString() } } throw new IllegalStateException("Invalid OAR submit response:\n$text\n\n") diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index abdea5f8eb..74e8b0efe3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -110,7 +110,7 @@ class PbsExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId( String text ) { + String parseJobId( String text ) { // return always the last line def result = text?.trim() if( result && !result.contains('\n')) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index fab1de5ed3..04721b9c89 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -103,7 +103,7 @@ class SgeExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId( String text ) { + String parseJobId( String text ) { // return always the last line String id def lines = text.trim().readLines() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 9f57f3ad34..bab793de10 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -115,7 +115,7 @@ class SlurmExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - def parseJobId(String text) { + String parseJobId(String text) { for( String line : text.readLines() ) { def m = SUBMIT_REGEX.matcher(line) @@ -209,5 +209,8 @@ class SlurmExecutor extends AbstractGridExecutor { protected String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } @Override - List getArraySubmitCommandLine() { List.of('sbatch') } + protected List getArraySubmitCommandLine() { List.of('sbatch') } + + @Override + protected String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index ca285afe71..39637eb0eb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -19,6 +19,7 @@ package nextflow.processor import static nextflow.processor.TaskStatus.* import java.nio.file.NoSuchFileException +import java.nio.file.Path import groovy.util.logging.Slf4j import nextflow.trace.TraceRecord @@ -85,6 +86,14 @@ abstract class TaskHandler { */ abstract void submit() + /** + * Prepare the launcher script. + * + * This method is optional. If it is not implemented, the launcher script should + * be prepared in the submit() method. + */ + Path prepareLauncher() { return null } + /** * Task status attribute setter. * diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 3ea4b9abca..0568aa89cb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -192,6 +192,7 @@ class TaskPollingMonitor implements TaskMonitor { */ protected void submit(TaskHandler handler) { // submit the job execution -- throws a ProcessException when submit operation fail + handler.prepareLauncher() handler.submit() // note: add the 'handler' into the polling queue *after* the submit operation, // this guarantees that in the queue are only jobs successfully submitted From e5ca280a139c8c11c6564b863045af3e432e3fb2 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 25 Apr 2023 15:38:13 -0500 Subject: [PATCH 09/97] Refactor array executor as process directive Signed-off-by: Ben Sherman --- docs/executor.md | 55 ---- docs/process.md | 58 +++++ .../src/main/groovy/nextflow/Session.groovy | 14 -- .../executor/AbstractGridExecutor.groovy | 24 +- .../nextflow/executor/ArrayExecutor.groovy | 54 ---- .../nextflow/executor/ArrayTaskAware.groovy | 6 +- .../nextflow/executor/ArrayTaskHandler.groovy | 152 ------------ .../executor/ArrayTaskSubmitter.groovy | 69 ++++++ .../nextflow/executor/BridgeExecutor.groovy | 2 +- .../nextflow/executor/CondorExecutor.groovy | 2 +- .../nextflow/executor/ExecutorFactory.groovy | 38 +-- .../nextflow/executor/FluxExecutor.groovy | 2 +- ...r.groovy => GridArrayTaskSubmitter.groovy} | 47 ++-- .../nextflow/executor/GridTaskHandler.groovy | 8 +- .../executor/HyperQueueExecutor.groovy | 2 +- .../nextflow/executor/LsfExecutor.groovy | 2 +- .../nextflow/executor/MoabExecutor.groovy | 2 +- .../nextflow/executor/NqsiiExecutor.groovy | 6 +- .../nextflow/executor/OarExecutor.groovy | 10 +- .../nextflow/executor/PbsExecutor.groovy | 2 +- .../nextflow/executor/SgeExecutor.groovy | 2 +- .../nextflow/executor/SlurmExecutor.groovy | 2 +- .../processor/ArrayTaskCollector.groovy | 112 +++++++++ .../processor/ArrayTaskPollingMonitor.groovy | 234 ------------------ .../nextflow/processor/TaskConfig.groovy | 6 +- .../nextflow/processor/TaskHandler.groovy | 10 +- .../processor/TaskPollingMonitor.groovy | 30 +-- .../nextflow/processor/TaskProcessor.groovy | 30 ++- .../nextflow/script/ProcessConfig.groovy | 20 ++ .../nextflow/trace/TraceObserver.groovy | 13 +- .../executor/ArrayExecutorTest.groovy | 46 ---- 31 files changed, 369 insertions(+), 691 deletions(-) delete mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy delete mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy rename modules/nextflow/src/main/groovy/nextflow/executor/{ArrayGridTaskHandler.groovy => GridArrayTaskSubmitter.groovy} (84%) create mode 100644 modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy delete mode 100644 modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy delete mode 100644 modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy diff --git a/docs/executor.md b/docs/executor.md index b6694000cf..b5cd49725f 100644 --- a/docs/executor.md +++ b/docs/executor.md @@ -8,61 +8,6 @@ The executor provides an abstraction between the pipeline processes and the unde In other words, you can write your pipeline script once and have it running on your computer, a cluster resource manager, or the cloud — simply change the executor definition in the Nextflow configuration file. -(array-executor)= - -## Array - -:::{note} -This feature requires Nextflow version `23.05.0-edge` or later. -::: - -:::{warning} -This feature is experimental and may change in a future release. -::: - -Many execution platforms support "array jobs", that is, a collection of jobs with the same resource requirements and script parameterized by an index. An array job incurs significantly less scheduling overhead compared to submitting each task separately, and they are a best practice in HPC environments. Nextflow supports array jobs through the Array executor. - -The Array executor takes two config options: an array size (default: `100`) and a "target" executor (default: `'local'`). These options can be specified in the Nextflow configuration as follows:: - -```groovy -process { - executor = 'array' -} - -executor { - $array { - arraySize = 100 - target = 'slurm' - } -} -``` - -The target executor can be any Nextflow executor that supports array jobs, which currently includes the following: - -- {ref}`local-executor` -- {ref}`slurm-executor` - -The Array executor submits tasks in batches to the target executor, as soon as a full batch is ready. Batches are separated by process. Any "leftover" tasks are submitted as a partial batch at the end. If any tasks in an array job fail and can be retried, they will be retried in another array job without interfering with the tasks that succeeded. Because the order in which tasks are executed varies across workflow runs, so too will the grouping of tasks into array jobs. - -Aside from the batching, tasks in an array job are executed in the same way, i.e. each task is executed in its own work directory with its own script. However, certain configuration properties must be the same for all tasks created by a process. - -The following process directives msut be the same for all tasks when using array jobs, because they are specified once for the array job: - -- {ref}`process-accelerator` -- {ref}`process-clusterOptions` -- {ref}`process-cpus` -- {ref}`process-disk` -- {ref}`process-machineType` -- {ref}`process-memory` -- {ref}`process-queue` -- {ref}`process-resourcelabels` -- {ref}`process-time` - -For cloud-based executors like AWS Batch, the following additional directives must be uniform: - -- {ref}`process-container` -- {ref}`process-containerOptions` - (awsbatch-executor)= ## AWS Batch diff --git a/docs/process.md b/docs/process.md index fd18f273f1..a8b52c861f 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1245,6 +1245,64 @@ Allowed values for the `arch` directive are as follows, grouped by equivalent me Examples of values for the architecture `target` option are `cascadelake`, `icelake`, `zen2` and `zen3`. See the Spack documentation for the full and up-to-date [list of meaningful targets](https://spack.readthedocs.io/en/latest/basic_usage.html#support-for-specific-microarchitectures). +(array-executor)= + +## array + +:::{note} +This feature requires Nextflow version `23.05.0-edge` or later. +::: + +:::{warning} +This feature is experimental and may change in a future release. +::: + +The `array` directive allows you to submit tasks as *array jobs* for executors that support it. + +An array job is a collection of jobs with the same resource requirements and the same script (parameterized by an index). Array jobs incur significantly less scheduling overhead compared to individual jobs, and as a result they are preferred by HPC schedulers where possible. + +It should be specified with a given array size, as well as an executor that supports array jobs. For example: + +```groovy +process cpu_task { + executor 'slurm' + array 100 + + ''' + your_command --here + ''' +} +``` + +Nextflow currently supports array jobs for the following executors: + +- {ref}`local-executor` +- {ref}`slurm-executor` + +A process using array jobs will collect tasks and submit each batch as an array job when it is ready. Any "leftover" tasks will be submitted as a partial array job. + +Once an array job is submitted, the "child" tasks are executed in the same way: + +- Each task is executed in its own work directory with its own script. +- Any tasks that fail (and can be retried) will be retried in another array job without interfering with the tasks that succeeded. + +The following directives msut be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: + +- {ref}`process-accelerator` +- {ref}`process-clusterOptions` +- {ref}`process-cpus` +- {ref}`process-disk` +- {ref}`process-machineType` +- {ref}`process-memory` +- {ref}`process-queue` +- {ref}`process-resourcelabels` +- {ref}`process-time` + +For cloud-based executors like AWS Batch, the following additional directives must be uniform: + +- {ref}`process-container` +- {ref}`process-containerOptions` + (process-beforescript)= ### beforeScript diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index c400e66a19..5e8550f15f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -424,9 +424,6 @@ class Session implements ISession { return result } - void registerObserver(TraceObserver observer) { - observers << observer - } /* * intercepts interruption signal i.e. CTRL+C @@ -938,17 +935,6 @@ class Session implements ISession { } } - void notifyProcessClose(String process) { - observers.each { observer -> - try { - observer.onProcessClose(process) - } - catch( Exception e ) { - log.debug(e.getMessage(), e) - } - } - } - void notifyProcessTerminate(TaskProcessor process) { for( int i=0; i array) { - new ArrayGridTaskHandler(array, this) + ArrayTaskSubmitter createArrayTaskSubmitter(List array) { + new GridArrayTaskSubmitter(array, this) } - String createArrayTaskWrapper(ArrayGridTaskHandler handler) { - final array = handler.array - final task = array.first().getTask() - - final arrayHeader = getArrayDirective(array.size()) - final taskHeaders = getHeaders(task) - final files = array - .collect { h -> ((GridTaskHandler)h).wrapperFile } + String createArrayTaskWrapper(GridArrayTaskSubmitter arraySubmitter) { + final array = arraySubmitter.getArray() + final arrayDirective = getArrayDirective(array.size()) + final taskHeaders = getHeaders(array.first().getTask()) + final wrapperFiles = array + .collect { handler -> ((GridTaskHandler)handler).wrapperFile } .join(' ') final builder = new StringBuilder() << '#!/bin/bash\n' - << "${headerToken} ${arrayHeader}\n" + << "${headerToken} ${arrayDirective}\n" << taskHeaders - << "declare -a array=( ${files} )\n" + << "declare -a array=( ${wrapperFiles} )\n" << "bash \${array[\$${arrayIndexName}]}\n" return builder.toString() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy deleted file mode 100644 index f4dd10707e..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayExecutor.groovy +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.executor - -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.processor.ArrayTaskPollingMonitor -import nextflow.processor.TaskHandler -import nextflow.processor.TaskMonitor -import nextflow.processor.TaskRun -import nextflow.util.Duration -/** - * Executor that submits tasks in batches to a target executor - * that supports array jobs. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class ArrayExecutor extends Executor { - - private Executor target - - @Override - TaskMonitor createTaskMonitor() { - final targetName = session.getExecConfigProp('array', 'target', 'local') as String - target = session.executorFactory.getExecutor(targetName, session) - - if( target !instanceof ArrayTaskAware ) - throw new IllegalArgumentException("Executor '${targetName}' does not support array jobs") - - return ArrayTaskPollingMonitor.create(session, target, 100, Duration.of('5 sec'), 100) - } - - @Override - TaskHandler createTaskHandler(TaskRun task) { - target.createTaskHandler(task) - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy index fe63d39748..5cc7d5d4bc 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy @@ -26,12 +26,12 @@ import nextflow.processor.TaskHandler trait ArrayTaskAware { /** - * Create a task handler for an array job. + * Create a submitter for an array job. * * @param array */ - ArrayTaskHandler createArrayTaskHandler(List array) { - new ArrayTaskHandler(array) + ArrayTaskSubmitter createArrayTaskSubmitter(List array) { + new ArrayTaskSubmitter(array) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy deleted file mode 100644 index 27a3d182aa..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskHandler.groovy +++ /dev/null @@ -1,152 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package nextflow.executor - -import java.nio.file.Path - -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.processor.TaskHandler -import nextflow.processor.TaskStatus -import nextflow.trace.TraceRecord -/** - * Generic handler for an array task, which simply launches - * each task and waits for it to finish. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class ArrayTaskHandler extends TaskHandler { - - final List array - - ArrayTaskHandler(List array) { - // use the first task to provide common properties (e.g. configuration) - super(array.first().getTask()) - - this.array = array - } - - @Override - Path prepareLauncher() { - for( TaskHandler handler : array ) - handler.prepareLauncher() - - return null - } - - @Override - void submit() { - for( TaskHandler handler : array ) - handler.submit() - - setStatus(TaskStatus.SUBMITTED) - } - - @Override - void setStatus(TaskStatus status) { - super.setStatus(status) - - for( TaskHandler handler : array ) - handler.setStatus(status) - } - - protected Set runningCache = [] as Set - - @Override - boolean checkIfRunning() { - for( TaskHandler handler : array ) - if( handler in runningCache ) - continue - else if( handler.checkIfRunning() ) - runningCache << handler - else - return false - - setStatus(TaskStatus.RUNNING) - return true - } - - protected Set completedCache = [] as Set - - @Override - boolean checkIfCompleted() { - for( TaskHandler handler : array ) - if( handler in completedCache ) - continue - else if( handler.checkIfCompleted() ) - completedCache << handler - else - return false - - setStatus(TaskStatus.COMPLETED) - return true - } - - @Override - void kill() { - for( TaskHandler handler : array ) - handler.kill() - } - - @Override - protected StringBuilder toStringBuilder( StringBuilder builder ) { - builder << '\n' - for( TaskHandler handler : array ) - builder << ' ' << handler.toString() << '\n' - - return builder - } - - @Override - String getStatusString() { - if( array.any { h -> h.task.failed } ) return 'FAILED' - if( array.any { h -> h.task.aborted } ) return 'ABORTED' - return this.status.toString() - } - - @Override - TraceRecord safeTraceRecord() { - throw new UnsupportedOperationException() - } - - @Override - TraceRecord getTraceRecord() { - throw new UnsupportedOperationException() - } - - @Override - boolean canForkProcess() { - final max = task.processor.maxForks - return !max - ? true - : task.processor.forksCount + array.size() <= max - } - - @Override - void incProcessForks() { - task.processor.forksCount?.add(array.size()) - } - - @Override - void decProcessForks() { - task.processor.forksCount?.add(-array.size()) - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy new file mode 100644 index 0000000000..5aeb2e5b78 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy @@ -0,0 +1,69 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.executor + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.processor.TaskHandler + +/** + * Submit tasks as an array job. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class ArrayTaskSubmitter { + + protected List array + + private volatile int collected = 0 + + ArrayTaskSubmitter(List array) { + this.array = array + + for( TaskHandler handler : array ) + handler.arraySubmitter = this + } + + List getArray() { array } + + /** + * Mark a task as ready to be submitted. + * + * @param handler + */ + synchronized void collect(TaskHandler handler) { + collected += 1 + + if( collected == array.size() ) + submit() + } + + /** + * Submit the array job. + * + * By default, this method simply submits each task individually. + * It should be overridden to submit an array job to the underlying + * executor. + */ + protected void submit() { + for( TaskHandler handler : array ) + handler.submit() + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy index 59896feef9..920f20afce 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy @@ -106,7 +106,7 @@ class BridgeExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId(String text) { + def parseJobId(String text) { for( String line : text.readLines() ) { def m = SUBMIT_REGEX.matcher(line) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy index 9d83d0d842..03a161bb3f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy @@ -93,7 +93,7 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - String parseJobId(String text) { + def parseJobId(String text) { text.tokenize(' -')[0] } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy index c6b00f9241..6b72deab11 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ExecutorFactory.groovy @@ -43,26 +43,25 @@ class ExecutorFactory { * Map the executor class to its 'friendly' name */ final static Map> BUILT_IN_EXECUTORS = [ - 'array': ArrayExecutor, + 'nope': NopeExecutor, + 'local': LocalExecutor, + 'flux': FluxExecutor, + 'sge': SgeExecutor, + 'oge': SgeExecutor, + 'uge': SgeExecutor, + 'lsf': LsfExecutor, + 'pbs': PbsExecutor, + 'pbspro': PbsProExecutor, + 'slurm': SlurmExecutor, 'bridge': BridgeExecutor, + 'crg': CrgExecutor, 'bsc': LsfExecutor, 'condor': CondorExecutor, - 'crg': CrgExecutor, - 'flux': FluxExecutor, - 'hq': HyperQueueExecutor, 'k8s': K8sExecutor, - 'local': LocalExecutor, - 'lsf': LsfExecutor, - 'moab': MoabExecutor, - 'nope': NopeExecutor, 'nqsii': NqsiiExecutor, + 'moab': MoabExecutor, 'oar': OarExecutor, - 'oge': SgeExecutor, - 'pbs': PbsExecutor, - 'pbspro': PbsProExecutor, - 'sge': SgeExecutor, - 'slurm': SlurmExecutor, - 'uge': SgeExecutor + 'hq': HyperQueueExecutor ] @PackageScope Map> executorsMap @@ -194,17 +193,6 @@ class ExecutorFactory { clazz = LocalExecutor.class } - return getExecutor0(name, clazz, session) - } - - Executor getExecutor(String name, Session session) { - // -- load the executor to be used - def clazz = getExecutorClass(name) - - return getExecutor0(name, clazz, session) - } - - protected Executor getExecutor0(String name, Class clazz, Session session) { // this code is not supposed to be executed parallel def result = executors.get(clazz) if( result ) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy index b08246c59e..37d7a4e1bd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy @@ -109,7 +109,7 @@ class FluxExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId(String text) { + def parseJobId(String text) { // Parse the special "f" first for( String line : text.readLines() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy similarity index 84% rename from modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy index df505ab47d..fa17d54250 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayGridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy @@ -21,29 +21,33 @@ import groovy.util.logging.Slf4j import nextflow.exception.ProcessFailedException import nextflow.exception.ProcessNonZeroExitStatusException import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun import nextflow.processor.TaskStatus import nextflow.util.CmdLineHelper /** - * Handles the execution of an array job for any grid executor. + * Submit tasks as an array job for a grid executor. * * @author Ben Sherman */ @Slf4j @CompileStatic -class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware { +class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAware { - final AbstractGridExecutor executor + private AbstractGridExecutor executor - private jobId - - ArrayGridTaskHandler(List array, AbstractGridExecutor executor) { + GridArrayTaskSubmitter(List array, AbstractGridExecutor executor) { super(array) - this.executor = executor } @Override - void submit() { + Executor getExecutor() { executor } + + @Override + TaskRun getTask() { array.first().getTask() } + + @Override + protected void submit() { ProcessBuilder builder = null try { // -- create the array job script @@ -54,9 +58,10 @@ class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware // -- submit the array job with a retryable strategy final result = safeExecute( () -> processStart(builder, arrayScript) ) + final jobId = (String)executor.parseJobId(result) - // -- save the job id - this.setJobId(executor.parseJobId(result)) + // -- set the job id and status of each task + this.setJobId(jobId) this.setStatus(TaskStatus.SUBMITTED) log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" @@ -78,19 +83,23 @@ class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware throw new ProcessFailedException("Error submitting array job for execution", e) } } - + protected void setJobId(String jobId) { - this.jobId = jobId array.eachWithIndex { handler, i -> ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) } } + protected void setStatus(TaskStatus status) { + for( TaskHandler handler : array ) + handler.setStatus(status) + } + protected ProcessBuilder createProcessBuilder() { // -- log the submit command final cli = executor.getArraySubmitCommandLine() - log.trace "submit array job > cli: ${cli}" + log.trace "[${executor.name.toUpperCase()}] Submit array job > cli: ${cli}" // -- launch array job script new ProcessBuilder() @@ -137,16 +146,4 @@ class ArrayGridTaskHandler extends ArrayTaskHandler implements SubmitRetryAware .toString() } - @Override - void kill() { - executor.killTask(jobId) - } - - @Override - protected StringBuilder toStringBuilder(StringBuilder builder) { - builder << "\n array jobId: $jobId; " - - return super.toStringBuilder(builder) - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 44bdad9ea3..77ad2563aa 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -189,6 +189,11 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr */ @Override void submit() { + if( arraySubmitter ) { + arraySubmitter.collect(this) + return + } + ProcessBuilder builder = null try { // -- start the execution and notify the event to the monitor @@ -219,11 +224,10 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr } - void setJobId(jobId) { + void setJobId(String jobId) { this.jobId = jobId } - private long startedMillis private long exitTimestampMillis0 = System.currentTimeMillis() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index 198ad334ee..0805c16609 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -86,7 +86,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - String parseJobId(String text) { + def parseJobId(String text) { try { text as Integer as String } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 9be44c9b45..a4216b59bb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -136,7 +136,7 @@ class LsfExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId(String text) { + def parseJobId(String text) { def pattern = ~/Job <(\d+)> is submitted/ for( String line : text.readLines() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy index 2483d228a2..21c47af000 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy @@ -99,7 +99,7 @@ class MoabExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId( String text ) { + def parseJobId( String text ) { String result try { result = new XmlSlurper() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy index 83d5ccc0e6..0a60e17485 100755 --- a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy @@ -98,9 +98,9 @@ class NqsiiExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId( String text ) { - def pattern = ~/Request (\d+).+ submitted to queue.+/ - for( String line : text.readLines() ) { + def parseJobId( String text ) { + def pattern = ~/Request (\d+).+ submitted to queue.+/ + for( String line : text.readLines() ) { def m = pattern.matcher(line) if( m.find() ) { return m.group(1) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy index 47e1cd9c48..e8a9820a3b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy @@ -113,11 +113,11 @@ class OarExecutor extends AbstractGridExecutor { static private Pattern SUBMIT_REGEX = ~/OAR_JOB_ID=(\d+)/ @Override - String parseJobId(String text) { - for( String line : text.readLines() ) { - def m = SUBMIT_REGEX.matcher(line) - if( m.matches() ) { - return m.group(1).toString() + def parseJobId(String text) { + for( String line : text.readLines() ) { + def m = SUBMIT_REGEX.matcher(line) + if( m.matches() ) { + return m.group(1).toString() } } throw new IllegalStateException("Invalid OAR submit response:\n$text\n\n") diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 74e8b0efe3..abdea5f8eb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -110,7 +110,7 @@ class PbsExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId( String text ) { + def parseJobId( String text ) { // return always the last line def result = text?.trim() if( result && !result.contains('\n')) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 04721b9c89..fab1de5ed3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -103,7 +103,7 @@ class SgeExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId( String text ) { + def parseJobId( String text ) { // return always the last line String id def lines = text.trim().readLines() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index bab793de10..24250d4de0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -115,7 +115,7 @@ class SlurmExecutor extends AbstractGridExecutor { * @return The actual job ID string */ @Override - String parseJobId(String text) { + def parseJobId(String text) { for( String line : text.readLines() ) { def m = SUBMIT_REGEX.matcher(line) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy new file mode 100644 index 0000000000..9e4c9c8c93 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy @@ -0,0 +1,112 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.processor + +import java.util.concurrent.ConcurrentLinkedQueue + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.executor.ArrayTaskAware +import nextflow.executor.Executor + +/** + * Task monitor that batches tasks and submits them as array jobs + * to an underlying task monitor. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class ArrayTaskCollector { + + private Executor executor + + private TaskMonitor monitor + + private int arraySize + + private volatile Queue queue = new ConcurrentLinkedQueue<>() + + private volatile boolean closed = false + + ArrayTaskCollector(Executor executor, int arraySize) { + if( executor !instanceof ArrayTaskAware ) + throw new IllegalArgumentException("Executor '${executor.name}' does not support array jobs") + + this.executor = executor + this.monitor = executor.monitor + this.arraySize = arraySize + } + + /** + * Add a task to the queue, and submit an array job when the + * queue reaches the desired size. + * + * @param handler + */ + synchronized void submit(TaskRun task) { + // submit task directly if the collector is closed + if( closed ) { + executor.submit(task) + return + } + + // create task handler + final handler = executor.createTaskHandler(task) + + // add task to the array queue + queue.add(handler) + + // submit array job when a batch is ready + if( queue.size() >= arraySize ) + submit0(queue, arraySize) + } + + /** + * Close the collector, submitting any remaining tasks as a partial array job. + * + * @param process + */ + void close() { + if( queue.size() > 0 ) + submit0(queue, queue.size()) + + closed = true + } + + synchronized protected void submit0( Queue queue, int size ) { + // remove tasks from the queue + def array = new ArrayList() + def iter = queue.iterator() + + while( iter.hasNext() && array.size() < size ) { + array << iter.next() + iter.remove() + } + + if( array.size() < size ) + log.warn "Array job expected ${size} tasks but received only ${array.size()}" + + // create submitter for array job + ((ArrayTaskAware)executor).createArrayTaskSubmitter(array) + + // submit each task to the underlying monitor + for( TaskHandler handler : array ) + monitor.schedule(handler) + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy deleted file mode 100644 index b9c19f6315..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskPollingMonitor.groovy +++ /dev/null @@ -1,234 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.processor - -import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.ConcurrentLinkedQueue - -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.Session -import nextflow.executor.ArrayTaskAware -import nextflow.executor.ArrayTaskHandler -import nextflow.executor.Executor -import nextflow.trace.TraceObserver -import nextflow.util.Duration - -/** - * Polling monitor that submits tasks in batches. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class ArrayTaskPollingMonitor extends TaskPollingMonitor implements TraceObserver { - - private Executor executor - - private int arraySize - - private Map> arrayQueues = new ConcurrentHashMap<>() - - private Map closedProcesses = new ConcurrentHashMap<>() - - protected ArrayTaskPollingMonitor(Map params) { - super(params) - this.executor = params.executor as Executor - this.arraySize = params.arraySize as int - - session.registerObserver(this) - } - - static ArrayTaskPollingMonitor create( Session session, Executor executor, int defQueueSize, Duration defPollInterval, int defArraySize ) { - final name = 'array' - final capacity = session.getQueueSize(name, defQueueSize) - final pollInterval = session.getPollInterval(name, defPollInterval) - final dumpInterval = session.getMonitorDumpInterval(name) - final arraySize = session.getExecConfigProp(name, 'arraySize', defArraySize) as Integer - - log.debug "Creating array task monitor for exector '$executor.name' > capacity: $capacity; pollInterval: $pollInterval; dumpInterval: $dumpInterval; arraySize: $arraySize" - - new ArrayTaskPollingMonitor( - name: name, - session: session, - executor: executor, - capacity: capacity, - pollInterval: pollInterval, - dumpInterval: dumpInterval, - arraySize: arraySize ) - } - - /** - * Add scheduled tasks to a queue, and schedule an array job when - * the queue reaches the desired size. - * - * @param handler - */ - @Override - synchronized void schedule(TaskHandler handler) { - final process = handler.task.processor.name - - // schedule task directly if process has already closed - if( process in closedProcesses ) { - executor.monitor.schedule(handler) - return - } - - // initialize array queue - if( process !in arrayQueues ) - arrayQueues[process] = new ConcurrentLinkedQueue<>() - - // add task to the array queue - final queue = arrayQueues[process] - queue.add(handler) - - // schedule array job when a batch is ready - if( queue.size() >= arraySize ) { - log.debug "Submitting array job for process '${process}'" - schedule0(queue, arraySize) - } - } - - /** - * Submit any remaining tasks as an array job when a process is closed. - * - * @param process - */ - @Override - void onProcessClose(String process) { - final queue = arrayQueues[process] - - if( queue != null && queue.size() > 0 ) { - log.debug "Submitting remainder array job for process '${process}'" - schedule0(queue, queue.size()) - } - - closedProcesses[process] = true - } - - synchronized protected void schedule0( Queue queue, int size ) { - // remove tasks from the queue - def array = new ArrayList() - def iter = queue.iterator() - - while( iter.hasNext() && array.size() < size ) { - array << iter.next() - iter.remove() - } - - if( array.size() < size ) - log.warn "Array job expected ${size} tasks but received only ${array.size()}" - - // create array task handler and schedule it - super.schedule(((ArrayTaskAware)executor).createArrayTaskHandler(array)) - } - - protected ArrayTaskHandler toArrayHandler(TaskHandler handler) { - assert handler instanceof ArrayTaskHandler - return (ArrayTaskHandler)handler - } - - @Override - protected void handleException(TaskHandler handler, Throwable error) { - handler = toArrayHandler(handler) - - def fault = null - def faultHandler = null - try { - // remove the array task from the processing queue - if( evict(handler) ) - handler.decProcessForks() - - // attempt to retry each task in the array - for( TaskHandler h : handler.array ) { - def fault0 = h.task.processor.resumeOrDie(h.task, error) - if( fault == null && fault0 ) { - fault = fault0 - faultHandler = h - } - } - - log.trace "Task fault (1): $fault" - } - finally { - // abort the session if a task fault was returned - if( fault instanceof TaskFault ) - session.fault(fault, handler) - } - } - - @Override - protected void checkTaskStatus( TaskHandler handler ) { - handler = toArrayHandler(handler) - - // check if the array task is started - if( handler.checkIfRunning() ) { - log.trace "Task started > $handler" - notifyTaskStart(handler) - } - - // check if the array task is completed - if( handler.checkIfCompleted() ) { - log.debug "Task completed > $handler" - - // decrement forks count - handler.decProcessForks() - - // remove the array task from the processing queue - evict(handler) - - // attempt to finalize each task in the array - def fault = null - def faultHandler = null - for( TaskHandler h : handler.array ) { - final fault0 = h.task.processor.finalizeTask(h.task) - if( fault == null && fault0 ) { - fault = fault0 - faultHandler = h - } - } - - // notify task completion - notifyTaskComplete(handler) - - // abort the session if a task fault was returned - if( fault instanceof TaskFault ) - session.fault(fault, faultHandler) - } - } - - @Override - protected void notifyTaskSubmit(TaskHandler handler) { - toArrayHandler(handler).array.each { h -> session.notifyTaskSubmit(h) } - } - - @Override - protected void notifyTaskPending(TaskHandler handler) { - toArrayHandler(handler).array.each { h -> session.notifyTaskPending(h) } - } - - @Override - protected void notifyTaskStart(TaskHandler handler) { - toArrayHandler(handler).array.each { h -> session.notifyTaskStart(h) } - } - - @Override - protected void notifyTaskComplete(TaskHandler handler) { - toArrayHandler(handler).array.each { h -> session.notifyTaskComplete(h) } - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy index b5f2e56f70..981737dec2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy @@ -42,7 +42,7 @@ import nextflow.util.MemoryUnit @CompileStatic class TaskConfig extends LazyMap implements Cloneable { - static public final EXIT_ZERO = 0 + static public final int EXIT_ZERO = 0 private transient Map cache = new LinkedHashMap(20) @@ -185,6 +185,10 @@ class TaskConfig extends LazyMap implements Cloneable { return false } + int getArray() { + get('array') as Integer ?: 0 + } + String getBeforeScript() { return get('beforeScript') } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 39637eb0eb..413863f962 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -22,6 +22,7 @@ import java.nio.file.NoSuchFileException import java.nio.file.Path import groovy.util.logging.Slf4j +import nextflow.executor.ArrayTaskSubmitter import nextflow.trace.TraceRecord /** * Actions to handle the underlying job running the user task. @@ -52,6 +53,11 @@ abstract class TaskHandler { */ TaskRun getTask() { task } + /** + * Submitter for array jobs + */ + ArrayTaskSubmitter arraySubmitter + /** * Task current status */ @@ -240,14 +246,14 @@ abstract class TaskHandler { /** * Increment the number of current forked processes */ - void incProcessForks() { + final void incProcessForks() { task.processor.forksCount?.increment() } /** * Decrement the number of current forked processes */ - void decProcessForks() { + final void decProcessForks() { task.processor.forksCount?.decrement() } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 0568aa89cb..df03554422 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -198,7 +198,7 @@ class TaskPollingMonitor implements TaskMonitor { // this guarantees that in the queue are only jobs successfully submitted runningQueue.add(handler) // notify task submission - notifyTaskSubmit(handler) + session.notifyTaskSubmit(handler) } /** @@ -226,7 +226,7 @@ class TaskPollingMonitor implements TaskMonitor { try{ pendingQueue << handler taskAvail.signal() // signal that a new task is available for execution - notifyTaskPending(handler) + session.notifyTaskPending(handler) log.trace "Scheduled task > $handler" } finally { @@ -563,7 +563,7 @@ class TaskPollingMonitor implements TaskMonitor { } catch ( Throwable e ) { handleException(handler, e) - notifyTaskComplete(handler) + session.notifyTaskComplete(handler) } // remove processed handler either on successful submit or failed one (managed by catch section) // when `canSubmit` return false the handler should be retained to be tried in a following iteration @@ -574,7 +574,7 @@ class TaskPollingMonitor implements TaskMonitor { } - protected void handleException( TaskHandler handler, Throwable error ) { + final protected void handleException( TaskHandler handler, Throwable error ) { def fault = null try { if (evict(handler)) { @@ -604,7 +604,7 @@ class TaskPollingMonitor implements TaskMonitor { // check if it is started if( handler.checkIfRunning() ) { log.trace "Task started > $handler" - notifyTaskStart(handler) + session.notifyTaskStart(handler) } // check if it is terminated @@ -620,7 +620,7 @@ class TaskPollingMonitor implements TaskMonitor { final fault = handler.task.processor.finalizeTask(handler.task) // notify task completion - notifyTaskComplete(handler) + session.notifyTaskComplete(handler) // abort the execution in case of task failure if (fault instanceof TaskFault) { @@ -659,7 +659,7 @@ class TaskPollingMonitor implements TaskMonitor { // notify task completion handler.task.aborted = true - notifyTaskComplete(handler) + session.notifyTaskComplete(handler) } try { @@ -677,21 +677,5 @@ class TaskPollingMonitor implements TaskMonitor { return pendingQueue } - protected void notifyTaskSubmit(TaskHandler handler) { - session.notifyTaskSubmit(handler) - } - - protected void notifyTaskPending(TaskHandler handler) { - session.notifyTaskPending(handler) - } - - protected void notifyTaskStart(TaskHandler handler) { - session.notifyTaskStart(handler) - } - - protected void notifyTaskComplete(TaskHandler handler) { - session.notifyTaskComplete(handler) - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index ab9999fec8..c9a043c007 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -253,6 +253,8 @@ class TaskProcessor { private Boolean isFair0 + private ArrayTaskCollector arrayCollector + private CompilerConfiguration compilerConfig() { final config = new CompilerConfiguration() config.addCompilationCustomizers( new ASTTransformationCustomizer(TaskTemplateVarsXform) ) @@ -307,6 +309,9 @@ class TaskProcessor { this.maxForks = config.maxForks ? config.maxForks as int : 0 this.forksCount = maxForks ? new LongAdder() : null this.isFair0 = config.getFair() + + final arraySize = config.getArray() + this.arrayCollector = arraySize > 0 ? new ArrayTaskCollector(executor, arraySize) : null } /** @@ -2204,7 +2209,10 @@ class TaskProcessor { makeTaskContextStage3(task, hash, folder) // add the task to the collection of running tasks - executor.submit(task) + if( arrayCollector ) + arrayCollector.submit(task) + else + executor.submit(task) } @@ -2298,6 +2306,10 @@ class TaskProcessor { state.update { StateObj it -> it.incCompleted() } } + protected void closeProcess() { + arrayCollector?.close() + } + protected void terminateProcess() { log.trace "<${name}> Sending poison pills and terminating process" sendPoisonPill() @@ -2432,21 +2444,6 @@ class TaskProcessor { log.trace "<${name}> Poison pill arrived; port: $index" openPorts.set(index, 0) // mark the port as closed state.update { StateObj it -> it.poison() } - - // check whether all input channels are closed - def closed = true - for( int i = 0; i < openPorts.length() - 1; i++ ) { - if( openPorts.get(i) != 0 ) { - closed = false - break - } - } - - // notify session that the process is closed - if( closed ) { - log.trace "<${name}> All ports are closed, closing process" - session.notifyProcessClose(name) - } } return message @@ -2455,6 +2452,7 @@ class TaskProcessor { @Override void afterStop(final DataflowProcessor processor) { log.trace "<${name}> After stop" + closeProcess() } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/script/ProcessConfig.groovy b/modules/nextflow/src/main/groovy/nextflow/script/ProcessConfig.groovy index 483adfa827..6970bd64be 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/ProcessConfig.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/ProcessConfig.groovy @@ -47,6 +47,7 @@ class ProcessConfig implements Map, Cloneable { 'accelerator', 'afterScript', 'arch', + 'array', 'beforeScript', 'cache', 'conda', @@ -953,4 +954,23 @@ class ProcessConfig implements Map, Cloneable { return this } + int getArray() { + final value = configProperties.get('array') + if( value == null ) + return 0 + if( value instanceof Closure ) + throw new IllegalArgumentException("Process directive `array` cannot be declared in a dynamic manner with a closure") + try { + final result = value as Integer + if( result < 0 ) + throw new IllegalArgumentException("Process directive `array` cannot be a negative number") + if( result == 1 ) + throw new IllegalArgumentException("Process directive `array` should be greater than 1") + return result + } + catch( NumberFormatException e ) { + throw new IllegalArgumentException("Process directive `array` should be an integer greater than 1 -- offending value: '$value'", e) + } + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy index f96cd0985a..45e414764e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy @@ -45,19 +45,14 @@ trait TraceObserver { */ void onFlowComplete() {} - /** + /* * Invoked when the process is created. */ void onProcessCreate( TaskProcessor process ){} - /** - * Invoked when all tasks for a process have been created. - */ - void onProcessClose( String process ){} - - /** - * Invoked when all tasks have been executed and process ends. - */ + /* + * Invoked when all tak have been executed and process ends. + */ void onProcessTerminate( TaskProcessor process ){} /** diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy deleted file mode 100644 index 8817d69b86..0000000000 --- a/modules/nextflow/src/test/groovy/nextflow/executor/ArrayExecutorTest.groovy +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.executor - -import nextflow.Session -import spock.lang.Specification -/** - * - * @author Ben Sherman - */ -class ArrayExecutorTest extends Specification { - - def 'should throw error if target executor does not support array jobs' () { - - setup: - def executorFactory = Spy(ExecutorFactory) - def session = Mock(Session) { - getExecutorFactory() >> executorFactory - } - def executor = new ArrayExecutor() - executor.session = session - - when: - executor.createTaskMonitor() - then: - session.getExecConfigProp('array', 'target', 'local') >> 'nope' - executorFactory.getExecutor('nope', session) >> new NopeExecutor() - IllegalArgumentException e = thrown() - e.message == "Executor 'nope' does not support array jobs" - } - -} From ed5089dbe4b8317b4fcc1102f2bca60ca57f2094 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 25 Apr 2023 16:34:28 -0500 Subject: [PATCH 10/97] minor edits Signed-off-by: Ben Sherman --- docs/process.md | 2 +- .../main/groovy/nextflow/processor/ArrayTaskCollector.groovy | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/process.md b/docs/process.md index a8b52c861f..efefe4ebca 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1245,7 +1245,7 @@ Allowed values for the `arch` directive are as follows, grouped by equivalent me Examples of values for the architecture `target` option are `cascadelake`, `icelake`, `zen2` and `zen3`. See the Spack documentation for the full and up-to-date [list of meaningful targets](https://spack.readthedocs.io/en/latest/basic_usage.html#support-for-specific-microarchitectures). -(array-executor)= +(process-array)= ## array diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy index 9e4c9c8c93..3ff288161e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy @@ -90,7 +90,7 @@ class ArrayTaskCollector { synchronized protected void submit0( Queue queue, int size ) { // remove tasks from the queue - def array = new ArrayList() + def array = new ArrayList(size) def iter = queue.iterator() while( iter.hasNext() && array.size() < size ) { From 5b36129a4eee98a4d3a1a4f75ada4c2fef16b226 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 25 Apr 2023 16:42:47 -0500 Subject: [PATCH 11/97] Support array jobs for LSF, PBS, SGE Signed-off-by: Ben Sherman --- docs/process.md | 4 ++++ .../groovy/nextflow/executor/LsfExecutor.groovy | 16 ++++++++++++++++ .../groovy/nextflow/executor/PbsExecutor.groovy | 16 ++++++++++++++++ .../groovy/nextflow/executor/SgeExecutor.groovy | 14 ++++++++++++++ 4 files changed, 50 insertions(+) diff --git a/docs/process.md b/docs/process.md index efefe4ebca..05a3aac236 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1277,6 +1277,10 @@ process cpu_task { Nextflow currently supports array jobs for the following executors: - {ref}`local-executor` +- {ref}`lsf-executor` +- {ref}`pbs-executor` +- {ref}`pbspro-executor` +- {ref}`sge-executor` - {ref}`slurm-executor` A process using array jobs will collect tasks and submit each batch as an array job when it is ready. Any "leftover" tasks will be submitted as a partial array job. diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index a4216b59bb..761e81bd41 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -300,4 +300,20 @@ class LsfExecutor extends AbstractGridExecutor { boolean isFusionEnabled() { return FusionHelper.isFusionEnabled(session) } + + private volatile int arrayTaskCount = 0 + + @Override + protected String getArrayDirective(int arraySize) { + "-J \"nf-array-${arrayTaskCount++}[0-${arraySize - 1}]\"" + } + + @Override + protected String getArrayIndexName() { 'LSB_JOBINDEX' } + + @Override + protected List getArraySubmitCommandLine() { List.of('bsub') } + + @Override + protected String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index abdea5f8eb..f131c8135a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -172,4 +172,20 @@ class PbsExecutor extends AbstractGridExecutor { static protected boolean matchOptions(String value) { value ? OPTS_REGEX.matcher(value).find() : null } + + @Override + protected String getArrayDirective(int arraySize) { + "-J 0-${arraySize - 1}" + } + + @Override + protected String getArrayIndexName() { 'PBS_ARRAY_INDEX' } + + @Override + protected List getArraySubmitCommandLine() { List.of('qsub') } + + @Override + protected String getArrayTaskId(String jobId, int index) { + jobId.replace('[]', "[$index]") + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index fab1de5ed3..b8081c62a7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -183,4 +183,18 @@ class SgeExecutor extends AbstractGridExecutor { boolean isFusionEnabled() { return FusionHelper.isFusionEnabled(session) } + + @Override + protected String getArrayDirective(int arraySize) { + "-t 0-${arraySize - 1}" + } + + @Override + protected String getArrayIndexName() { 'SGE_TASK_ID' } + + @Override + protected List getArraySubmitCommandLine() { List.of('qsub', '-') } + + @Override + protected String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } } From d162948643642fa7320bffd304bebd8003a1023d Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 26 Apr 2023 11:30:07 -0500 Subject: [PATCH 12/97] Replace synchronized with atomics and locks Signed-off-by: Ben Sherman --- .../executor/ArrayTaskSubmitter.groovy | 10 +-- .../nextflow/executor/LsfExecutor.groovy | 5 +- .../processor/ArrayTaskCollector.groovy | 77 +++++++++++-------- 3 files changed, 51 insertions(+), 41 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy index 5aeb2e5b78..16464e6b62 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy @@ -16,6 +16,8 @@ package nextflow.executor +import java.util.concurrent.atomic.AtomicInteger + import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.processor.TaskHandler @@ -31,7 +33,7 @@ class ArrayTaskSubmitter { protected List array - private volatile int collected = 0 + private AtomicInteger collected = new AtomicInteger() ArrayTaskSubmitter(List array) { this.array = array @@ -47,10 +49,8 @@ class ArrayTaskSubmitter { * * @param handler */ - synchronized void collect(TaskHandler handler) { - collected += 1 - - if( collected == array.size() ) + void collect(TaskHandler handler) { + if( collected.incrementAndGet() == array.size() ) submit() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 761e81bd41..a7cd8f2031 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -18,6 +18,7 @@ package nextflow.executor import java.nio.file.Path import java.nio.file.Paths +import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern import groovy.util.logging.Slf4j @@ -301,11 +302,11 @@ class LsfExecutor extends AbstractGridExecutor { return FusionHelper.isFusionEnabled(session) } - private volatile int arrayTaskCount = 0 + private AtomicInteger arrayTaskCount = new AtomicInteger() @Override protected String getArrayDirective(int arraySize) { - "-J \"nf-array-${arrayTaskCount++}[0-${arraySize - 1}]\"" + "-J \"nf-array-${arrayTaskCount.getAndIncrement()}[0-${arraySize - 1}]\"" } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy index 3ff288161e..8143874963 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy @@ -17,6 +17,8 @@ package nextflow.processor import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.locks.Lock +import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -39,9 +41,11 @@ class ArrayTaskCollector { private int arraySize - private volatile Queue queue = new ConcurrentLinkedQueue<>() + private Lock sync = new ReentrantLock() - private volatile boolean closed = false + private List array + + private boolean closed = false ArrayTaskCollector(Executor executor, int arraySize) { if( executor !instanceof ArrayTaskAware ) @@ -50,30 +54,40 @@ class ArrayTaskCollector { this.executor = executor this.monitor = executor.monitor this.arraySize = arraySize + this.array = new ArrayList<>(arraySize) } /** - * Add a task to the queue, and submit an array job when the - * queue reaches the desired size. + * Add a task to the current array, and submit the array when it + * reaches the desired size. * * @param handler */ - synchronized void submit(TaskRun task) { - // submit task directly if the collector is closed - if( closed ) { - executor.submit(task) - return + void submit(TaskRun task) { + sync.lock() + + try { + // submit task directly if the collector is closed + if( closed ) { + executor.submit(task) + return + } + + // create task handler + final handler = executor.createTaskHandler(task) + + // add task to the array array + array << handler + + // submit array job when the array is ready + if( array.size() == arraySize ) { + submit0(array) + array = new ArrayList<>(arraySize) + } + } + finally { + sync.unlock() } - - // create task handler - final handler = executor.createTaskHandler(task) - - // add task to the array queue - queue.add(handler) - - // submit array job when a batch is ready - if( queue.size() >= arraySize ) - submit0(queue, arraySize) } /** @@ -82,25 +96,20 @@ class ArrayTaskCollector { * @param process */ void close() { - if( queue.size() > 0 ) - submit0(queue, queue.size()) + sync.lock() - closed = true - } - - synchronized protected void submit0( Queue queue, int size ) { - // remove tasks from the queue - def array = new ArrayList(size) - def iter = queue.iterator() + try { + if( array.size() > 0 ) + submit0(array) - while( iter.hasNext() && array.size() < size ) { - array << iter.next() - iter.remove() + closed = true } + finally { + sync.unlock() + } + } - if( array.size() < size ) - log.warn "Array job expected ${size} tasks but received only ${array.size()}" - + protected void submit0(List array) { // create submitter for array job ((ArrayTaskAware)executor).createArrayTaskSubmitter(array) From 99e92d5d63bc7eb0b19b8bb37bf5938dfbc7d3c9 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Sun, 30 Apr 2023 14:24:33 -0500 Subject: [PATCH 13/97] Add array job support to AWS Batch Signed-off-by: Ben Sherman --- docs/process.md | 1 + .../nextflow/fusion/FusionHelper.groovy | 4 + .../aws/batch/AwsBatchArraySubmitter.groovy | 146 ++++ .../cloud/aws/batch/AwsBatchExecutor.groovy | 9 +- .../aws/batch/AwsBatchTaskHandler.groovy | 742 +++--------------- .../cloud/aws/batch/SubmitJobAware.groovy | 532 +++++++++++++ .../aws/batch/AwsBatchTaskHandlerTest.groovy | 181 ++--- 7 files changed, 910 insertions(+), 705 deletions(-) create mode 100644 plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy create mode 100644 plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy diff --git a/docs/process.md b/docs/process.md index 05a3aac236..74add51fdb 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1276,6 +1276,7 @@ process cpu_task { Nextflow currently supports array jobs for the following executors: +- {ref}`awsbatch-executor` - {ref}`local-executor` - {ref}`lsf-executor` - {ref}`pbs-executor` diff --git a/modules/nextflow/src/main/groovy/nextflow/fusion/FusionHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/fusion/FusionHelper.groovy index cca6e3f5f0..7f7acecf42 100644 --- a/modules/nextflow/src/main/groovy/nextflow/fusion/FusionHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/fusion/FusionHelper.groovy @@ -88,4 +88,8 @@ class FusionHelper { return Path.of(result) } + static Path toContainerMount(Path path) { + toContainerMount(path, path.scheme) + } + } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy new file mode 100644 index 0000000000..ab55d0444a --- /dev/null +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy @@ -0,0 +1,146 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.cloud.aws.batch + +import static nextflow.fusion.FusionConfig.FUSION_PATH +import static nextflow.fusion.FusionHelper.* + +import com.amazonaws.services.batch.AWSBatch +import com.amazonaws.services.batch.model.AWSBatchException +import com.amazonaws.services.batch.model.ArrayProperties +import com.amazonaws.services.batch.model.SubmitJobRequest +import com.amazonaws.services.batch.model.SubmitJobResult +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.exception.ProcessSubmitException +import nextflow.executor.ArrayTaskSubmitter +import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun +import nextflow.processor.TaskStatus +/** + * Submit tasks as an array job for a grid executor. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class AwsBatchArraySubmitter extends ArrayTaskSubmitter implements SubmitJobAware { + + private AwsBatchExecutor executor + + private AWSBatch client + + AwsBatchArraySubmitter(List array, AwsBatchExecutor executor) { + super(array) + this.executor = executor + this.client = executor.client + + if( array.size() > 10_000 ) + throw new IllegalArgumentException("Array jobs on AWS Batch may not have size greater than 10,000") + } + + @Override + TaskRun getTask() { array.first().getTask() } + + @Override + AWSBatch getClient() { executor.client } + + @Override + AwsOptions getAwsOptions() { executor.getAwsOptions() } + + @Override + List getSubmitCommand() { + return fusionEnabled() + ? fusionSubmitCli() + : classicSubmitCli() + } + + @Override + List fusionSubmitCli() { + final workDirs = array + .collect { handler -> toContainerMount(handler.task.workDir) } + .join(' ') + + final cmd = """ + declare -a array=( ${workDirs} ) + bash \${array[\$AWS_BATCH_JOB_ARRAY_INDEX]}/${TaskRun.CMD_RUN} + """.stripIndent().trim() + + return List.of(FUSION_PATH, 'bash', cmd.toString()) + } + + protected List classicSubmitCli() { + final workDirs = array + .collect { handler -> handler.task.workDir.toUriString() } + .join(' ') + + final opts = getAwsOptions() + final cli = opts.getAwsCli() + final debug = opts.debug ? ' --debug' : '' + final sse = opts.storageEncryption ? " --sse ${opts.storageEncryption}" : '' + final kms = opts.storageKmsKeyId ? " --sse-kms-key-id ${opts.storageKmsKeyId}" : '' + final aws = "${cli} s3 cp --only-show-errors${debug}${sse}${kms}" + + final cmd = """ + declare -a array=( ${workDirs} ) + task_dir=\${array[\$AWS_BATCH_JOB_ARRAY_INDEX]} + trap "{ ret=\$?; ${aws} ${TaskRun.CMD_LOG} \$task_dir/${TaskRun.CMD_LOG}||true; exit \$ret; }" EXIT + ${aws} \$task_dir/${TaskRun.CMD_RUN} - | bash 2>&1 | tee ${TaskRun.CMD_LOG} + """.stripIndent().trim() + + return List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) + } + + @Override + void submit() { + // -- create the submit request + final request = newSubmitRequest(getTask()) + .withArrayProperties(new ArrayProperties().withSize(array.size())) + + // -- submit the array job + final response = submitJobRequest0(bypassProxy(client), request) + final jobId = response.jobId + + // -- set the job id, queue, and status of each task + array.eachWithIndex { handler, i -> + ((AwsBatchTaskHandler)handler).jobId = getArrayTaskId(jobId, i) + ((AwsBatchTaskHandler)handler).queueName = request.getJobQueue() + handler.status = TaskStatus.SUBMITTED + } + + log.debug "[AWS BATCH] submitted array job > jobId: ${jobId}" + } + + static private SubmitJobResult submitJobRequest0(AWSBatch client, SubmitJobRequest request) { + try { + return client.submitJob(request) + } + catch( AWSBatchException e ) { + if( e.statusCode >= 500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + + String getArrayTaskId(String jobId, int index) { + "${jobId}:${index}" + } + +} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 8c1fbffc7c..219a6207bb 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -32,6 +32,8 @@ import nextflow.cloud.aws.AwsClientFactory import nextflow.cloud.aws.config.AwsConfig import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException +import nextflow.executor.ArrayTaskAware +import nextflow.executor.ArrayTaskSubmitter import nextflow.executor.Executor import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx @@ -54,7 +56,7 @@ import org.pf4j.ExtensionPoint @Slf4j @ServiceName('awsbatch') @CompileStatic -class AwsBatchExecutor extends Executor implements ExtensionPoint { +class AwsBatchExecutor extends Executor implements ExtensionPoint, ArrayTaskAware { /** * Proxy to throttle AWS batch client requests @@ -219,6 +221,11 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint { new AwsBatchTaskHandler(task, this) } + @Override + ArrayTaskSubmitter createArrayTaskSubmitter(List array) { + new AwsBatchArraySubmitter(array, this) + } + /** * @return Creates a {@link ThrottlingExecutor} service to throttle * the API requests to the AWS Batch service. diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 458e8582a4..9becc141f4 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -16,64 +16,38 @@ package nextflow.cloud.aws.batch -import static nextflow.cloud.aws.batch.AwsContainerOptionsMapper.* - import java.nio.file.Path -import java.nio.file.Paths -import java.time.Instant import com.amazonaws.services.batch.AWSBatch import com.amazonaws.services.batch.model.AWSBatchException import com.amazonaws.services.batch.model.AttemptContainerDetail -import com.amazonaws.services.batch.model.ClientException -import com.amazonaws.services.batch.model.ContainerOverrides -import com.amazonaws.services.batch.model.ContainerProperties -import com.amazonaws.services.batch.model.DescribeJobDefinitionsRequest -import com.amazonaws.services.batch.model.DescribeJobDefinitionsResult import com.amazonaws.services.batch.model.DescribeJobsRequest import com.amazonaws.services.batch.model.DescribeJobsResult -import com.amazonaws.services.batch.model.EvaluateOnExit -import com.amazonaws.services.batch.model.Host -import com.amazonaws.services.batch.model.JobDefinition -import com.amazonaws.services.batch.model.JobDefinitionType import com.amazonaws.services.batch.model.JobDetail -import com.amazonaws.services.batch.model.JobTimeout -import com.amazonaws.services.batch.model.KeyValuePair -import com.amazonaws.services.batch.model.LogConfiguration -import com.amazonaws.services.batch.model.MountPoint -import com.amazonaws.services.batch.model.RegisterJobDefinitionRequest -import com.amazonaws.services.batch.model.RegisterJobDefinitionResult -import com.amazonaws.services.batch.model.ResourceRequirement -import com.amazonaws.services.batch.model.ResourceType -import com.amazonaws.services.batch.model.RetryStrategy import com.amazonaws.services.batch.model.SubmitJobRequest import com.amazonaws.services.batch.model.SubmitJobResult import com.amazonaws.services.batch.model.TerminateJobRequest -import com.amazonaws.services.batch.model.Volume +import groovy.transform.CompileDynamic import groovy.transform.CompileStatic -import groovy.transform.Memoized import groovy.util.logging.Slf4j -import nextflow.Const import nextflow.cloud.types.CloudMachineInfo -import nextflow.container.ContainerNameValidator import nextflow.exception.ProcessSubmitException import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder -import nextflow.executor.res.AcceleratorResource -import nextflow.fusion.FusionAwareTask import nextflow.processor.BatchContext import nextflow.processor.BatchHandler import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus import nextflow.trace.TraceRecord -import nextflow.util.CacheHelper /** - * Implements a task handler for AWS Batch jobs + * Implements a task handler for AWS Batch jobs. + * + * @author Paolo Di Tommaso */ -// note: do not declare this class as `CompileStatic` otherwise the proxy is not get invoked @Slf4j -class AwsBatchTaskHandler extends TaskHandler implements BatchHandler, FusionAwareTask { +@CompileStatic +class AwsBatchTaskHandler extends TaskHandler implements BatchHandler, SubmitJobAware { private final Path exitFile @@ -93,19 +67,13 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler environment + private volatile String queueName - final static private Map jobDefinitions = [:] + private volatile CloudMachineInfo machineInfo /** * Batch context shared between multiple task handlers @@ -124,8 +92,6 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler getSubmitCommand() { + return fusionEnabled() + ? fusionSubmitCli() + : classicSubmitCli() + } + + protected List classicSubmitCli() { + final opts = getAwsOptions() + final cli = opts.getAwsCli() + final debug = opts.debug ? ' --debug' : '' + final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' + final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' + final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" + final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $aws s3:/${getWrapperFile()} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" + return ['bash','-o','pipefail','-c', cmd.toString()] + } + + @Override + void submit() { + if( arraySubmitter ) { + arraySubmitter.collect(this) + return + } + + // -- create the submit request + final request = newSubmitRequest(task) + log.trace "[AWS BATCH] new job request > $request" + + // -- submit the job + // use the real client object because this method + // should be invoked by the thread pool + final response = submitJobRequest0(bypassProxy(client), request) + this.jobId = response.jobId + this.queueName = request.getJobQueue() + this.status = TaskStatus.SUBMITTED + log.debug "[AWS BATCH] submitted > job=$jobId; work-dir=${task.getWorkDirStr()}" + } + + static private SubmitJobResult submitJobRequest0(AWSBatch client, SubmitJobRequest request) { + try { + return client.submitJob(request) + } + catch( AWSBatchException e ) { + if( e.statusCode >= 500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + + void setJobId(String jobId) { + this.jobId = jobId + } + + void setQueueName(String queueName) { + this.queueName = queueName + } + + @Override + boolean checkIfRunning() { + if( !jobId || !isSubmitted() ) + return false + final job = describeJob(jobId) + final result = job?.status in ['RUNNING', 'SUCCEEDED', 'FAILED'] + if( result ) + this.status = TaskStatus.RUNNING + // fetch the task arn + if( !taskArn ) + taskArn = job?.getContainer()?.getTaskArn() + return result } /** @@ -169,6 +220,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler $req" - - /* - * submit the task execution - */ - // note use the real client object because this method - // is supposed to be invoked by the thread pool - final resp = submit0(bypassProxy(client), req) - this.jobId = resp.jobId - this.status = TaskStatus.SUBMITTED - this.queueName = req.getJobQueue() - log.debug "[AWS BATCH] submitted > job=$jobId; work-dir=${task.getWorkDirStr()}" - } - - protected BashWrapperBuilder createTaskWrapper() { - return fusionEnabled() - ? fusionLauncher() - : new AwsBatchScriptLauncher(task.toTaskBean(), getAwsOptions()) - } - - protected void buildTaskWrapper() { - createTaskWrapper().build() - } - - protected AWSBatch bypassProxy(AWSBatch batch) { - batch instanceof AwsBatchProxy ? batch.getClient() : batch - } - - /** - * Retrieve the queue name to use to submit the task execution - * - * @param task The {@link TaskRun} object to be executed - * @return The Batch queue name defined by this job execution - */ - protected String getJobQueue(TaskRun task) { - final queue = task.config.queue?.toString() - if( !queue ) - throw new ProcessUnrecoverableException("Missing AWS Batch job queue -- provide it by using the process `queue` directive") - - return queue - } - - /** - * Get the Batch job definition name used to run the specified task - * - * @param task The {@link TaskRun} object to be executed - * @return The Batch job definition name defined by this job execution - */ - protected String getJobDefinition(TaskRun task) { - final container = task.getContainer() - if( !container ) - throw new ProcessUnrecoverableException("Invalid AWS Batch job definition -- provide a Docker image name or a Batch job definition name") - - if( container.startsWith('job-definition://')) { - return container.substring(17) - } - - resolveJobDefinition(container) - } - - /** - * Maps a docker container image to a Batch job definition name - * - * @param container The Docker container image name which need to be used to run the job - * @return The Batch Job Definition name associated with the specified container - */ - @CompileStatic - protected String resolveJobDefinition(String container) { - final int DEFAULT_BACK_OFF_BASE = 3 - final int DEFAULT_BACK_OFF_DELAY = 250 - final int MAX_ATTEMPTS = 5 - int attempt=0 - while( true ) { - try { - return resolveJobDefinition0(container) - } - catch (ClientException e) { - if( e.statusCode != 404 || attempt++ > MAX_ATTEMPTS) - throw e - - final delay = (Math.pow(DEFAULT_BACK_OFF_BASE, attempt) as long) * DEFAULT_BACK_OFF_DELAY - log.debug "Got AWS Client exception on Batch resolve job definition - message=$e.message; waiting for ${delay}ms (attempt=$attempt)" - Thread.sleep(delay) - } - } - } - - protected String resolveJobDefinition0(String container) { - final req = makeJobDefRequest(container) - final token = req.getParameters().get('nf-token') - final jobKey = "$container:$token".toString() - if( jobDefinitions.containsKey(jobKey) ) - return jobDefinitions[jobKey] - - synchronized(jobDefinitions) { - if( jobDefinitions.containsKey(jobKey) ) - return jobDefinitions[jobKey] - - def msg - def name = findJobDef(req.jobDefinitionName, token) - if( name ) { - msg = "[AWS BATCH] Found job definition name=$name; container=$container" - } - else { - name = createJobDef(req) - msg = "[AWS BATCH] Created job definition name=$name; container=$container" - } - // log the request - if( log.isTraceEnabled() ) - log.debug "[AWS BATCH] $msg; request=${req.toString().indent()}" - else - log.debug "[AWS BATCH] $msg" - - jobDefinitions[jobKey] = name - return name - } - } - - /** - * Create a Batch job definition request object for the specified Docker image - * - * @param image The Docker container image for which is required to create a Batch job definition - * @return An instance of {@link com.amazonaws.services.batch.model.RegisterJobDefinitionRequest} for the specified Docker image - */ - protected RegisterJobDefinitionRequest makeJobDefRequest(String image) { - final uniq = new ArrayList() - final result = configJobDefRequest(image, uniq) - - // create a job marker uuid - def hash = computeUniqueToken(uniq) - result.setParameters(['nf-token':hash]) - - return result - } - - protected String computeUniqueToken(List uniq) { - return CacheHelper.hasher(uniq).hash().toString() - } - - /** - * Create and configure the actual RegisterJobDefinitionRequest object - * - * @param image - * The Docker container image for which is required to create a Batch job definition - * @param hashingTokens - * A list used to collect values that should be used to create a unique job definition Id for the given job request. - * It should be used to return such values in the calling context - * @return - * An instance of {@link com.amazonaws.services.batch.model.RegisterJobDefinitionRequest} for the specified Docker image - */ - protected RegisterJobDefinitionRequest configJobDefRequest(String image, List hashingTokens) { - final name = normalizeJobDefinitionName(image) - final opts = getAwsOptions() - - final result = new RegisterJobDefinitionRequest() - result.setJobDefinitionName(name) - result.setType(JobDefinitionType.Container) - - // create the container opts based on task config - final containerOpts = task.getConfig().getContainerOptionsMap() - final container = createContainerProperties(containerOpts) - - // container definition - final _1_cpus = new ResourceRequirement().withType(ResourceType.VCPU).withValue('1') - final _1_gb = new ResourceRequirement().withType(ResourceType.MEMORY).withValue('1024') - container - .withImage(image) - .withCommand('true') - // note the actual command, memory and cpus are overridden when the job is executed - .withResourceRequirements( _1_cpus, _1_gb ) - - final jobRole = opts.getJobRole() - if( jobRole ) - container.setJobRoleArn(jobRole) - - final logsGroup = opts.getLogsGroup() - if( logsGroup ) - container.setLogConfiguration(getLogConfiguration(logsGroup, opts.getRegion())) - - if( fusionEnabled() ) - container.setPrivileged(true) - - final mountsMap = new LinkedHashMap( 10) - final awscli = opts.cliPath - if( awscli ) { - def path = Paths.get(awscli).parent.parent.toString() - mountsMap.put('aws-cli', "$path:$path:ro") - } - - int c=0 - final volumes = opts.getVolumes() - for( String vol : volumes ) { - mountsMap.put("vol-"+(++c), vol) - } - - if( mountsMap ) - addVolumeMountsToContainer(mountsMap, container) - - // finally set the container options - result.setContainerProperties(container) - - // add to this list all values that has to contribute to the - // job definition unique name creation - hashingTokens.add(name) - hashingTokens.add(container.toString()) - if( containerOpts ) - hashingTokens.add(containerOpts) - - return result - } - - @Memoized - LogConfiguration getLogConfiguration(String name, String region) { - new LogConfiguration() - .withLogDriver('awslogs') - .withOptions([ - 'awslogs-region': region, - 'awslogs-group': name - ]) - } - - protected void addVolumeMountsToContainer(Map mountsMap, ContainerProperties container) { - final mounts = new ArrayList(mountsMap.size()) - final volumes = new ArrayList(mountsMap.size()) - for( Map.Entry entry : mountsMap.entrySet() ) { - final mountName = entry.key - final parts = entry.value.tokenize(':') - final containerPath = parts[0] - final hostPath = parts.size()>1 ? parts[1] : containerPath - final readOnly = parts.size()>2 ? parts[2]=='ro' : false - if( parts.size()>3 ) - throw new IllegalArgumentException("Not a valid volume mount syntax: $entry.value") - - def mount = new MountPoint() - .withSourceVolume(mountName) - .withContainerPath(hostPath) - .withReadOnly(readOnly) - mounts << mount - - def vol = new Volume() - .withName(mountName) - .withHost(new Host() - .withSourcePath(containerPath)) - volumes << vol - } - - if( mountsMap ) { - container.setMountPoints(mounts) - container.setVolumes(volumes) - } - } - - /** - * Look for a Batch job definition in ACTIVE status for the given name and NF job definition ID - * - * @param name The Batch job definition name - * @param jobId A unique job definition ID generated by NF - * @return The fully qualified Batch job definition name eg {@code my-job-definition:3} - */ - protected String findJobDef(String name, String jobId) { - log.trace "[AWS BATCH] checking job definition with name=$name; jobid=$jobId" - final req = new DescribeJobDefinitionsRequest().withJobDefinitionName(name) - // bypass the proxy because this method is invoked during a - // job submit request that's already in a separate thread pool request - // therefore it's protected by a TooManyRequestsException - final res = describeJobDefinitions0(bypassProxy(this.client), req) - final jobs = res.getJobDefinitions() - if( jobs.size()==0 ) - return null - - def job = jobs.find { JobDefinition it -> it.status == 'ACTIVE' && it.parameters?.'nf-token' == jobId } - return job ? "$name:$job.revision" : null - } - - /** - * Create (aka register) a new Batch job definition - * - * @param req A {@link RegisterJobDefinitionRequest} representing the Batch jib definition to create - * @return The fully qualified Batch job definition name eg {@code my-job-definition:3} - */ - protected String createJobDef(RegisterJobDefinitionRequest req) { - // add nextflow tags - req.addTagsEntry('nextflow.io/createdAt', Instant.now().toString()) - req.addTagsEntry('nextflow.io/version', Const.APP_VER) - // create the job def - final res = createJobDef0(bypassProxy(client), req) // bypass the client proxy! see #1024 - return "${res.jobDefinitionName}:$res.revision" - } - - /** - * Make a name string compliant with the Batch job definition format - * - * @param name A job name - * @return A given name formatted to be used as Job definition name - */ - protected String normalizeJobDefinitionName(String name) { - if( !name ) return null - if( !ContainerNameValidator.isValidImageName(name) ) throw new IllegalArgumentException("Invalid container image name: $name") - - def result = name.replaceAll(/[^a-zA-Z0-9\-_]+/,'-') - // Batch job definition length cannot exceed 128 characters - // take first 40 chars + add a unique MD5 hash (32 chars) - if( result.length()>125 ) { - final hash = name.md5() - result = result.substring(0,40) + '-' + hash - } - - return "nf-" + result - } - - protected List classicSubmitCli() { - // the cmd list to launch it - final opts = getAwsOptions() - final cli = opts.getAwsCli() - final debug = opts.debug ? ' --debug' : '' - final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' - final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' - final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" - final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $aws s3:/${getWrapperFile()} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" - return ['bash','-o','pipefail','-c', cmd.toString()] - } - - protected List getSubmitCommand() { - // final launcher command - return fusionEnabled() - ? fusionSubmitCli() - : classicSubmitCli() - } - - protected maxSpotAttempts() { - return executor.awsOptions.maxSpotAttempts - } - - /** - * Create a new Batch job request for the given NF {@link TaskRun} - * - * @param task A {@link TaskRun} to be executed as Batch job - * @return A {@link com.amazonaws.services.batch.model.SubmitJobRequest} instance representing the Batch job to submit - */ - protected SubmitJobRequest newSubmitRequest(TaskRun task) { - - /* - * create the request object - */ - final labels = task.config.getResourceLabels() - final result = new SubmitJobRequest() - result.setJobName(normalizeJobName(task.name)) - result.setJobQueue(getJobQueue(task)) - result.setJobDefinition(getJobDefinition(task)) - if( labels ) { - result.setTags(labels) - result.setPropagateTags(true) - } - // set the share identifier - if( this.getAwsOptions().shareIdentifier ) { - result.setShareIdentifier(this.getAwsOptions().shareIdentifier) - result.setSchedulingPriorityOverride(this.getAwsOptions().schedulingPriority) - } - - /* - * retry on spot reclaim - * https://aws.amazon.com/blogs/compute/introducing-retry-strategies-for-aws-batch/ - */ - final attempts = maxSpotAttempts() - if( attempts>0 ) { - // retry the job when an Ec2 instance is terminate - final cond1 = new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*') - // the exit condition prevent to retry for other reason and delegate - // instead to nextflow error strategy the handling of the error - final cond2 = new EvaluateOnExit().withAction('EXIT').withOnReason('*') - final retry = new RetryStrategy() - .withAttempts( attempts ) - .withEvaluateOnExit(cond1, cond2) - result.setRetryStrategy(retry) - } - - // set task timeout - final time = task.config.getTime() - if( time ) { - def secs = time.toSeconds() as Integer - if( secs < 60 ) { - secs = 60 // Batch minimal allowed timeout is 60 seconds - } - result.setTimeout(new JobTimeout().withAttemptDurationSeconds(secs)) - } - - // set the actual command - final resources = new ArrayList(5) - def container = new ContainerOverrides() - container.command = getSubmitCommand() - // set the task memory - if( task.config.getMemory() ) { - final mega = (int)task.config.getMemory().toMega() - if( mega >= 4 ) - resources << new ResourceRequirement().withType(ResourceType.MEMORY).withValue(mega.toString()) - else - log.warn "Ignoring task ${task.lazyName()} memory directive: ${task.config.getMemory()} -- AWS Batch job memory request cannot be lower than 4 MB" - } - // set the task cpus - if( task.config.getCpus() > 1 ) - resources << new ResourceRequirement().withType(ResourceType.VCPU).withValue(task.config.getCpus().toString()) - - if( task.config.getAccelerator() ) - resources << createGpuResource(task.config.getAccelerator()) - - if( resources ) - container.withResourceRequirements(resources) - - // set the environment - def vars = getEnvironmentVars() - if( vars ) - container.setEnvironment(vars) - - result.setContainerOverrides(container) - - return result - } - - protected ResourceRequirement createGpuResource(AcceleratorResource acc) { - final res = new ResourceRequirement() - final type = acc.type ?: 'GPU' - final count = acc.request?.toString() ?: '1' - res.setType(type.toUpperCase()) - res.setValue(count) - return res - } - - /** - * @return The list of environment variables to be defined in the Batch job execution context - */ - protected List getEnvironmentVars() { - def vars = [] - if( this.environment?.containsKey('NXF_DEBUG') ) - vars << new KeyValuePair().withName('NXF_DEBUG').withValue(this.environment['NXF_DEBUG']) - if( this.getAwsOptions().retryMode && this.getAwsOptions().retryMode in AwsOptions.VALID_RETRY_MODES) - vars << new KeyValuePair().withName('AWS_RETRY_MODE').withValue(this.getAwsOptions().retryMode) - if( this.getAwsOptions().maxTransferAttempts ) { - vars << new KeyValuePair().withName('AWS_MAX_ATTEMPTS').withValue(this.getAwsOptions().maxTransferAttempts as String) - vars << new KeyValuePair().withName('AWS_METADATA_SERVICE_NUM_ATTEMPTS').withValue(this.getAwsOptions().maxTransferAttempts as String) - } - if( fusionEnabled() ) { - for(Map.Entry it : fusionLauncher().fusionEnv()) { - vars << new KeyValuePair().withName(it.key).withValue(it.value) - } - } - return vars - } - - /** - * @return The launcher script file {@link Path} - */ - protected Path getWrapperFile() { wrapperFile } - - /** - * @return The launcher log file {@link Path} - */ - protected Path getLogFile() { logFile } - - /** - * Remove invalid characters from a job name string - * - * @param name A job name containing possible invalid character - * @return A job name without invalid characters - */ - protected String normalizeJobName(String name) { - def result = name.replaceAll(' ','_').replaceAll(/[^a-zA-Z0-9_]/,'') - result.size()>128 ? result.substring(0,128) : result + TraceRecord getTraceRecord() { + def record = super.getTraceRecord() + record.put('native_id', jobId) + record.machineInfo = getMachineInfo() + return record } - protected CloudMachineInfo getMachineInfo() { if( machineInfo ) return machineInfo @@ -793,56 +354,5 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler=500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to submit job: ${req.jobName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - - static private DescribeJobDefinitionsResult describeJobDefinitions0(AWSBatch client, DescribeJobDefinitionsRequest req) { - try { - client.describeJobDefinitions(req) - } - catch (AWSBatchException e) { - if( e.statusCode>=500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to describe job definitions: ${req.jobDefinitions} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - - static private RegisterJobDefinitionResult createJobDef0(AWSBatch client, RegisterJobDefinitionRequest req) { - try { - return client.registerJobDefinition(req) - } - catch (AWSBatchException e) { - if( e.statusCode>=500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to register job definition: ${req.jobDefinitionName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy new file mode 100644 index 0000000000..7eb8b50b3f --- /dev/null +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy @@ -0,0 +1,532 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.cloud.aws.batch + +import static nextflow.cloud.aws.batch.AwsContainerOptionsMapper.* + +import java.nio.file.Paths +import java.time.Instant + +import com.amazonaws.services.batch.AWSBatch +import com.amazonaws.services.batch.model.AWSBatchException +import com.amazonaws.services.batch.model.ClientException +import com.amazonaws.services.batch.model.ContainerOverrides +import com.amazonaws.services.batch.model.ContainerProperties +import com.amazonaws.services.batch.model.DescribeJobDefinitionsRequest +import com.amazonaws.services.batch.model.DescribeJobDefinitionsResult +import com.amazonaws.services.batch.model.EvaluateOnExit +import com.amazonaws.services.batch.model.Host +import com.amazonaws.services.batch.model.JobDefinition +import com.amazonaws.services.batch.model.JobDefinitionType +import com.amazonaws.services.batch.model.JobTimeout +import com.amazonaws.services.batch.model.KeyValuePair +import com.amazonaws.services.batch.model.LogConfiguration +import com.amazonaws.services.batch.model.MountPoint +import com.amazonaws.services.batch.model.RegisterJobDefinitionRequest +import com.amazonaws.services.batch.model.RegisterJobDefinitionResult +import com.amazonaws.services.batch.model.ResourceRequirement +import com.amazonaws.services.batch.model.ResourceType +import com.amazonaws.services.batch.model.RetryStrategy +import com.amazonaws.services.batch.model.SubmitJobRequest +import com.amazonaws.services.batch.model.Volume +import groovy.transform.CompileStatic +import groovy.transform.Memoized +import nextflow.Const +import nextflow.container.ContainerNameValidator +import nextflow.exception.ProcessSubmitException +import nextflow.exception.ProcessUnrecoverableException +import nextflow.fusion.FusionAwareTask +import nextflow.processor.TaskRun +import nextflow.util.CacheHelper +import org.slf4j.Logger +import org.slf4j.LoggerFactory +/** + * Implementation of submit job requests for tasks. + * + * @author Paolo Di Tommaso + * @author Ben Sherman + */ +@CompileStatic +trait SubmitJobAware extends FusionAwareTask { + + static private Logger log = LoggerFactory.getLogger(SubmitJobAware) + + static private Map jobDefinitions = [:] + + Map environment = System.getenv() + + AWSBatch bypassProxy(AWSBatch client) { + client instanceof AwsBatchProxy ? client.getClient() : client + } + + /** + * Get the AWS Batch client. + */ + abstract AWSBatch getClient() + + /** + * Get the Nextflow configuration options for AWS Batch. + */ + abstract AwsOptions getAwsOptions() + + /** + * Get the submit command for a job request. + */ + abstract List getSubmitCommand() + + /** + * Create a new submit job request for a task. + * + * @param task + */ + SubmitJobRequest newSubmitRequest(TaskRun task) { + + // create the request object + final request = new SubmitJobRequest() + request.setJobName(normalizeJobName(task.name)) + request.setJobQueue(getJobQueue(task)) + request.setJobDefinition(getJobDefinition(task)) + + // set task timeout + final time = task.config.getTime() + if( time ) { + def secs = time.toSeconds() as Integer + if( secs < 60 ) { + secs = 60 // Batch minimal allowed timeout is 60 seconds + } + request.setTimeout(new JobTimeout().withAttemptDurationSeconds(secs)) + } + + // set the container overrides + final container = new ContainerOverrides() + + // set the submit command + container.setCommand(getSubmitCommand()) + + // set the environment vars + final env = getEnvironmentVars() + if( env ) + container.setEnvironment(env) + + // set the task resources + final resources = new ArrayList(5) + + if( task.config.getMemory() ) { + final mega = (int)task.config.getMemory().toMega() + if( mega >= 4 ) + resources << new ResourceRequirement().withType(ResourceType.MEMORY).withValue(mega.toString()) + else + log.warn "Ignoring task ${task.lazyName()} memory directive: ${task.config.getMemory()} -- AWS Batch job memory request cannot be lower than 4 MB" + } + + if( task.config.getCpus() > 1 ) + resources << new ResourceRequirement().withType(ResourceType.VCPU).withValue(task.config.getCpus().toString()) + + final accelerator = task.config.getAccelerator() + if( accelerator ) { + final type = accelerator.type?.toUpperCase() ?: 'GPU' + final count = accelerator.request?.toString() ?: '1' + + resources << new ResourceRequirement().withType(type).withValue(count) + } + + if( resources ) + container.withResourceRequirements(resources) + + request.setContainerOverrides(container) + + // set the resource labels + final labels = task.config.getResourceLabels() + if( labels ) { + request.setTags(labels) + request.setPropagateTags(true) + } + + // set the share identifier + if( awsOptions.shareIdentifier ) { + request.setShareIdentifier(awsOptions.shareIdentifier) + request.setSchedulingPriorityOverride(awsOptions.schedulingPriority) + } + + // retry on spot reclamation + // https://aws.amazon.com/blogs/compute/introducing-retry-strategies-for-aws-batch/ + if( awsOptions.maxSpotAttempts > 0 ) { + final retryStrategy = new RetryStrategy() + .withAttempts(awsOptions.maxSpotAttempts) + .withEvaluateOnExit( + // retry the job when an EC2 instance is terminated + new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*'), + // delegate all other exit conditions to Nextflow + new EvaluateOnExit().withAction('EXIT').withOnReason('*') + ) + + request.setRetryStrategy(retryStrategy) + } + + return request + } + + /** + * Remove invalid characters from a job name. + * + * @param name + */ + String normalizeJobName(String name) { + final result = name.replaceAll(' ','_').replaceAll(/[^a-zA-Z0-9_]/,'') + result.size() > 128 ? result.substring(0,128) : result + } + + /** + * Get the Batch queue name for a task. + * + * @param task + */ + String getJobQueue(TaskRun task) { + final queue = task.config.queue?.toString() + if( !queue ) + throw new ProcessUnrecoverableException("Missing AWS Batch job queue -- provide it by using the process `queue` directive") + + return queue + } + + /** + * Get the list of environment variables for a job request. + */ + List getEnvironmentVars() { + def vars = [] as List + + if( environment.containsKey('NXF_DEBUG') ) + vars << new KeyValuePair().withName('NXF_DEBUG').withValue(environment['NXF_DEBUG']) + + if( awsOptions.retryMode && awsOptions.retryMode in AwsOptions.VALID_RETRY_MODES ) + vars << new KeyValuePair().withName('AWS_RETRY_MODE').withValue(awsOptions.retryMode) + + if( awsOptions.maxTransferAttempts ) { + vars << new KeyValuePair().withName('AWS_MAX_ATTEMPTS').withValue(awsOptions.maxTransferAttempts as String) + vars << new KeyValuePair().withName('AWS_METADATA_SERVICE_NUM_ATTEMPTS').withValue(awsOptions.maxTransferAttempts as String) + } + + if( fusionEnabled() ) { + for( Map.Entry it : fusionLauncher().fusionEnv() ) { + vars << new KeyValuePair().withName(it.key).withValue(it.value) + } + } + + return vars + } + + /** + * Get the Batch job definition name for a task. + * + * @param task + */ + String getJobDefinition(TaskRun task) { + final container = task.getContainer() + if( !container ) + throw new ProcessUnrecoverableException("Invalid AWS Batch job definition -- provide a Docker image name or a Batch job definition name") + + if( container.startsWith('job-definition://')) + return container.substring(17) + + resolveJobDefinition(container) + } + + /** + * Get the Batch job definition name associated with a Docker container image. + * + * @param container + */ + String resolveJobDefinition(String container) { + final int DEFAULT_BACK_OFF_BASE = 3 + final int DEFAULT_BACK_OFF_DELAY = 250 + final int MAX_ATTEMPTS = 5 + int attempt = 0 + while( true ) { + try { + return resolveJobDefinition0(container) + } + catch (ClientException e) { + if( e.statusCode != 404 || attempt++ > MAX_ATTEMPTS) + throw e + + final delay = (Math.pow(DEFAULT_BACK_OFF_BASE, attempt) as long) * DEFAULT_BACK_OFF_DELAY + log.debug "Got AWS Client exception on Batch resolve job definition - message=$e.message; waiting for ${delay}ms (attempt=$attempt)" + Thread.sleep(delay) + } + } + } + + private String resolveJobDefinition0(String container) { + final request = makeJobDefRequest(container) + final token = request.getParameters().get('nf-token') + final jobKey = "$container:$token".toString() + if( jobDefinitions.containsKey(jobKey) ) + return jobDefinitions[jobKey] + + synchronized (jobDefinitions) { + if( jobDefinitions.containsKey(jobKey) ) + return jobDefinitions[jobKey] + + def msg + def name = findJobDefinition(request.jobDefinitionName, token) + if( name ) { + msg = "[AWS BATCH] Found job definition name=$name; container=$container" + } + else { + name = registerJobDefinition(request) + msg = "[AWS BATCH] Created job definition name=$name; container=$container" + } + // log the request + if( log.isTraceEnabled() ) + log.debug "[AWS BATCH] $msg; request=${request.toString().indent()}" + else + log.debug "[AWS BATCH] $msg" + + jobDefinitions[jobKey] = name + return name + } + } + + /** + * Create a Batch job definition request for a container image. + * + * @param container + */ + RegisterJobDefinitionRequest makeJobDefRequest(String container) { + // create the job definition request + final hashingTokens = new ArrayList() + final request = makeJobDefRequest0(container, hashingTokens) + + // create a unique id for the job definition + final hash = computeUniqueToken(hashingTokens) + request.setParameters(['nf-token': hash]) + + return request + } + + /** + * Create a unique id from a list of hashable tokens that represent + * a unique object. + * + * @param uniq + */ + private String computeUniqueToken(List uniq) { + return CacheHelper.hasher(uniq).hash().toString() + } + + /** + * Create a job definition request for a container image. + * + * The hashing tokens collect values that should be used to create a + * unique job definition id for the job request. + * + * @param image + * @param hashingTokens + */ + private RegisterJobDefinitionRequest makeJobDefRequest0(String image, List hashingTokens) { + final name = normalizeJobDefinitionName(image) + final opts = getAwsOptions() + + final request = new RegisterJobDefinitionRequest() + request.setJobDefinitionName(name) + request.setType(JobDefinitionType.Container) + + // apply the container options from the task configuration + final containerOpts = task.getConfig().getContainerOptionsMap() + final container = createContainerProperties(containerOpts) + + // set the container configuration + // the actual command, cpus, and memory are overridden when the job is executed + container + .withImage(image) + .withCommand('true') + .withResourceRequirements( + new ResourceRequirement().withType(ResourceType.VCPU).withValue('1'), + new ResourceRequirement().withType(ResourceType.MEMORY).withValue('1024') + ) + + // set the job role + final jobRole = opts.getJobRole() + if( jobRole ) + container.setJobRoleArn(jobRole) + + // set the logs group + final logsGroup = opts.getLogsGroup() + if( logsGroup ) + container.setLogConfiguration(getLogConfiguration(logsGroup, opts.getRegion())) + + // set privilged mode if fusion is enabled + if( fusionEnabled() ) + container.setPrivileged(true) + + // add the volume mounts + final mountsMap = new LinkedHashMap( 10) + if( opts.cliPath ) { + def path = Paths.get(opts.cliPath).parent.parent.toString() + mountsMap.put('aws-cli', "$path:$path:ro") + } + + int c = 0 + final volumes = opts.getVolumes() + for( String vol : volumes ) { + mountsMap.put("vol-${++c}", vol) + } + + if( mountsMap ) + addVolumeMountsToContainer(mountsMap, container) + + // set the container options + request.setContainerProperties(container) + + // add to the hashing tokens all values that contribute to the + // uniqueness of the job definition + hashingTokens.add(name) + hashingTokens.add(container.toString()) + if( containerOpts ) + hashingTokens.add(containerOpts) + + return request + } + + /** + * Normalize a name to be compliant with the Batch job definition format. + * + * @param name + */ + String normalizeJobDefinitionName(String name) { + if( !name ) + return null + if( !ContainerNameValidator.isValidImageName(name) ) + throw new IllegalArgumentException("Invalid container image name: $name") + + def result = name.replaceAll(/[^a-zA-Z0-9\-_]+/,'-') + // Batch job definition length cannot exceed 128 characters + // take first 40 chars + a unique MD5 hash (32 chars) + if( result.length() > 125 ) { + result = result.substring(0,40) + '-' + name.md5() + } + + return "nf-" + result + } + + @Memoized + LogConfiguration getLogConfiguration(String name, String region) { + new LogConfiguration() + .withLogDriver('awslogs') + .withOptions([ + 'awslogs-region': region, + 'awslogs-group': name + ]) + } + + void addVolumeMountsToContainer(Map mountsMap, ContainerProperties container) { + final mounts = new ArrayList(mountsMap.size()) + final volumes = new ArrayList(mountsMap.size()) + for( Map.Entry entry : mountsMap.entrySet() ) { + final mountName = entry.key + final parts = entry.value.tokenize(':') + final containerPath = parts[0] + final hostPath = parts.size() > 1 ? parts[1] : containerPath + final readOnly = parts.size() > 2 ? parts[2]=='ro' : false + if( parts.size() > 3 ) + throw new IllegalArgumentException("Not a valid volume mount syntax: $entry.value") + + def mount = new MountPoint() + .withSourceVolume(mountName) + .withContainerPath(hostPath) + .withReadOnly(readOnly) + mounts << mount + + def vol = new Volume() + .withName(mountName) + .withHost(new Host() + .withSourcePath(containerPath)) + volumes << vol + } + + if( mountsMap ) { + container.setMountPoints(mounts) + container.setVolumes(volumes) + } + } + + /** + * Search for a Batch job definition in ACTIVE status for the given name and job definition id. + * + * @param name + * @param jobId + * @return The fully qualified Batch job definition name, e.g. {@code my-job-definition:3} + */ + String findJobDefinition(String name, String jobId) { + log.trace "[AWS BATCH] checking job definition with name=$name; jobid=$jobId" + final request = new DescribeJobDefinitionsRequest().withJobDefinitionName(name) + // bypass the proxy because this method is invoked during a + // job submit request that's already in a separate thread pool request + // therefore it's private by a TooManyRequestsException + final response = describeJobDefinitions0(bypassProxy(client), request) + final jobs = response.getJobDefinitions() + if( jobs.size() == 0 ) + return null + + def job = jobs.find { JobDefinition it -> it.status == 'ACTIVE' && it.parameters?.'nf-token' == jobId } + return job ? "${name}:${job.revision}" : null + } + + static private DescribeJobDefinitionsResult describeJobDefinitions0(AWSBatch client, DescribeJobDefinitionsRequest req) { + try { + client.describeJobDefinitions(req) + } + catch( AWSBatchException e ) { + if( e.statusCode >= 500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to describe job definitions: ${req.jobDefinitions} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + + /** + * Register a new Batch job definition. + * + * @param request + * @return The fully qualified Batch job definition name, e.g. {@code my-job-definition:3} + */ + String registerJobDefinition(RegisterJobDefinitionRequest request) { + // add nextflow tags + request.addTagsEntry('nextflow.io/createdAt', Instant.now().toString()) + request.addTagsEntry('nextflow.io/version', Const.APP_VER) + // create the job def + final res = registerJobDefinition0(bypassProxy(client), request) // bypass the client proxy! see #1024 + return "${res.jobDefinitionName}:$res.revision" + } + + static private RegisterJobDefinitionResult registerJobDefinition0(AWSBatch client, RegisterJobDefinitionRequest req) { + try { + return client.registerJobDefinition(req) + } + catch( AWSBatchException e ) { + if( e.statusCode >= 500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to register job definition: ${req.jobDefinitionName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + +} diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy index 2f55259b93..a42c12712b 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy @@ -88,18 +88,21 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + maxSpotAttempts >> 5 + } + and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] - 1 * handler.maxSpotAttempts() >> 5 - _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - + and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] req.getRetryStrategy() == new RetryStrategy() @@ -109,18 +112,21 @@ class AwsBatchTaskHandlerTest extends Specification { when: req = handler.newSubmitRequest(task) then: + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + region >> 'eu-west-1' + } + and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] - 1 * handler.maxSpotAttempts() >> 0 - _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'], region: 'eu-west-1')) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - + and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] req.getRetryStrategy() == null // <-- retry is managed by NF, hence this must be null @@ -139,38 +145,28 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + debug >> true + storageEncryption >> 'AES256' + schedulingPriority >> 9999 + shareIdentifier >> 'priority/high' + maxSpotAttempts >> 5 + } + and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] - 1 * handler.maxSpotAttempts() >> 5 - _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'],client: [storageEncryption: 'AES256'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [] - + and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] - - when: - def req2 = handler.newSubmitRequest(task) - then: - 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] - 1 * handler.maxSpotAttempts() >> 5 - _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws',schedulingPriority: 9999,shareIdentifier: 'priority/high'], client:[storageEncryption: 'AES256', debug: true])) } - 1 * handler.getJobQueue(task) >> 'queue1' - 1 * handler.getJobDefinition(task) >> 'job-def:1' - 1 * handler.getEnvironmentVars() >> [] - - req2.getJobName() == 'batchtask' - req2.getJobQueue() == 'queue1' - req2.getJobDefinition() == 'job-def:1' - req2.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req2.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' - req2.getContainerOverrides().getCommand() ==['bash', '-c', 'something'] - req2.getShareIdentifier() == 'priority/high' - req2.getSchedulingPriorityOverride() == 9999 + req.getShareIdentifier() == 'priority/high' + req.getSchedulingPriorityOverride() == 9999 } @@ -186,19 +182,19 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'],region: 'eu-west-1')) } + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + region >> 'eu-west-1' + } and: _ * handler.fusionEnabled() >> false - 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: - def res = req.getContainerOverrides().getResourceRequirements() - res.size()==3 - and: - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '2048' - req.getContainerOverrides().getResourceRequirements().find { it.type=='GPU'}.getValue() == '2' + req.getContainerOverrides().getResourceRequirements().size() == 3 + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '2048' + req.getContainerOverrides().getResourceRequirements().find { it.type=='GPU' }.getValue() == '2' } @@ -213,10 +209,9 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig() - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } + handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } and: _ * handler.fusionEnabled() >> false - 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: @@ -230,10 +225,9 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '5 sec') - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } + handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } and: _ * handler.fusionEnabled() >> false - 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue2' 1 * handler.getJobDefinition(task) >> 'job-def:2' and: @@ -243,23 +237,20 @@ class AwsBatchTaskHandlerTest extends Specification { // minimal allowed timeout is 60 seconds req.getTimeout().getAttemptDurationSeconds() == 60 - when: req = handler.newSubmitRequest(task) then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '1 hour') - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } + handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } and: _ * handler.fusionEnabled() >> false - 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue3' 1 * handler.getJobDefinition(task) >> 'job-def:3' and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue3' req.getJobDefinition() == 'job-def:3' - // minimal allowed timeout is 60 seconds req.getTimeout().getAttemptDurationSeconds() == 3600 } @@ -279,18 +270,22 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws', retryMode: 'adaptive', maxTransferAttempts: 10])) } + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + retryMode >> 'adaptive' + maxTransferAttempts >> 10 + maxSpotAttempts >> 3 + } and: _ * handler.fusionEnabled() >> false 1 * handler.getSubmitCommand() >> ['bash','-c','foo'] - 1 * handler.maxSpotAttempts() >> 3 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - // no error `retry` error strategy is defined by NF, use `maxRetries` to se Batch attempts + // no error `retry` error strategy is defined by NF, use `maxRetries` to set Batch attempts req.getRetryStrategy() == new RetryStrategy() .withAttempts(3) .withEvaluateOnExit( new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*'), new EvaluateOnExit().withOnReason('*').withAction('EXIT') ) @@ -415,16 +410,16 @@ class AwsBatchTaskHandlerTest extends Specification { handler.resolveJobDefinition(IMAGE) then: 1 * handler.makeJobDefRequest(IMAGE) >> req - 1 * handler.findJobDef(JOB_NAME, JOB_ID) >> null - 1 * handler.createJobDef(req) >> null + 1 * handler.findJobDefinition(JOB_NAME, JOB_ID) >> null + 1 * handler.registerJobDefinition(req) >> null when: handler.resolveJobDefinition(IMAGE) then: // second time are not invoked for the same image 1 * handler.makeJobDefRequest(IMAGE) >> req - 0 * handler.findJobDef(JOB_NAME, JOB_ID) >> null - 0 * handler.createJobDef(req) >> null + 0 * handler.findJobDefinition(JOB_NAME, JOB_ID) >> null + 0 * handler.registerJobDefinition(req) >> null } @@ -434,41 +429,42 @@ class AwsBatchTaskHandlerTest extends Specification { def JOB_NAME = 'foo-bar-1-0' def JOB_ID = '123' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = client + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> client + } def req = new DescribeJobDefinitionsRequest().withJobDefinitionName(JOB_NAME) def res = Mock(DescribeJobDefinitionsResult) def job = Mock(JobDefinition) when: - def result = handler.findJobDef(JOB_NAME, JOB_ID) + def result = handler.findJobDefinition(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [] result == null when: - result = handler.findJobDef(JOB_NAME, JOB_ID) + result = handler.findJobDefinition(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] 1 * job.getStatus() >> 'ACTIVE' - 1 * job.getParameters() >> ['nf-token': JOB_ID] + _ * job.getParameters() >> ['nf-token': JOB_ID] 1 * job.getRevision() >> 3 result == "$JOB_NAME:3" when: - result = handler.findJobDef(JOB_NAME, JOB_ID) + result = handler.findJobDefinition(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] 1 * job.getStatus() >> 'ACTIVE' - 1 * job.getParameters() >> [:] + _ * job.getParameters() >> [:] result == null when: - result = handler.findJobDef(JOB_NAME, JOB_ID) + result = handler.findJobDefinition(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] @@ -478,19 +474,20 @@ class AwsBatchTaskHandlerTest extends Specification { } - def 'should create job definition existence' () { + def 'should register job definition' () { given: def JOB_NAME = 'foo-bar-1-0' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = client + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> client + } def req = new RegisterJobDefinitionRequest() def res = Mock(RegisterJobDefinitionResult) when: - def result = handler.createJobDef(req) + def result = handler.registerJobDefinition(req) then: 1 * client.registerJobDefinition(req) >> res 1 * res.getJobDefinitionName() >> JOB_NAME @@ -591,7 +588,7 @@ class AwsBatchTaskHandlerTest extends Specification { given: def IMAGE = 'foo/bar:1.0' def JOB_NAME = 'nf-foo-bar-1-0' - AwsBatchTaskHandler handler = Spy(AwsBatchTaskHandler) { + def handler = Spy(AwsBatchTaskHandler) { getTask() >> Mock(TaskRun) { getConfig() >> Mock(TaskConfig) } fusionEnabled() >> true } @@ -695,8 +692,9 @@ class AwsBatchTaskHandlerTest extends Specification { given: def JOB_ID = 'job-2' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = client + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> client + } def JOB1 = new JobDetail().withJobId('job-1') def JOB2 = new JobDetail().withJobId('job-2') @@ -719,8 +717,9 @@ class AwsBatchTaskHandlerTest extends Specification { def collector = Mock(BatchContext) def JOB_ID = 'job-1' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = client + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> client + } handler.@jobId = JOB_ID handler.batch(collector) @@ -747,8 +746,9 @@ class AwsBatchTaskHandlerTest extends Specification { def collector = Mock(BatchContext) def JOB_ID = 'job-1' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = client + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> client + } handler.@jobId = JOB_ID handler.batch(collector) @@ -771,9 +771,10 @@ class AwsBatchTaskHandlerTest extends Specification { def task = Mock(TaskRun) def client = Mock(AWSBatch) def proxy = Mock(AwsBatchProxy) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = proxy - handler.task = task + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> proxy + getTask() >> task + } def req = Mock(SubmitJobRequest) def resp = Mock(SubmitJobResult) @@ -781,7 +782,6 @@ class AwsBatchTaskHandlerTest extends Specification { when: handler.submit() then: - 1 * handler.buildTaskWrapper() >> null 1 * handler.newSubmitRequest(task) >> req 1 * handler.bypassProxy(proxy) >> client 1 * client.submitJob(req) >> resp @@ -822,8 +822,9 @@ class AwsBatchTaskHandlerTest extends Specification { task.getProcessor() >> processor task.getConfig() >> GroovyMock(TaskConfig) def proxy = Mock(AwsBatchProxy) - def handler = Spy(AwsBatchTaskHandler) - handler.@client = proxy + def handler = Spy(AwsBatchTaskHandler) { + getClient() >> proxy + } handler.task = task handler.@jobId = 'xyz-123' @@ -886,18 +887,21 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: + handler.getAwsOptions() >> Mock(AwsOptions) { + cliPath >> '/bin/aws' + maxSpotAttempts >> 5 + } + and: 1 * handler.getSubmitCommand() >> ['sh','-c','hello'] - 1 * handler.maxSpotAttempts() >> 5 - 1 * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - + and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['sh', '-c','hello'] req.getRetryStrategy() == new RetryStrategy() @@ -906,7 +910,8 @@ class AwsBatchTaskHandlerTest extends Specification { req.getTags() == [a:'b'] req.getPropagateTags() == true } - def 'get fusion submit command' () { + + def 'should create fusion submit command' () { given: def handler = Spy(AwsBatchTaskHandler) { fusionEnabled() >> true From 75135d7924334ab4c8ac378cf331c6ed008414d2 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Sun, 30 Apr 2023 15:26:12 -0500 Subject: [PATCH 14/97] Fix bug in awsbatch + fusion + array job Signed-off-by: Ben Sherman --- .../main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy index ab55d0444a..fe979b335f 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy @@ -80,7 +80,7 @@ class AwsBatchArraySubmitter extends ArrayTaskSubmitter implements SubmitJobAwar bash \${array[\$AWS_BATCH_JOB_ARRAY_INDEX]}/${TaskRun.CMD_RUN} """.stripIndent().trim() - return List.of(FUSION_PATH, 'bash', cmd.toString()) + return List.of(FUSION_PATH, 'bash', '-c', cmd.toString()) } protected List classicSubmitCli() { From 2ee42ae8443dad2d668713f44675f9139ec6076c Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Sun, 30 Apr 2023 15:30:35 -0500 Subject: [PATCH 15/97] Support grid + fusion + array jobs Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 44 +++++-------- .../executor/GridArrayTaskSubmitter.groovy | 66 ++++++++++++++++--- .../nextflow/executor/LsfExecutor.groovy | 13 ++-- .../nextflow/executor/PbsExecutor.groovy | 13 ++-- .../nextflow/executor/SgeExecutor.groovy | 13 ++-- .../nextflow/executor/SlurmExecutor.groovy | 13 ++-- 6 files changed, 105 insertions(+), 57 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 9ab04bce73..2d576abe26 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -94,15 +94,17 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { * @return A multi-line string containing the job directives */ String getHeaders( TaskRun task ) { + getHeaders(getDirectives(task)) + } + String getHeaders( List directives ) { final token = getHeaderToken() def result = new StringBuilder() def header = new ArrayList(2) - def dir = getDirectives(task) - def len = dir.size()-1 - for( int i=0; i ((GridTaskHandler)handler).wrapperFile } - .join(' ') - - final builder = new StringBuilder() - << '#!/bin/bash\n' - << "${headerToken} ${arrayDirective}\n" - << taskHeaders - << "declare -a array=( ${wrapperFiles} )\n" - << "bash \${array[\$${arrayIndexName}]}\n" - - return builder.toString() + String getArrayHeaders(int arraySize, TaskRun task) { + arrayTaskNotSupported() } - protected String getArrayDirective(int arraySize) { - throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + String getArrayIndexName() { + arrayTaskNotSupported() } - protected String getArrayIndexName() { - throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + List getArraySubmitCommandLine() { + arrayTaskNotSupported() } - protected List getArraySubmitCommandLine() { - throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + String getArrayTaskId(String jobId, int index) { + arrayTaskNotSupported() } - protected String getArrayTaskId(String jobId, int index) { + protected void arrayTaskNotSupported() { throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy index fa17d54250..d03ad28f2a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy @@ -20,6 +20,8 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.exception.ProcessFailedException import nextflow.exception.ProcessNonZeroExitStatusException +import nextflow.fusion.FusionAwareTask +import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus @@ -31,7 +33,7 @@ import nextflow.util.CmdLineHelper */ @Slf4j @CompileStatic -class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAware { +class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements FusionAwareTask, SubmitRetryAware { private AbstractGridExecutor executor @@ -51,13 +53,13 @@ class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAw ProcessBuilder builder = null try { // -- create the array job script - final arrayScript = executor.createArrayTaskWrapper(this) + final launcherScript = getLauncherScript() // -- create the submit command builder = createProcessBuilder() // -- submit the array job with a retryable strategy - final result = safeExecute( () -> processStart(builder, arrayScript) ) + final result = safeExecute( () -> processStart(builder, launcherScript) ) final jobId = (String)executor.parseJobId(result) // -- set the job id and status of each task @@ -83,6 +85,52 @@ class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAw throw new ProcessFailedException("Error submitting array job for execution", e) } } + + protected String getLauncherScript() { + return fusionEnabled() + ? fusionLauncherScript() + : classicLauncherScript() + } + + protected String fusionLauncherScript() { + final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() + final fusionWorkDir = FusionHelper.toContainerMount(task.workDir).toString() + final arrayHeaders = executor.getArrayHeaders(array.size(), getTask()) + final arrayIndexName = executor.getArrayIndexName() + final workDirs = array + .collect { handler -> FusionHelper.toContainerMount(handler.task.workDir) } + .join(' ') + + final cmd = FusionHelper.runWithContainer( + fusionLauncher(), + task.getContainerConfig(), + task.getContainer(), + fusionSubmitCli() ) + + final builder = new StringBuilder() + << '#!/bin/bash\n' + << arrayHeaders.replace(remoteLog, '/dev/null') + << "declare -a array=( ${workDirs} )\n" + << cmd.replace(fusionWorkDir, "\${array[\$${arrayIndexName}]}") + + return builder.toString() + } + + protected String classicLauncherScript() { + final arrayHeaders = executor.getArrayHeaders(array.size(), getTask()) + final arrayIndexName = executor.getArrayIndexName() + final workDirs = array + .collect { handler -> handler.task.workDir } + .join(' ') + + final builder = new StringBuilder() + << '#!/bin/bash\n' + << arrayHeaders + << "declare -a array=( ${workDirs} )\n" + << "bash \${array[\$${arrayIndexName}]}/${TaskRun.CMD_RUN}\n" + + return builder.toString() + } protected void setJobId(String jobId) { array.eachWithIndex { handler, i -> @@ -107,19 +155,19 @@ class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAw .redirectErrorStream(true) } - protected String processStart(ProcessBuilder builder, String arrayScript) { + protected String processStart(ProcessBuilder builder, String launcherScript) { final process = builder.start() try { // -- pipe the array job script to the command stdin - log.trace "[${executor.name.toUpperCase()}] Submit array job >\n${arrayScript.indent()}" - process.out << arrayScript + log.trace "[${executor.name.toUpperCase()}] Submit array job >\n${launcherScript.indent()}" + process.out << launcherScript process.out.close() // -- wait for the submission to complete final result = process.text final exitStatus = process.waitFor() - final cmd = launchCmd0(builder, arrayScript) + final cmd = launchCmd0(builder, launcherScript) if( exitStatus ) throw new ProcessNonZeroExitStatusException("Failed to submit array job to grid scheduler for execution", result, exitStatus, cmd) @@ -136,12 +184,12 @@ class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements SubmitRetryAw } } - protected String launchCmd0(ProcessBuilder builder, String arrayScript) { + protected String launchCmd0(ProcessBuilder builder, String launcherScript) { final cmd = CmdLineHelper.toLine(builder.command()) new StringBuilder() .append("cat << 'LAUNCH_COMMAND_EOF' | ${cmd}\n") - .append(arrayScript.trim()) + .append(launcherScript.trim()) .append('\nLAUNCH_COMMAND_EOF\n') .toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index a7cd8f2031..97147fae45 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -305,16 +305,19 @@ class LsfExecutor extends AbstractGridExecutor { private AtomicInteger arrayTaskCount = new AtomicInteger() @Override - protected String getArrayDirective(int arraySize) { - "-J \"nf-array-${arrayTaskCount.getAndIncrement()}[0-${arraySize - 1}]\"" + String getArrayHeaders(int arraySize, TaskRun task) { + final directives = getDirectives(task) + << '-J' << "nf-array-${arrayTaskCount.getAndIncrement()}[0-${arraySize - 1}]" + + getHeaders(directives) } @Override - protected String getArrayIndexName() { 'LSB_JOBINDEX' } + String getArrayIndexName() { 'LSB_JOBINDEX' } @Override - protected List getArraySubmitCommandLine() { List.of('bsub') } + List getArraySubmitCommandLine() { List.of('bsub') } @Override - protected String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } + String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index f131c8135a..c65f4c524d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -174,18 +174,21 @@ class PbsExecutor extends AbstractGridExecutor { } @Override - protected String getArrayDirective(int arraySize) { - "-J 0-${arraySize - 1}" + String getArrayHeaders(int arraySize, TaskRun task) { + final directives = getDirectives(task) + << '-J' << "0-${arraySize - 1}" + + getHeaders(directives) } @Override - protected String getArrayIndexName() { 'PBS_ARRAY_INDEX' } + String getArrayIndexName() { 'PBS_ARRAY_INDEX' } @Override - protected List getArraySubmitCommandLine() { List.of('qsub') } + List getArraySubmitCommandLine() { List.of('qsub') } @Override - protected String getArrayTaskId(String jobId, int index) { + String getArrayTaskId(String jobId, int index) { jobId.replace('[]', "[$index]") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index b8081c62a7..7f005b5292 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -185,16 +185,19 @@ class SgeExecutor extends AbstractGridExecutor { } @Override - protected String getArrayDirective(int arraySize) { - "-t 0-${arraySize - 1}" + String getArrayHeaders(int arraySize, TaskRun task) { + final directives = getDirectives(task) + << '-t' << "0-${arraySize - 1}" + + getHeaders(directives) } @Override - protected String getArrayIndexName() { 'SGE_TASK_ID' } + String getArrayIndexName() { 'SGE_TASK_ID' } @Override - protected List getArraySubmitCommandLine() { List.of('qsub', '-') } + List getArraySubmitCommandLine() { List.of('qsub', '-') } @Override - protected String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } + String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 24250d4de0..b0da60e9c7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -201,16 +201,19 @@ class SlurmExecutor extends AbstractGridExecutor { } @Override - protected String getArrayDirective(int arraySize) { - "--array 0-${arraySize - 1}" + String getArrayHeaders(int arraySize, TaskRun task) { + final directives = getDirectives(task) + << '--array' << "0-${arraySize - 1}" + + getHeaders(directives) } @Override - protected String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } + String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } @Override - protected List getArraySubmitCommandLine() { List.of('sbatch') } + List getArraySubmitCommandLine() { List.of('sbatch') } @Override - protected String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } + String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } } From 3d84594bae57fbdf4d995de3f2c43c5544a63c95 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 1 May 2023 06:40:08 -0500 Subject: [PATCH 16/97] Refactor ArrayTask* to TaskArray* Signed-off-by: Ben Sherman --- .../nextflow/executor/AbstractGridExecutor.groovy | 6 +++--- ...kSubmitter.groovy => GridTaskArraySubmitter.groovy} | 4 ++-- .../{ArrayTaskAware.groovy => TaskArrayAware.groovy} | 6 +++--- ...yTaskSubmitter.groovy => TaskArraySubmitter.groovy} | 4 ++-- .../nextflow/executor/local/LocalExecutor.groovy | 4 ++-- ...yTaskCollector.groovy => TaskArrayCollector.groovy} | 10 +++++----- .../main/groovy/nextflow/processor/TaskHandler.groovy | 4 ++-- .../groovy/nextflow/processor/TaskProcessor.groovy | 4 ++-- .../nextflow/cloud/aws/batch/AwsBatchExecutor.groovy | 10 +++++----- ...mitter.groovy => AwsBatchTaskArraySubmitter.groovy} | 6 +++--- 10 files changed, 29 insertions(+), 29 deletions(-) rename modules/nextflow/src/main/groovy/nextflow/executor/{GridArrayTaskSubmitter.groovy => GridTaskArraySubmitter.groovy} (98%) rename modules/nextflow/src/main/groovy/nextflow/executor/{ArrayTaskAware.groovy => TaskArrayAware.groovy} (87%) rename modules/nextflow/src/main/groovy/nextflow/executor/{ArrayTaskSubmitter.groovy => TaskArraySubmitter.groovy} (95%) rename modules/nextflow/src/main/groovy/nextflow/processor/{ArrayTaskCollector.groovy => TaskArrayCollector.groovy} (92%) rename plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/{AwsBatchArraySubmitter.groovy => AwsBatchTaskArraySubmitter.groovy} (95%) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 2d576abe26..328fe9684f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -37,7 +37,7 @@ import org.apache.commons.lang.StringUtils */ @Slf4j @CompileStatic -abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { +abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { protected Duration queueInterval @@ -412,8 +412,8 @@ abstract class AbstractGridExecutor extends Executor implements ArrayTaskAware { } @Override - ArrayTaskSubmitter createArrayTaskSubmitter(List array) { - new GridArrayTaskSubmitter(array, this) + TaskArraySubmitter createArrayTaskSubmitter(List array) { + new GridTaskArraySubmitter(array, this) } String getArrayHeaders(int arraySize, TaskRun task) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy similarity index 98% rename from modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index d03ad28f2a..243abbd8bb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridArrayTaskSubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -33,11 +33,11 @@ import nextflow.util.CmdLineHelper */ @Slf4j @CompileStatic -class GridArrayTaskSubmitter extends ArrayTaskSubmitter implements FusionAwareTask, SubmitRetryAware { +class GridTaskArraySubmitter extends TaskArraySubmitter implements FusionAwareTask, SubmitRetryAware { private AbstractGridExecutor executor - GridArrayTaskSubmitter(List array, AbstractGridExecutor executor) { + GridTaskArraySubmitter(List array, AbstractGridExecutor executor) { super(array) this.executor = executor } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy similarity index 87% rename from modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index 5cc7d5d4bc..e3b90a3f80 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -23,15 +23,15 @@ import nextflow.processor.TaskHandler * * @author Ben Sherman */ -trait ArrayTaskAware { +trait TaskArrayAware { /** * Create a submitter for an array job. * * @param array */ - ArrayTaskSubmitter createArrayTaskSubmitter(List array) { - new ArrayTaskSubmitter(array) + TaskArraySubmitter createArrayTaskSubmitter(List array) { + new TaskArraySubmitter(array) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy similarity index 95% rename from modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index 16464e6b62..a808a6336c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/ArrayTaskSubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -29,13 +29,13 @@ import nextflow.processor.TaskHandler */ @Slf4j @CompileStatic -class ArrayTaskSubmitter { +class TaskArraySubmitter { protected List array private AtomicInteger collected = new AtomicInteger() - ArrayTaskSubmitter(List array) { + TaskArraySubmitter(List array) { this.array = array for( TaskHandler handler : array ) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy index d499a1d60d..2e98718ced 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy @@ -19,7 +19,7 @@ package nextflow.executor.local import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.executor.ArrayTaskAware +import nextflow.executor.TaskArrayAware import nextflow.executor.Executor import nextflow.executor.SupportedScriptTypes import nextflow.fusion.FusionHelper @@ -36,7 +36,7 @@ import nextflow.script.ScriptType @Slf4j @CompileStatic @SupportedScriptTypes( [ScriptType.SCRIPTLET, ScriptType.GROOVY] ) -class LocalExecutor extends Executor implements ArrayTaskAware { +class LocalExecutor extends Executor implements TaskArrayAware { @Override protected TaskMonitor createTaskMonitor() { diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy similarity index 92% rename from modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy rename to modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 8143874963..c35ef1ecb2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ArrayTaskCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -22,7 +22,7 @@ import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.executor.ArrayTaskAware +import nextflow.executor.TaskArrayAware import nextflow.executor.Executor /** @@ -33,7 +33,7 @@ import nextflow.executor.Executor */ @Slf4j @CompileStatic -class ArrayTaskCollector { +class TaskArrayCollector { private Executor executor @@ -47,8 +47,8 @@ class ArrayTaskCollector { private boolean closed = false - ArrayTaskCollector(Executor executor, int arraySize) { - if( executor !instanceof ArrayTaskAware ) + TaskArrayCollector(Executor executor, int arraySize) { + if( executor !instanceof TaskArrayAware ) throw new IllegalArgumentException("Executor '${executor.name}' does not support array jobs") this.executor = executor @@ -111,7 +111,7 @@ class ArrayTaskCollector { protected void submit0(List array) { // create submitter for array job - ((ArrayTaskAware)executor).createArrayTaskSubmitter(array) + ((TaskArrayAware)executor).createArrayTaskSubmitter(array) // submit each task to the underlying monitor for( TaskHandler handler : array ) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 413863f962..cf22c5b9b9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -22,7 +22,7 @@ import java.nio.file.NoSuchFileException import java.nio.file.Path import groovy.util.logging.Slf4j -import nextflow.executor.ArrayTaskSubmitter +import nextflow.executor.TaskArraySubmitter import nextflow.trace.TraceRecord /** * Actions to handle the underlying job running the user task. @@ -56,7 +56,7 @@ abstract class TaskHandler { /** * Submitter for array jobs */ - ArrayTaskSubmitter arraySubmitter + TaskArraySubmitter arraySubmitter /** * Task current status diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index c9a043c007..e5bf9f2a24 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -253,7 +253,7 @@ class TaskProcessor { private Boolean isFair0 - private ArrayTaskCollector arrayCollector + private TaskArrayCollector arrayCollector private CompilerConfiguration compilerConfig() { final config = new CompilerConfiguration() @@ -311,7 +311,7 @@ class TaskProcessor { this.isFair0 = config.getFair() final arraySize = config.getArray() - this.arrayCollector = arraySize > 0 ? new ArrayTaskCollector(executor, arraySize) : null + this.arrayCollector = arraySize > 0 ? new TaskArrayCollector(executor, arraySize) : null } /** diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 915886da2f..782ce73347 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -32,8 +32,8 @@ import nextflow.cloud.aws.AwsClientFactory import nextflow.cloud.aws.config.AwsConfig import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException -import nextflow.executor.ArrayTaskAware -import nextflow.executor.ArrayTaskSubmitter +import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArraySubmitter import nextflow.executor.Executor import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx @@ -56,7 +56,7 @@ import org.pf4j.ExtensionPoint @Slf4j @ServiceName('awsbatch') @CompileStatic -class AwsBatchExecutor extends Executor implements ExtensionPoint, ArrayTaskAware { +class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAware { /** * Proxy to throttle AWS batch client requests @@ -222,8 +222,8 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, ArrayTaskAwar } @Override - ArrayTaskSubmitter createArrayTaskSubmitter(List array) { - new AwsBatchArraySubmitter(array, this) + TaskArraySubmitter createArrayTaskSubmitter(List array) { + new AwsBatchTaskArraySubmitter(array, this) } /** diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy similarity index 95% rename from plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy rename to plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy index fe979b335f..3e59fab6fa 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchArraySubmitter.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy @@ -27,7 +27,7 @@ import com.amazonaws.services.batch.model.SubmitJobResult import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.exception.ProcessSubmitException -import nextflow.executor.ArrayTaskSubmitter +import nextflow.executor.TaskArraySubmitter import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus @@ -38,13 +38,13 @@ import nextflow.processor.TaskStatus */ @Slf4j @CompileStatic -class AwsBatchArraySubmitter extends ArrayTaskSubmitter implements SubmitJobAware { +class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAware { private AwsBatchExecutor executor private AWSBatch client - AwsBatchArraySubmitter(List array, AwsBatchExecutor executor) { + AwsBatchTaskArraySubmitter(List array, AwsBatchExecutor executor) { super(array) this.executor = executor this.client = executor.client From fc7847034c5378fc6fd677798f8a47fc2e80bc51 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 1 May 2023 07:33:12 -0500 Subject: [PATCH 17/97] Reduce code duplication Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 22 ++---- .../nextflow/executor/BridgeExecutor.groovy | 13 +--- .../nextflow/executor/CondorExecutor.groovy | 4 +- .../nextflow/executor/FluxExecutor.groovy | 9 +-- .../executor/GridTaskArraySubmitter.groovy | 58 +-------------- .../nextflow/executor/GridTaskHandler.groovy | 68 ++---------------- .../executor/HyperQueueExecutor.groovy | 9 ++- .../nextflow/executor/LsfExecutor.groovy | 15 +--- .../nextflow/executor/MoabExecutor.groovy | 16 +---- .../nextflow/executor/NqsiiExecutor.groovy | 9 +-- .../nextflow/executor/OarExecutor.groovy | 11 +-- .../nextflow/executor/PbsExecutor.groovy | 17 ++--- .../nextflow/executor/SgeExecutor.groovy | 11 +-- .../nextflow/executor/SlurmExecutor.groovy | 14 +--- ...etryAware.groovy => SubmitJobAware.groovy} | 72 +++++++++++++++++-- .../nextflow/executor/TaskArrayAware.groovy | 19 ++++- .../processor/TaskArrayCollector.groovy | 6 +- .../nextflow/processor/TaskConfig.groovy | 2 +- .../nextflow/processor/TaskHandler.groovy | 2 +- .../executor/GridTaskHandlerTest.groovy | 15 ++++ .../executor/SubmitRetryAwareTest.groovy | 43 ----------- .../cloud/aws/batch/AwsBatchExecutor.groovy | 16 +++-- .../batch/AwsBatchTaskArraySubmitter.groovy | 12 ++-- .../aws/batch/AwsBatchTaskHandler.groovy | 2 +- 24 files changed, 165 insertions(+), 300 deletions(-) rename modules/nextflow/src/main/groovy/nextflow/executor/{SubmitRetryAware.groovy => SubmitJobAware.groovy} (60%) delete mode 100644 modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 328fe9684f..29ff2ccb1b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -196,7 +196,11 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { * @param task The task instance descriptor * @return A list holding the command line */ - abstract List getSubmitCommandLine(TaskRun task, Path scriptFile) + List getSubmitCommandLine(TaskRun task, Path scriptFile) { + getSubmitCommandLine(task, scriptFile, pipeLauncherScript()) + } + + abstract List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) /** * Defines how script is run the by the grid-engine. @@ -417,22 +421,6 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { } String getArrayHeaders(int arraySize, TaskRun task) { - arrayTaskNotSupported() - } - - String getArrayIndexName() { - arrayTaskNotSupported() - } - - List getArraySubmitCommandLine() { - arrayTaskNotSupported() - } - - String getArrayTaskId(String jobId, int index) { - arrayTaskNotSupported() - } - - protected void arrayTaskNotSupported() { throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy index 920f20afce..aa48fb0721 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy @@ -85,18 +85,9 @@ class BridgeExecutor extends AbstractGridExecutor { String getHeaderToken() { '#MSUB' } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { - - ['ccc_msub', scriptFile.getName()] - + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List.of('ccc_msub', scriptFile.getName()) } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy index 03a161bb3f..a52cab4af0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy @@ -88,8 +88,8 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile) { - return ['condor_submit', '--terse', CMD_CONDOR] + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List.of('condor_submit', '--terse', CMD_CONDOR) } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy index 37d7a4e1bd..dc5cb9f526 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy @@ -48,15 +48,8 @@ class FluxExecutor extends AbstractGridExecutor { // Flux does not require a special token or header String getHeaderToken() { null } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { List result = ['flux', 'mini', 'submit'] result << '--setattr=cwd=' + quote(task.workDir) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index 243abbd8bb..59bcb52a40 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -20,7 +20,6 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.exception.ProcessFailedException import nextflow.exception.ProcessNonZeroExitStatusException -import nextflow.fusion.FusionAwareTask import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun @@ -33,7 +32,7 @@ import nextflow.util.CmdLineHelper */ @Slf4j @CompileStatic -class GridTaskArraySubmitter extends TaskArraySubmitter implements FusionAwareTask, SubmitRetryAware { +class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAware { private AbstractGridExecutor executor @@ -43,7 +42,7 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements FusionAwareTa } @Override - Executor getExecutor() { executor } + AbstractGridExecutor getExecutor() { executor } @Override TaskRun getTask() { array.first().getTask() } @@ -56,7 +55,7 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements FusionAwareTa final launcherScript = getLauncherScript() // -- create the submit command - builder = createProcessBuilder() + builder = createProcessBuilder(true) // -- submit the array job with a retryable strategy final result = safeExecute( () -> processStart(builder, launcherScript) ) @@ -143,55 +142,4 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements FusionAwareTa handler.setStatus(status) } - protected ProcessBuilder createProcessBuilder() { - - // -- log the submit command - final cli = executor.getArraySubmitCommandLine() - log.trace "[${executor.name.toUpperCase()}] Submit array job > cli: ${cli}" - - // -- launch array job script - new ProcessBuilder() - .command( cli as String[] ) - .redirectErrorStream(true) - } - - protected String processStart(ProcessBuilder builder, String launcherScript) { - final process = builder.start() - - try { - // -- pipe the array job script to the command stdin - log.trace "[${executor.name.toUpperCase()}] Submit array job >\n${launcherScript.indent()}" - process.out << launcherScript - process.out.close() - - // -- wait for the submission to complete - final result = process.text - final exitStatus = process.waitFor() - final cmd = launchCmd0(builder, launcherScript) - - if( exitStatus ) - throw new ProcessNonZeroExitStatusException("Failed to submit array job to grid scheduler for execution", result, exitStatus, cmd) - - // -- return the process stdout - return result - } - finally { - // make sure to release all resources - process.in.closeQuietly() - process.out.closeQuietly() - process.err.closeQuietly() - process.destroy() - } - } - - protected String launchCmd0(ProcessBuilder builder, String launcherScript) { - final cmd = CmdLineHelper.toLine(builder.command()) - - new StringBuilder() - .append("cat << 'LAUNCH_COMMAND_EOF' | ${cmd}\n") - .append(launcherScript.trim()) - .append('\nLAUNCH_COMMAND_EOF\n') - .toString() - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 77ad2563aa..62ceecad2d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -26,7 +26,6 @@ import nextflow.exception.ProcessException import nextflow.exception.ProcessFailedException import nextflow.exception.ProcessNonZeroExitStatusException import nextflow.file.FileHelper -import nextflow.fusion.FusionAwareTask import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun @@ -38,7 +37,7 @@ import nextflow.util.Throttle * Handles a job execution in the underlying grid platform */ @Slf4j -class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetryAware { +class GridTaskHandler extends TaskHandler implements SubmitJobAware { /** The target executor platform */ final AbstractGridExecutor executor @@ -89,58 +88,11 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr this.sanityCheckInterval = duration } - protected ProcessBuilder createProcessBuilder() { - - // -- log the qsub command - final cli = executor.getSubmitCommandLine(task, wrapperFile) - log.trace "start process ${task.name} > cli: ${cli}" - - /* - * launch 'sub' script wrapper - */ - ProcessBuilder builder = new ProcessBuilder() - .command( cli as String[] ) - .redirectErrorStream(true) - if( !fusionEnabled() ) - builder .directory(task.workDir.toFile()) - - return builder - } - - protected String processStart(ProcessBuilder builder, String pipeScript) { - final process = builder.start() - - try { - // -- forward the job launcher script to the command stdin if required - if( pipeScript ) { - log.trace "[${executor.name.toUpperCase()}] Submit STDIN command ${task.name} >\n${pipeScript.indent()}" - process.out << pipeScript - process.out.close() - } - - // -- wait the the process completes - final result = process.text - final exitStatus = process.waitFor() - final cmd = launchCmd0(builder,pipeScript) - - if( exitStatus ) { - throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) - } - - // -- return the process stdout - return result - } - finally { - // make sure to release all resources - process.in.closeQuietly() - process.out.closeQuietly() - process.err.closeQuietly() - process.destroy() - } - } + @Override + AbstractGridExecutor getExecutor() { executor } @Override - Path prepareLauncher() { + void prepareLauncher() { // -- create the wrapper script createTaskWrapper(task).build() } @@ -174,16 +126,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr return result } - protected String launchCmd0(ProcessBuilder builder, String pipeScript) { - def result = CmdLineHelper.toLine(builder.command()) - if( pipeScript ) { - result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" - result += pipeScript.trim() + '\n' - result += 'LAUNCH_COMMAND_EOF\n' - } - return result - } - /* * {@inheritDocs} */ @@ -197,7 +139,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask, SubmitRetr ProcessBuilder builder = null try { // -- start the execution and notify the event to the monitor - builder = createProcessBuilder() + builder = createProcessBuilder(executor.pipeLauncherScript()) // -- forward the job launcher script to the command stdin if required final stdinScript = executor.pipeLauncherScript() ? stdinLauncherScript() : null // -- execute with a re-triable strategy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index 0805c16609..4ffb0fd204 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -23,7 +23,6 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.processor.TaskRun import nextflow.util.ServiceName -import nextflow.util.TupleHelper /** * Implements executor for HyperQueue * https://github.com/It4innovations/hyperqueue/ @@ -81,8 +80,8 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile) { - return TupleHelper.listOf('hq', '--output-mode=quiet', 'submit', '--directives=file', scriptFile.getName()) + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + return List.of('hq', '--output-mode=quiet', 'submit', '--directives=file', scriptFile.getName()) } @Override @@ -97,7 +96,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { @Override protected List getKillCommand() { - return TupleHelper.listOf('hq', 'job', 'cancel') + return List.of('hq', 'job', 'cancel') } @Override @@ -114,7 +113,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { @Override protected List queueStatusCommand(Object queue) { - return TupleHelper.listOf('hq', '--output-mode=quiet', 'job', 'list', '--all') + return List.of('hq', '--output-mode=quiet', 'job', 'list', '--all') } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 97147fae45..1b8f8eb926 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -111,16 +111,10 @@ class LsfExecutor extends AbstractGridExecutor { return result } - - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { ['bsub'] } + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List.of('bsub') + } /** * @return {@code true} since BSC grid requires the script to be piped to the {@code bsub} command @@ -315,9 +309,6 @@ class LsfExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'LSB_JOBINDEX' } - @Override - List getArraySubmitCommandLine() { List.of('bsub') } - @Override String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy index 21c47af000..d37904c6d7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy @@ -75,19 +75,9 @@ class MoabExecutor extends AbstractGridExecutor { return result } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { - def cmd = new ArrayList(5) - cmd << 'msub' - cmd << '--xml' - cmd << scriptFile.name - return cmd + @Override + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List.of('msub', '--xml', scriptFile.name) } protected String getHeaderToken() { '#MSUB' } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy index 0a60e17485..19e13165e6 100755 --- a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy @@ -76,12 +76,9 @@ class NqsiiExecutor extends AbstractGridExecutor { return result } - - /* - * Prepare the 'qsub' cmdline - */ - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { - return ['qsub', scriptFile.name] + @Override + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List.of('qsub', scriptFile.name) } protected String getHeaderToken() { '#PBS' } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy index e8a9820a3b..d6933fb1be 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy @@ -90,18 +90,11 @@ class OarExecutor extends AbstractGridExecutor { String getHeaderToken() { '#OAR' } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { // Scripts need to be executable scriptFile.setPermissions(7,0,0) - return ["oarsub", "-S", "./${scriptFile.getName()}"] + List.of("oarsub", "-S", "./${scriptFile.getName()}") } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index c65f4c524d..724a241c01 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -88,17 +88,13 @@ class PbsExecutor extends AbstractGridExecutor { name.size()>15 ? name.substring(0,15) : name } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + @Override + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { // in some PBS implementation the submit command will fail if the script name starts with a dot eg `.command.run` // add the `-N ` to fix this -- see issue #228 - [ 'qsub', '-N', getJobNameFor(task), scriptFile.getName() ] + pipeLauncherScript + ? List.of('qsub') + : List.of('qsub', '-N', getJobNameFor(task), scriptFile.getName()) } protected String getHeaderToken() { '#PBS' } @@ -184,9 +180,6 @@ class PbsExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'PBS_ARRAY_INDEX' } - @Override - List getArraySubmitCommandLine() { List.of('qsub') } - @Override String getArrayTaskId(String jobId, int index) { jobId.replace('[]', "[$index]") diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 7f005b5292..ee62bed2e5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -79,16 +79,14 @@ class SgeExecutor extends AbstractGridExecutor { return result } - /* - * Prepare the 'qsub' cmdline - */ - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + @Override + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { // The '-terse' command line control the output of the qsub command line, when // used it only return the ID of the submitted job. // NOTE: In some SGE implementations the '-terse' only works on the qsub command line // and it is ignored when used in the script job as directive, fir this reason it // should not be remove from here - return pipeLauncherScript() + return pipeLauncherScript ? List.of('qsub', '-') : List.of('qsub', '-terse', scriptFile.name) } @@ -195,9 +193,6 @@ class SgeExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'SGE_TASK_ID' } - @Override - List getArraySubmitCommandLine() { List.of('qsub', '-') } - @Override String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index b0da60e9c7..6d911ddce7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -94,16 +94,9 @@ class SlurmExecutor extends AbstractGridExecutor { @Override String getHeaderToken() { '#SBATCH' } - /** - * The command line to submit this job - * - * @param task The {@link TaskRun} instance to submit for execution to the cluster - * @param scriptFile The file containing the job launcher script - * @return A list representing the submit command line - */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile ) { - return pipeLauncherScript() + List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + return pipeLauncherScript ? List.of('sbatch') : List.of('sbatch', scriptFile.getName()) } @@ -211,9 +204,6 @@ class SlurmExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } - @Override - List getArraySubmitCommandLine() { List.of('sbatch') } - @Override String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy similarity index 60% rename from modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy index c209fa5d99..968c541383 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitRetryAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy @@ -29,21 +29,85 @@ import groovy.transform.Memoized import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.exception.ProcessNonZeroExitStatusException +import nextflow.fusion.FusionAwareTask import nextflow.processor.TaskRun +import nextflow.util.CmdLineHelper import nextflow.util.Duration +import org.slf4j.Logger +import org.slf4j.LoggerFactory /** - * Generic retry-able submit implementation for executors. + * Implementation of job submission for grid executors. * * @author Paolo Di Tommaso */ -@Slf4j @CompileStatic -trait SubmitRetryAware { +trait SubmitJobAware extends FusionAwareTask { - abstract Executor getExecutor() + static private Logger log = LoggerFactory.getLogger(SubmitJobAware) + + abstract AbstractGridExecutor getExecutor() abstract TaskRun getTask() + ProcessBuilder createProcessBuilder(boolean pipeLauncherScript) { + + // -- log the submit command line + final cli = executor.getSubmitCommandLine(task, task.workDir.resolve(TaskRun.CMD_RUN), pipeLauncherScript) + log.trace "start process ${task.name} > cli: ${cli}" + + // -- create the submit process + ProcessBuilder builder = new ProcessBuilder() + .command( cli as String[] ) + .redirectErrorStream(true) + + if( !pipeLauncherScript ) + builder .directory(task.workDir.toFile()) + + return builder + } + + String processStart(ProcessBuilder builder, String pipeScript) { + final process = builder.start() + + try { + // -- forward the job launcher script to the command stdin if required + if( pipeScript ) { + log.trace "[${executor.name.toUpperCase()}] Submit STDIN command ${task.name} >\n${pipeScript.indent()}" + process.out << pipeScript + process.out.close() + } + + // -- wait the the process completes + final result = process.text + final exitStatus = process.waitFor() + final cmd = launchCmd0(builder,pipeScript) + + if( exitStatus ) { + throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) + } + + // -- return the process stdout + return result + } + finally { + // make sure to release all resources + process.in.closeQuietly() + process.out.closeQuietly() + process.err.closeQuietly() + process.destroy() + } + } + + private String launchCmd0(ProcessBuilder builder, String pipeScript) { + def result = CmdLineHelper.toLine(builder.command()) + if( pipeScript ) { + result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" + result += pipeScript.trim() + '\n' + result += 'LAUNCH_COMMAND_EOF\n' + } + return result + } + T safeExecute(CheckedSupplier action) { Failsafe.with(retryPolicy()).get(action) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index e3b90a3f80..aa371df384 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -17,21 +17,36 @@ package nextflow.executor import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun /** * Interface for executors that support array jobs. * * @author Ben Sherman */ -trait TaskArrayAware { +interface TaskArrayAware { + + String getName() + + void submit( TaskRun task ) + + TaskHandler createTaskHandler(TaskRun task) /** * Create a submitter for an array job. * * @param array */ - TaskArraySubmitter createArrayTaskSubmitter(List array) { + default TaskArraySubmitter createArrayTaskSubmitter(List array) { new TaskArraySubmitter(array) } + default String getArrayIndexName() { + throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") + } + + default String getArrayTaskId(String jobId, int index) { + throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index c35ef1ecb2..19712a9713 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -35,7 +35,7 @@ import nextflow.executor.Executor @CompileStatic class TaskArrayCollector { - private Executor executor + private TaskArrayAware executor private TaskMonitor monitor @@ -51,7 +51,7 @@ class TaskArrayCollector { if( executor !instanceof TaskArrayAware ) throw new IllegalArgumentException("Executor '${executor.name}' does not support array jobs") - this.executor = executor + this.executor = (TaskArrayAware)executor this.monitor = executor.monitor this.arraySize = arraySize this.array = new ArrayList<>(arraySize) @@ -111,7 +111,7 @@ class TaskArrayCollector { protected void submit0(List array) { // create submitter for array job - ((TaskArrayAware)executor).createArrayTaskSubmitter(array) + executor.createArrayTaskSubmitter(array) // submit each task to the underlying monitor for( TaskHandler handler : array ) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy index 981737dec2..8e2c599915 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskConfig.groovy @@ -42,7 +42,7 @@ import nextflow.util.MemoryUnit @CompileStatic class TaskConfig extends LazyMap implements Cloneable { - static public final int EXIT_ZERO = 0 + static public final EXIT_ZERO = 0 private transient Map cache = new LinkedHashMap(20) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index cf22c5b9b9..3e09703438 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -98,7 +98,7 @@ abstract class TaskHandler { * This method is optional. If it is not implemented, the launcher script should * be prepared in the submit() method. */ - Path prepareLauncher() { return null } + void prepareLauncher() {} /** * Task status attribute setter. diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 96dc2b485f..01a028e05a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -35,6 +35,21 @@ import test.TestHelper */ class GridTaskHandlerTest extends Specification { + def 'should check retry predicate' () { + given: + def handler = new GridTaskHandler() + + when: + def predicate = handler.retryCondition("Socket timed out") + then: + predicate.test(new ProcessNonZeroExitStatusException('Error', 'Socket timed out', 1, null)) + and: + predicate.test(new ProcessNonZeroExitStatusException('Error', 'error\nBatch job submission failed\nSocket timed out on send/recv operation', 1, null)) + and: + !predicate.test(new ProcessNonZeroExitStatusException('Error', 'OK', 0, null)) + + } + def 'should capture error cause' () { given: def task = new TaskRun(name: 'foo', workDir: Paths.get('/some/work')) diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy deleted file mode 100644 index 7c220c4c19..0000000000 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SubmitRetryAwareTest.groovy +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ - -package nextflow.executor - -import nextflow.exception.ProcessNonZeroExitStatusException -import spock.lang.Specification -/** - * - * @author Paolo Di Tommaso - */ -class SubmitRetryAwareTest extends Specification { - - def 'should check retry predicate' () { - given: - def retry = [:] as SubmitRetryAware - - when: - def predicate = retry.retryCondition("Socket timed out") - then: - predicate.test(new ProcessNonZeroExitStatusException('Error', 'Socket timed out', 1, null)) - and: - predicate.test(new ProcessNonZeroExitStatusException('Error', 'error\nBatch job submission failed\nSocket timed out on send/recv operation', 1, null)) - and: - !predicate.test(new ProcessNonZeroExitStatusException('Error', 'OK', 0, null)) - - } - -} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 782ce73347..616460965e 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -221,11 +221,6 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar new AwsBatchTaskHandler(task, this) } - @Override - TaskArraySubmitter createArrayTaskSubmitter(List array) { - new AwsBatchTaskArraySubmitter(array, this) - } - /** * @return Creates a {@link ThrottlingExecutor} service to throttle * the API requests to the AWS Batch service. @@ -313,6 +308,17 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar ThreadPoolHelper.await(reaper, Duration.of('60min'), waitMsg, exitMsg) } + @Override + TaskArraySubmitter createArrayTaskSubmitter(List array) { + new AwsBatchTaskArraySubmitter(array, this) + } + + @Override + String getArrayIndexName() { 'AWS_BATCH_JOB_ARRAY_INDEX' } + + @Override + String getArrayTaskId(String jobId, int index) { "${jobId}:${index}" } + } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy index 3e59fab6fa..89927ac2ff 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy @@ -74,10 +74,11 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob final workDirs = array .collect { handler -> toContainerMount(handler.task.workDir) } .join(' ') + final arrayIndexName = executor.getArrayIndexName() final cmd = """ declare -a array=( ${workDirs} ) - bash \${array[\$AWS_BATCH_JOB_ARRAY_INDEX]}/${TaskRun.CMD_RUN} + bash \${array[\$${arrayIndexName}]}/${TaskRun.CMD_RUN} """.stripIndent().trim() return List.of(FUSION_PATH, 'bash', '-c', cmd.toString()) @@ -87,6 +88,7 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob final workDirs = array .collect { handler -> handler.task.workDir.toUriString() } .join(' ') + final arrayIndexName = executor.getArrayIndexName() final opts = getAwsOptions() final cli = opts.getAwsCli() @@ -97,7 +99,7 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob final cmd = """ declare -a array=( ${workDirs} ) - task_dir=\${array[\$AWS_BATCH_JOB_ARRAY_INDEX]} + task_dir=\${array[\$${arrayIndexName}]} trap "{ ret=\$?; ${aws} ${TaskRun.CMD_LOG} \$task_dir/${TaskRun.CMD_LOG}||true; exit \$ret; }" EXIT ${aws} \$task_dir/${TaskRun.CMD_RUN} - | bash 2>&1 | tee ${TaskRun.CMD_LOG} """.stripIndent().trim() @@ -117,7 +119,7 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob // -- set the job id, queue, and status of each task array.eachWithIndex { handler, i -> - ((AwsBatchTaskHandler)handler).jobId = getArrayTaskId(jobId, i) + ((AwsBatchTaskHandler)handler).jobId = executor.getArrayTaskId(jobId, i) ((AwsBatchTaskHandler)handler).queueName = request.getJobQueue() handler.status = TaskStatus.SUBMITTED } @@ -138,9 +140,5 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob throw e } } - - String getArrayTaskId(String jobId, int index) { - "${jobId}:${index}" - } } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 9becc141f4..cfc4b91c53 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -129,7 +129,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler Date: Mon, 1 May 2023 07:47:50 -0500 Subject: [PATCH 18/97] minor edits Signed-off-by: Ben Sherman --- .../groovy/nextflow/executor/AbstractGridExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/LsfExecutor.groovy | 6 +++--- .../src/main/groovy/nextflow/executor/TaskArrayAware.groovy | 2 +- .../groovy/nextflow/processor/TaskArrayCollector.groovy | 2 +- .../main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 29ff2ccb1b..d58520754f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -416,7 +416,7 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { } @Override - TaskArraySubmitter createArrayTaskSubmitter(List array) { + TaskArraySubmitter createTaskArraySubmitter(List array) { new GridTaskArraySubmitter(array, this) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 1b8f8eb926..ba24ef6823 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -41,6 +41,8 @@ class LsfExecutor extends AbstractGridExecutor { static private Pattern QUOTED_STRING_REGEX = ~/"((?:[^"\\]|\\.)*)"(\s*#.*)?/ + static private AtomicInteger taskArrayCount = new AtomicInteger() + private boolean perJobMemLimit private boolean perTaskReserve @@ -296,12 +298,10 @@ class LsfExecutor extends AbstractGridExecutor { return FusionHelper.isFusionEnabled(session) } - private AtomicInteger arrayTaskCount = new AtomicInteger() - @Override String getArrayHeaders(int arraySize, TaskRun task) { final directives = getDirectives(task) - << '-J' << "nf-array-${arrayTaskCount.getAndIncrement()}[0-${arraySize - 1}]" + << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]" getHeaders(directives) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index aa371df384..d69d30a2d8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -37,7 +37,7 @@ interface TaskArrayAware { * * @param array */ - default TaskArraySubmitter createArrayTaskSubmitter(List array) { + default TaskArraySubmitter createTaskArraySubmitter(List array) { new TaskArraySubmitter(array) } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 19712a9713..d53691b0fd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -111,7 +111,7 @@ class TaskArrayCollector { protected void submit0(List array) { // create submitter for array job - executor.createArrayTaskSubmitter(array) + executor.createTaskArraySubmitter(array) // submit each task to the underlying monitor for( TaskHandler handler : array ) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 616460965e..405dd00640 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -309,7 +309,7 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar } @Override - TaskArraySubmitter createArrayTaskSubmitter(List array) { + TaskArraySubmitter createTaskArraySubmitter(List array) { new AwsBatchTaskArraySubmitter(array, this) } From 16af92f68ff7347f19993f6e6ac53a6cc51a95b8 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 1 May 2023 07:57:50 -0500 Subject: [PATCH 19/97] Fix failing tests Signed-off-by: Ben Sherman --- .../groovy/nextflow/executor/GridTaskArraySubmitter.groovy | 2 +- .../main/groovy/nextflow/executor/GridTaskHandler.groovy | 2 +- .../groovy/nextflow/executor/HyperQueueExecutor.groovy | 7 ++++--- .../main/groovy/nextflow/executor/SubmitJobAware.groovy | 6 +++--- .../groovy/nextflow/executor/GridTaskHandlerTest.groovy | 6 +++--- 5 files changed, 12 insertions(+), 11 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index 59bcb52a40..7aa62baebd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -58,7 +58,7 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAwar builder = createProcessBuilder(true) // -- submit the array job with a retryable strategy - final result = safeExecute( () -> processStart(builder, launcherScript) ) + final result = safeExecute( () -> launchProcess(builder, launcherScript) ) final jobId = (String)executor.parseJobId(result) // -- set the job id and status of each task diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 62ceecad2d..d4070d79a5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -143,7 +143,7 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { // -- forward the job launcher script to the command stdin if required final stdinScript = executor.pipeLauncherScript() ? stdinLauncherScript() : null // -- execute with a re-triable strategy - final result = safeExecute( () -> processStart(builder, stdinScript) ) + final result = safeExecute( () -> launchProcess(builder, stdinScript) ) // -- save the JobId in the this.jobId = executor.parseJobId(result) this.status = SUBMITTED diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index 4ffb0fd204..cb81a51c93 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -23,6 +23,7 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.processor.TaskRun import nextflow.util.ServiceName +import nextflow.util.TupleHelper /** * Implements executor for HyperQueue * https://github.com/It4innovations/hyperqueue/ @@ -81,7 +82,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { @Override List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - return List.of('hq', '--output-mode=quiet', 'submit', '--directives=file', scriptFile.getName()) + return TupleHelper.listOf('hq', '--output-mode=quiet', 'submit', '--directives=file', scriptFile.getName()) } @Override @@ -96,7 +97,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { @Override protected List getKillCommand() { - return List.of('hq', 'job', 'cancel') + return TupleHelper.listOf('hq', 'job', 'cancel') } @Override @@ -113,7 +114,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { @Override protected List queueStatusCommand(Object queue) { - return List.of('hq', '--output-mode=quiet', 'job', 'list', '--all') + return TupleHelper.listOf('hq', '--output-mode=quiet', 'job', 'list', '--all') } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy index 968c541383..782173de92 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy @@ -66,7 +66,7 @@ trait SubmitJobAware extends FusionAwareTask { return builder } - String processStart(ProcessBuilder builder, String pipeScript) { + String launchProcess(ProcessBuilder builder, String pipeScript) { final process = builder.start() try { @@ -80,7 +80,7 @@ trait SubmitJobAware extends FusionAwareTask { // -- wait the the process completes final result = process.text final exitStatus = process.waitFor() - final cmd = launchCmd0(builder,pipeScript) + final cmd = getLaunchCommand(builder,pipeScript) if( exitStatus ) { throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) @@ -98,7 +98,7 @@ trait SubmitJobAware extends FusionAwareTask { } } - private String launchCmd0(ProcessBuilder builder, String pipeScript) { + String getLaunchCommand(ProcessBuilder builder, String pipeScript) { def result = CmdLineHelper.toLine(builder.command()) if( pipeScript ) { result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 01a028e05a..47af310d7b 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -66,7 +66,7 @@ class GridTaskHandlerTest extends Specification { exec.pipeLauncherScript() >> false and: handler.fusionEnabled() >> false - handler.createProcessBuilder() >> GroovyMock(ProcessBuilder) + handler.createProcessBuilder(false) >> GroovyMock(ProcessBuilder) and: thrown(ProcessFailedException) and: @@ -132,9 +132,9 @@ class GridTaskHandlerTest extends Specification { def exec = Spy(GridTaskHandler) expect: - exec.launchCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' + exec.getLaunchCommand(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' and: - exec.launchCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == + exec.getLaunchCommand(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == '''\ cat << 'LAUNCH_COMMAND_EOF' | qsub docker run /some/file From 603913f3c476139872ff27ba7d84da122ec0909b Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 2 May 2023 01:43:51 -0500 Subject: [PATCH 20/97] minor edits Signed-off-by: Ben Sherman --- docs/process.md | 2 +- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 2 +- .../src/main/groovy/nextflow/processor/TaskProcessor.groovy | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/process.md b/docs/process.md index 74add51fdb..dd4d52b2a7 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1291,7 +1291,7 @@ Once an array job is submitted, the "child" tasks are executed in the same way: - Each task is executed in its own work directory with its own script. - Any tasks that fail (and can be retried) will be retried in another array job without interfering with the tasks that succeeded. -The following directives msut be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: +The following directives must be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: - {ref}`process-accelerator` - {ref}`process-clusterOptions` diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index d53691b0fd..60991f402c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -63,7 +63,7 @@ class TaskArrayCollector { * * @param handler */ - void submit(TaskRun task) { + void collect(TaskRun task) { sync.lock() try { diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index e5bf9f2a24..535470e514 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -2210,7 +2210,7 @@ class TaskProcessor { // add the task to the collection of running tasks if( arrayCollector ) - arrayCollector.submit(task) + arrayCollector.collect(task) else executor.submit(task) From 98cfb0edab6bc041b21ce1f8126a06a8e98b6cfc Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 2 May 2023 02:16:53 -0500 Subject: [PATCH 21/97] Reduce code duplication Signed-off-by: Ben Sherman --- .../executor/AbstractGridExecutor.groovy | 4 +- .../nextflow/executor/BridgeExecutor.groovy | 2 +- .../nextflow/executor/CondorExecutor.groovy | 2 +- .../nextflow/executor/FluxExecutor.groovy | 2 +- .../executor/GridTaskArraySubmitter.groovy | 122 +++++++----------- .../nextflow/executor/GridTaskHandler.groovy | 44 +++---- .../executor/HyperQueueExecutor.groovy | 2 +- .../nextflow/executor/LsfExecutor.groovy | 9 +- .../nextflow/executor/MoabExecutor.groovy | 2 +- .../nextflow/executor/NqsiiExecutor.groovy | 2 +- .../nextflow/executor/OarExecutor.groovy | 2 +- .../nextflow/executor/PbsExecutor.groovy | 9 +- .../nextflow/executor/PbsProExecutor.groovy | 2 +- .../nextflow/executor/SgeExecutor.groovy | 9 +- .../nextflow/executor/SlurmExecutor.groovy | 9 +- .../nextflow/executor/SubmitJobAware.groovy | 24 ++++ .../executor/GridTaskHandlerTest.groovy | 2 +- .../batch/AwsBatchTaskArraySubmitter.groovy | 73 +++-------- .../aws/batch/AwsBatchTaskHandler.groovy | 22 +--- .../cloud/aws/batch/SubmitJobAware.groovy | 15 +++ 20 files changed, 147 insertions(+), 211 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index d58520754f..f91344ae65 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -136,7 +136,7 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { * @param initial An initial list of directives * @return A list of directives for this task used for the job submission */ - abstract protected List getDirectives(TaskRun task, List initial) + abstract List getDirectives(TaskRun task, List initial) /** * Given a task returns a *clean* name used to submit the job to the grid engine. @@ -420,7 +420,7 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { new GridTaskArraySubmitter(array, this) } - String getArrayHeaders(int arraySize, TaskRun task) { + List getArrayDirective(int arraySize, TaskRun task) { throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy index aa48fb0721..fb5a0e97b2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy @@ -44,7 +44,7 @@ class BridgeExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { String job_name = "" // parenthesis are not compatible with bridge submission commands diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy index a52cab4af0..ccb3ebcdff 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy @@ -49,7 +49,7 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << "universe = vanilla" result << "executable = ${TaskRun.CMD_RUN}" diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy index dc5cb9f526..218d1b4ffa 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy @@ -41,7 +41,7 @@ class FluxExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { return result } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index 7aa62baebd..7927bcb423 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -48,98 +48,66 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAwar TaskRun getTask() { array.first().getTask() } @Override - protected void submit() { - ProcessBuilder builder = null - try { - // -- create the array job script - final launcherScript = getLauncherScript() + void submit() { + final jobId = submitJob(true) - // -- create the submit command - builder = createProcessBuilder(true) - - // -- submit the array job with a retryable strategy - final result = safeExecute( () -> launchProcess(builder, launcherScript) ) - final jobId = (String)executor.parseJobId(result) + array.eachWithIndex { handler, i -> + ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) + handler.setStatus(TaskStatus.SUBMITTED) + } - // -- set the job id and status of each task - this.setJobId(jobId) - this.setStatus(TaskStatus.SUBMITTED) + log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" + } - log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" - } - catch( Exception e ) { - // update task exit status and message + @Override + Exception submitError(Exception e, String submitCommand) { + // update task exit status and message + for( TaskHandler handler : array ) { if( e instanceof ProcessNonZeroExitStatusException ) { - for( TaskHandler handler : array ) { - handler.task.exitStatus = e.getExitStatus() - handler.task.stdout = e.getReason() - handler.task.script = e.getCommand() - } + handler.task.exitStatus = e.getExitStatus() + handler.task.stdout = e.getReason() + handler.task.script = e.getCommand() } else { - for( TaskHandler handler : array ) - handler.task.script = builder ? CmdLineHelper.toLine(builder.command()) : null + handler.task.script = submitCommand } - this.setStatus(TaskStatus.COMPLETED) - throw new ProcessFailedException("Error submitting array job for execution", e) + handler.setStatus(TaskStatus.COMPLETED) } + throw new ProcessFailedException("Error submitting array job for execution", e) } - protected String getLauncherScript() { - return fusionEnabled() - ? fusionLauncherScript() - : classicLauncherScript() - } - - protected String fusionLauncherScript() { - final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() - final fusionWorkDir = FusionHelper.toContainerMount(task.workDir).toString() - final arrayHeaders = executor.getArrayHeaders(array.size(), getTask()) - final arrayIndexName = executor.getArrayIndexName() - final workDirs = array - .collect { handler -> FusionHelper.toContainerMount(handler.task.workDir) } - .join(' ') - - final cmd = FusionHelper.runWithContainer( - fusionLauncher(), - task.getContainerConfig(), - task.getContainer(), - fusionSubmitCli() ) - - final builder = new StringBuilder() - << '#!/bin/bash\n' - << arrayHeaders.replace(remoteLog, '/dev/null') - << "declare -a array=( ${workDirs} )\n" - << cmd.replace(fusionWorkDir, "\${array[\$${arrayIndexName}]}") - - return builder.toString() - } - - protected String classicLauncherScript() { - final arrayHeaders = executor.getArrayHeaders(array.size(), getTask()) - final arrayIndexName = executor.getArrayIndexName() - final workDirs = array - .collect { handler -> handler.task.workDir } - .join(' ') + @Override + String stdinLauncherScript() { + def arrayDirective = executor.getArrayDirective(array.size(), task) + def directives = executor.getDirectives(task, arrayDirective) + def headers = executor.getHeaders(directives) + def arrayIndexName = executor.getArrayIndexName() + + def workDirs + def cmd + + if( fusionEnabled() ) { + final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() + headers = headers.replaceAll(remoteLog, '/dev/null') + + workDirs = array.collect { h -> FusionHelper.toContainerMount(h.task.workDir).toString() } + cmd = FusionHelper + .runWithContainer(fusionLauncher(), task.getContainerConfig(), task.getContainer(), fusionSubmitCli()) + .replaceAll(workDirs.first(), '\\$task_dir') + } + else { + workDirs = array.collect { h -> h.task.workDir.toString() } + cmd = "bash \$task_dir/${TaskRun.CMD_RUN}" + } final builder = new StringBuilder() << '#!/bin/bash\n' - << arrayHeaders - << "declare -a array=( ${workDirs} )\n" - << "bash \${array[\$${arrayIndexName}]}/${TaskRun.CMD_RUN}\n" + << headers + << "declare -a array=( ${workDirs.join(' ')} )\n" + << "task_dir=\${array[\$${arrayIndexName}]}\n" + << "${cmd}\n" return builder.toString() } - - protected void setJobId(String jobId) { - array.eachWithIndex { handler, i -> - ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) - } - } - - protected void setStatus(TaskStatus status) { - for( TaskHandler handler : array ) - handler.setStatus(status) - } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index d4070d79a5..d9e28e90d7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -103,7 +103,8 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { : executor.createBashWrapperBuilder(task) } - protected String stdinLauncherScript() { + @Override + String stdinLauncherScript() { return fusionEnabled() ? fusionStdinWrapper() : wrapperFile.text } @@ -136,34 +137,25 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { return } - ProcessBuilder builder = null - try { - // -- start the execution and notify the event to the monitor - builder = createProcessBuilder(executor.pipeLauncherScript()) - // -- forward the job launcher script to the command stdin if required - final stdinScript = executor.pipeLauncherScript() ? stdinLauncherScript() : null - // -- execute with a re-triable strategy - final result = safeExecute( () -> launchProcess(builder, stdinScript) ) - // -- save the JobId in the - this.jobId = executor.parseJobId(result) - this.status = SUBMITTED - log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" + this.jobId = submitJob(executor.pipeLauncherScript()) + this.status = SUBMITTED + + log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" + } + @Override + Exception submitError(Exception e, String submitCommand) { + // update task exit status and message + if( e instanceof ProcessNonZeroExitStatusException ) { + task.exitStatus = e.getExitStatus() + task.stdout = e.getReason() + task.script = e.getCommand() } - catch( Exception e ) { - // update task exit status and message - if( e instanceof ProcessNonZeroExitStatusException ) { - task.exitStatus = e.getExitStatus() - task.stdout = e.getReason() - task.script = e.getCommand() - } - else { - task.script = builder ? CmdLineHelper.toLine(builder.command()) : null - } - status = COMPLETED - throw new ProcessFailedException("Error submitting process '${task.name}' for execution", e ) + else { + task.script = submitCommand } - + this.status = COMPLETED + return new ProcessFailedException("Error submitting process '${task.name}' for execution", e ) } void setJobId(String jobId) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index cb81a51c93..b1b6be15df 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -56,7 +56,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '--name' << getJobNameFor(task) result << '--log' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index ba24ef6823..bd992f06ef 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -67,7 +67,7 @@ class LsfExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() @@ -299,11 +299,8 @@ class LsfExecutor extends AbstractGridExecutor { } @Override - String getArrayHeaders(int arraySize, TaskRun task) { - final directives = getDirectives(task) - << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]" - - getHeaders(directives) + List getArrayDirective(int arraySize, TaskRun task) { + ['-J', "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]"] } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy index d37904c6d7..4a65e690e6 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy @@ -39,7 +39,7 @@ class MoabExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives( TaskRun task, List result ) { + List getDirectives( TaskRun task, List result ) { assert result !=null result << '-N' << getJobNameFor(task) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy index 19e13165e6..c441e6c28e 100755 --- a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy @@ -38,7 +38,7 @@ class NqsiiExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy index d6933fb1be..acc381c35a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy @@ -38,7 +38,7 @@ class OarExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '-d' << quote(task.workDir) result << '-n' << getJobNameFor(task) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 724a241c01..26f4d21171 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -38,7 +38,7 @@ class PbsExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives( TaskRun task, List result ) { + List getDirectives( TaskRun task, List result ) { assert result !=null result << '-N' << getJobNameFor(task) @@ -170,11 +170,8 @@ class PbsExecutor extends AbstractGridExecutor { } @Override - String getArrayHeaders(int arraySize, TaskRun task) { - final directives = getDirectives(task) - << '-J' << "0-${arraySize - 1}" - - getHeaders(directives) + List getArrayDirective(int arraySize, TaskRun task) { + ['-J', "0-${arraySize - 1}"] } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 7f9c5ad38e..d6e944b9d0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -41,7 +41,7 @@ class PbsProExecutor extends PbsExecutor { * @return A {@link List} containing all directive tokens and values. */ @Override - protected List getDirectives(TaskRun task, List result ) { + List getDirectives(TaskRun task, List result ) { assert result !=null // when multiple competing directives are provided, only the first one will take effect diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index ee62bed2e5..c7a5a2f4ba 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -33,7 +33,7 @@ class SgeExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) @@ -183,11 +183,8 @@ class SgeExecutor extends AbstractGridExecutor { } @Override - String getArrayHeaders(int arraySize, TaskRun task) { - final directives = getDirectives(task) - << '-t' << "0-${arraySize - 1}" - - getHeaders(directives) + List getArrayDirective(int arraySize, TaskRun task) { + ['-t', "0-${arraySize - 1}"] } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 6d911ddce7..d99aa504f0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -49,7 +49,7 @@ class SlurmExecutor extends AbstractGridExecutor { * @return A {@link List} containing all directive tokens and values. */ @Override - protected List getDirectives(TaskRun task, List result) { + List getDirectives(TaskRun task, List result) { result << '-J' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) // -o OUTFILE and no -e option => stdout and stderr merged to stdout/OUTFILE @@ -194,11 +194,8 @@ class SlurmExecutor extends AbstractGridExecutor { } @Override - String getArrayHeaders(int arraySize, TaskRun task) { - final directives = getDirectives(task) - << '--array' << "0-${arraySize - 1}" - - getHeaders(directives) + List getArrayDirective(int arraySize, TaskRun task) { + ['--array', "0-${arraySize - 1}"] } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy index 782173de92..e044c997ee 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy @@ -49,6 +49,30 @@ trait SubmitJobAware extends FusionAwareTask { abstract TaskRun getTask() + String submitJob(boolean pipeLauncherScript) { + ProcessBuilder builder = null + try { + // -- create the stdin launcher script + final stdinScript = pipeLauncherScript ? stdinLauncherScript() : null + + // -- create the submit command + builder = createProcessBuilder(pipeLauncherScript) + + // -- submit the job with a retryable strategy + final result = safeExecute( () -> launchProcess(builder, stdinScript) ) + + return (String)executor.parseJobId(result) + } + catch( Exception e ) { + final submitCommand = builder ? CmdLineHelper.toLine(builder.command()) : null + throw submitError(e, submitCommand) + } + } + + abstract String stdinLauncherScript() + + abstract Exception submitError(Exception e, String submitCommand) + ProcessBuilder createProcessBuilder(boolean pipeLauncherScript) { // -- log the submit command line diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 47af310d7b..df15954065 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -66,7 +66,7 @@ class GridTaskHandlerTest extends Specification { exec.pipeLauncherScript() >> false and: handler.fusionEnabled() >> false - handler.createProcessBuilder(false) >> GroovyMock(ProcessBuilder) + handler.createProcessBuilder(false) >> new ProcessBuilder().command([]) and: thrown(ProcessFailedException) and: diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy index 89927ac2ff..25c0c55b28 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy @@ -20,13 +20,9 @@ import static nextflow.fusion.FusionConfig.FUSION_PATH import static nextflow.fusion.FusionHelper.* import com.amazonaws.services.batch.AWSBatch -import com.amazonaws.services.batch.model.AWSBatchException import com.amazonaws.services.batch.model.ArrayProperties -import com.amazonaws.services.batch.model.SubmitJobRequest -import com.amazonaws.services.batch.model.SubmitJobResult import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.exception.ProcessSubmitException import nextflow.executor.TaskArraySubmitter import nextflow.processor.TaskHandler import nextflow.processor.TaskRun @@ -64,47 +60,32 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob @Override List getSubmitCommand() { - return fusionEnabled() - ? fusionSubmitCli() - : classicSubmitCli() - } - - @Override - List fusionSubmitCli() { - final workDirs = array - .collect { handler -> toContainerMount(handler.task.workDir) } - .join(' ') + // create wrapper script final arrayIndexName = executor.getArrayIndexName() + final workDirs = fusionEnabled() + ? array.collect { h -> toContainerMount(h.task.workDir).toString() } + : array.collect { h -> h.task.workDir.toUriString() } - final cmd = """ - declare -a array=( ${workDirs} ) - bash \${array[\$${arrayIndexName}]}/${TaskRun.CMD_RUN} - """.stripIndent().trim() + def cmd = ((AwsBatchTaskHandler)array.first()) + .getSubmitCommand() + .last() + .replaceAll(workDirs.first(), '\\$task_dir') - return List.of(FUSION_PATH, 'bash', '-c', cmd.toString()) - } - - protected List classicSubmitCli() { - final workDirs = array - .collect { handler -> handler.task.workDir.toUriString() } - .join(' ') - final arrayIndexName = executor.getArrayIndexName() + if( fusionEnabled() ) + cmd = "bash ${cmd}" - final opts = getAwsOptions() - final cli = opts.getAwsCli() - final debug = opts.debug ? ' --debug' : '' - final sse = opts.storageEncryption ? " --sse ${opts.storageEncryption}" : '' - final kms = opts.storageKmsKeyId ? " --sse-kms-key-id ${opts.storageKmsKeyId}" : '' - final aws = "${cli} s3 cp --only-show-errors${debug}${sse}${kms}" - - final cmd = """ - declare -a array=( ${workDirs} ) + cmd = """ + declare -a array=( ${workDirs.join(' ')} ) task_dir=\${array[\$${arrayIndexName}]} - trap "{ ret=\$?; ${aws} ${TaskRun.CMD_LOG} \$task_dir/${TaskRun.CMD_LOG}||true; exit \$ret; }" EXIT - ${aws} \$task_dir/${TaskRun.CMD_RUN} - | bash 2>&1 | tee ${TaskRun.CMD_LOG} + ${cmd} """.stripIndent().trim() - return List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) + // create command line + final cli = ['bash', '-o', 'pipefail', '-c', cmd.toString()] + if( fusionEnabled() ) + cli.add(0, FUSION_PATH) + + return cli } @Override @@ -114,7 +95,7 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob .withArrayProperties(new ArrayProperties().withSize(array.size())) // -- submit the array job - final response = submitJobRequest0(bypassProxy(client), request) + final response = submitJobRequest(bypassProxy(client), request) final jobId = response.jobId // -- set the job id, queue, and status of each task @@ -127,18 +108,4 @@ class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJob log.debug "[AWS BATCH] submitted array job > jobId: ${jobId}" } - static private SubmitJobResult submitJobRequest0(AWSBatch client, SubmitJobRequest request) { - try { - return client.submitJob(request) - } - catch( AWSBatchException e ) { - if( e.statusCode >= 500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index cfc4b91c53..541a07c10b 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -19,19 +19,15 @@ package nextflow.cloud.aws.batch import java.nio.file.Path import com.amazonaws.services.batch.AWSBatch -import com.amazonaws.services.batch.model.AWSBatchException import com.amazonaws.services.batch.model.AttemptContainerDetail import com.amazonaws.services.batch.model.DescribeJobsRequest import com.amazonaws.services.batch.model.DescribeJobsResult import com.amazonaws.services.batch.model.JobDetail -import com.amazonaws.services.batch.model.SubmitJobRequest -import com.amazonaws.services.batch.model.SubmitJobResult import com.amazonaws.services.batch.model.TerminateJobRequest import groovy.transform.CompileDynamic import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.cloud.types.CloudMachineInfo -import nextflow.exception.ProcessSubmitException import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder import nextflow.processor.BatchContext @@ -154,7 +150,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler&1 | tee ${TaskRun.CMD_LOG}" - return ['bash','-o','pipefail','-c', cmd.toString()] + return List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) } @Override @@ -171,27 +167,13 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler job=$jobId; work-dir=${task.getWorkDirStr()}" } - static private SubmitJobResult submitJobRequest0(AWSBatch client, SubmitJobRequest request) { - try { - return client.submitJob(request) - } - catch( AWSBatchException e ) { - if( e.statusCode >= 500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - void setJobId(String jobId) { this.jobId = jobId } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy index 7eb8b50b3f..ac50bc50b3 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy @@ -42,6 +42,7 @@ import com.amazonaws.services.batch.model.ResourceRequirement import com.amazonaws.services.batch.model.ResourceType import com.amazonaws.services.batch.model.RetryStrategy import com.amazonaws.services.batch.model.SubmitJobRequest +import com.amazonaws.services.batch.model.SubmitJobResult import com.amazonaws.services.batch.model.Volume import groovy.transform.CompileStatic import groovy.transform.Memoized @@ -529,4 +530,18 @@ trait SubmitJobAware extends FusionAwareTask { } } + static SubmitJobResult submitJobRequest(AWSBatch client, SubmitJobRequest request) { + try { + return client.submitJob(request) + } + catch( AWSBatchException e ) { + if( e.statusCode >= 500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + } From f02de406a85e55cf0202f308ff90dd7d13addb49 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 2 May 2023 02:23:23 -0500 Subject: [PATCH 22/97] Reduce code duplication Signed-off-by: Ben Sherman --- .../executor/GridTaskArraySubmitter.groovy | 17 +++-------------- .../nextflow/executor/GridTaskHandler.groovy | 1 - 2 files changed, 3 insertions(+), 15 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index 7927bcb423..45c2f5fbea 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -19,12 +19,10 @@ package nextflow.executor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.exception.ProcessFailedException -import nextflow.exception.ProcessNonZeroExitStatusException import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus -import nextflow.util.CmdLineHelper /** * Submit tasks as an array job for a grid executor. * @@ -61,18 +59,9 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAwar @Override Exception submitError(Exception e, String submitCommand) { - // update task exit status and message - for( TaskHandler handler : array ) { - if( e instanceof ProcessNonZeroExitStatusException ) { - handler.task.exitStatus = e.getExitStatus() - handler.task.stdout = e.getReason() - handler.task.script = e.getCommand() - } - else { - handler.task.script = submitCommand - } - handler.setStatus(TaskStatus.COMPLETED) - } + for( TaskHandler handler : array ) + ((GridTaskHandler)handler).submitError(e, submitCommand) + throw new ProcessFailedException("Error submitting array job for execution", e) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index d9e28e90d7..2569c0513c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -30,7 +30,6 @@ import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.trace.TraceRecord -import nextflow.util.CmdLineHelper import nextflow.util.Duration import nextflow.util.Throttle /** From fe3a3d7bd7264291eb3183d20809c1527eeafe07 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 2 May 2023 10:53:16 -0500 Subject: [PATCH 23/97] minor edits Signed-off-by: Ben Sherman --- docs/process.md | 5 +- .../executor/GridTaskArraySubmitter.groovy | 40 ++-- .../nextflow/executor/SubmitJobAware.groovy | 1 + .../executor/TaskArraySubmitter.groovy | 2 - .../processor/TaskArrayCollector.groovy | 8 +- .../nextflow/processor/TaskHandler.groovy | 1 - .../aws/batch/AwsBatchTaskHandler.groovy | 223 ++++++++++-------- 7 files changed, 149 insertions(+), 131 deletions(-) diff --git a/docs/process.md b/docs/process.md index dd4d52b2a7..a150c52420 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1286,10 +1286,7 @@ Nextflow currently supports array jobs for the following executors: A process using array jobs will collect tasks and submit each batch as an array job when it is ready. Any "leftover" tasks will be submitted as a partial array job. -Once an array job is submitted, the "child" tasks are executed in the same way: - -- Each task is executed in its own work directory with its own script. -- Any tasks that fail (and can be retried) will be retried in another array job without interfering with the tasks that succeeded. +Once an array job is submitted, each "child" task is executed as an independent job. Any tasks that fail (and can be retried) will be retried in another array job without interfering with the tasks that succeeded. The following directives must be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy index 45c2f5fbea..17f639f55a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy @@ -45,26 +45,6 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAwar @Override TaskRun getTask() { array.first().getTask() } - @Override - void submit() { - final jobId = submitJob(true) - - array.eachWithIndex { handler, i -> - ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) - handler.setStatus(TaskStatus.SUBMITTED) - } - - log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" - } - - @Override - Exception submitError(Exception e, String submitCommand) { - for( TaskHandler handler : array ) - ((GridTaskHandler)handler).submitError(e, submitCommand) - - throw new ProcessFailedException("Error submitting array job for execution", e) - } - @Override String stdinLauncherScript() { def arrayDirective = executor.getArrayDirective(array.size(), task) @@ -99,4 +79,24 @@ class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAwar return builder.toString() } + @Override + void submit() { + final jobId = submitJob(true) + + array.eachWithIndex { handler, i -> + ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) + handler.setStatus(TaskStatus.SUBMITTED) + } + + log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" + } + + @Override + Exception submitError(Exception e, String submitCommand) { + for( TaskHandler handler : array ) + ((GridTaskHandler)handler).submitError(e, submitCommand) + + throw new ProcessFailedException("Error submitting array job for execution", e) + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy index e044c997ee..00eaa8a7ce 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory * Implementation of job submission for grid executors. * * @author Paolo Di Tommaso + * @author Ben Sherman */ @CompileStatic trait SubmitJobAware extends FusionAwareTask { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index a808a6336c..32595cbd3e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -42,8 +42,6 @@ class TaskArraySubmitter { handler.arraySubmitter = this } - List getArray() { array } - /** * Mark a task as ready to be submitted. * diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 60991f402c..8b72eff334 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -61,7 +61,7 @@ class TaskArrayCollector { * Add a task to the current array, and submit the array when it * reaches the desired size. * - * @param handler + * @param task */ void collect(TaskRun task) { sync.lock() @@ -76,10 +76,10 @@ class TaskArrayCollector { // create task handler final handler = executor.createTaskHandler(task) - // add task to the array array + // add task to the array array << handler - // submit array job when the array is ready + // submit array job when it is ready if( array.size() == arraySize ) { submit0(array) array = new ArrayList<>(arraySize) @@ -92,8 +92,6 @@ class TaskArrayCollector { /** * Close the collector, submitting any remaining tasks as a partial array job. - * - * @param process */ void close() { sync.lock() diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 3e09703438..341b274aef 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -19,7 +19,6 @@ package nextflow.processor import static nextflow.processor.TaskStatus.* import java.nio.file.NoSuchFileException -import java.nio.file.Path import groovy.util.logging.Slf4j import nextflow.executor.TaskArraySubmitter diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 541a07c10b..866fd53a85 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -98,18 +98,17 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler getSubmitCommand() { - return fusionEnabled() - ? fusionSubmitCli() - : classicSubmitCli() - } - - protected List classicSubmitCli() { - final opts = getAwsOptions() - final cli = opts.getAwsCli() - final debug = opts.debug ? ' --debug' : '' - final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' - final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' - final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" - final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $aws s3:/${getWrapperFile()} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" - return List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) - } - - @Override - void submit() { - if( arraySubmitter ) { - arraySubmitter.collect(this) - return - } - - // -- create the submit request - final request = newSubmitRequest(task) - log.trace "[AWS BATCH] new job request > $request" - - // -- submit the job - // use the real client object because this method - // should be invoked by the thread pool - final response = submitJobRequest(bypassProxy(client), request) - this.jobId = response.jobId - this.queueName = request.getJobQueue() - this.status = TaskStatus.SUBMITTED - log.debug "[AWS BATCH] submitted > job=$jobId; work-dir=${task.getWorkDirStr()}" - } - - void setJobId(String jobId) { - this.jobId = jobId - } - - void setQueueName(String queueName) { - this.queueName = queueName - } - - @Override - boolean checkIfRunning() { - if( !jobId || !isSubmitted() ) - return false - final job = describeJob(jobId) - final result = job?.status in ['RUNNING', 'SUCCEEDED', 'FAILED'] - if( result ) - this.status = TaskStatus.RUNNING - // fetch the task arn - if( !taskArn ) - taskArn = job?.getContainer()?.getTaskArn() - return result + private String jobIdsToString(Collection items) { + final MAX=10 + final sz = items.size() + items.size()<=MAX ? items.join(', ').toString() : items.take(MAX).join(', ').toString() + ", ... other ${sz-MAX} omitted" } /** @@ -243,14 +176,38 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler $req" + + /* + * submit the task execution + */ + // note use the real client object because this method + // is supposed to be invoked by the thread pool + final resp = submitJobRequest(bypassProxy(client), req) + this.jobId = resp.jobId + this.status = TaskStatus.SUBMITTED + this.queueName = req.getJobQueue() + log.debug "[AWS BATCH] submitted > job=$jobId; work-dir=${task.getWorkDirStr()}" } + void setJobId(String jobId) { + this.jobId = jobId + } + + void setQueueName(String queueName) { + this.queueName = queueName + } + + protected BashWrapperBuilder createTaskWrapper() { + return fusionEnabled() + ? fusionLauncher() + : new AwsBatchScriptLauncher(task.toTaskBean(), getAwsOptions()) + } + + protected List classicSubmitCli() { + // the cmd list to launch it + final opts = getAwsOptions() + final cli = opts.getAwsCli() + final debug = opts.debug ? ' --debug' : '' + final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' + final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' + final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" + final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $aws s3:/${getWrapperFile()} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" + return ['bash','-o','pipefail','-c', cmd.toString()] + } + + @Override + List getSubmitCommand() { + // final launcher command + return fusionEnabled() + ? fusionSubmitCli() + : classicSubmitCli() + } + + /** + * @return The launcher script file {@link Path} + */ + protected Path getWrapperFile() { wrapperFile } + + /** + * @return The launcher log file {@link Path} + */ + protected Path getLogFile() { logFile } + protected CloudMachineInfo getMachineInfo() { if( machineInfo ) return machineInfo @@ -336,5 +353,13 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler Date: Tue, 2 May 2023 18:46:42 -0500 Subject: [PATCH 24/97] Use TaskRun subclass to submit array job in a generic manner Signed-off-by: Ben Sherman --- docs/process.md | 1 - .../executor/AbstractGridExecutor.groovy | 29 +- .../nextflow/executor/BridgeExecutor.groovy | 15 +- .../nextflow/executor/CondorExecutor.groovy | 6 +- .../nextflow/executor/FluxExecutor.groovy | 11 +- .../executor/GridTaskArraySubmitter.groovy | 102 ---- .../nextflow/executor/GridTaskHandler.groovy | 196 +++++- .../executor/HyperQueueExecutor.groovy | 4 +- .../nextflow/executor/LsfExecutor.groovy | 26 +- .../nextflow/executor/MoabExecutor.groovy | 18 +- .../nextflow/executor/NqsiiExecutor.groovy | 11 +- .../nextflow/executor/OarExecutor.groovy | 13 +- .../nextflow/executor/PbsExecutor.groovy | 28 +- .../nextflow/executor/PbsProExecutor.groovy | 10 +- .../nextflow/executor/SgeExecutor.groovy | 22 +- .../nextflow/executor/SlurmExecutor.groovy | 27 +- .../nextflow/executor/SubmitJobAware.groovy | 195 ------ .../nextflow/executor/TaskArrayAware.groovy | 18 +- .../executor/TaskArraySubmitter.groovy | 85 ++- .../executor/local/LocalExecutor.groovy | 3 +- .../nextflow/processor/TaskArray.groovy | 34 ++ .../processor/TaskArrayCollector.groovy | 11 +- .../nextflow/processor/TaskHandler.groovy | 5 + .../executor/GridTaskHandlerTest.groovy | 6 +- .../cloud/aws/batch/AwsBatchExecutor.groovy | 5 - .../batch/AwsBatchTaskArraySubmitter.groovy | 111 ---- .../aws/batch/AwsBatchTaskHandler.groovy | 563 +++++++++++++++++- .../cloud/aws/batch/SubmitJobAware.groovy | 547 ----------------- .../aws/batch/AwsBatchTaskHandlerTest.groovy | 180 +++--- 29 files changed, 1065 insertions(+), 1217 deletions(-) delete mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy delete mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy delete mode 100644 plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy delete mode 100644 plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy diff --git a/docs/process.md b/docs/process.md index a150c52420..7a2bf68c2b 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1277,7 +1277,6 @@ process cpu_task { Nextflow currently supports array jobs for the following executors: - {ref}`awsbatch-executor` -- {ref}`local-executor` - {ref}`lsf-executor` - {ref}`pbs-executor` - {ref}`pbspro-executor` diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index f91344ae65..0365f69f6e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -94,17 +94,15 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { * @return A multi-line string containing the job directives */ String getHeaders( TaskRun task ) { - getHeaders(getDirectives(task)) - } - String getHeaders( List directives ) { final token = getHeaderToken() def result = new StringBuilder() def header = new ArrayList(2) - def len = directives.size()-1 - for( int i = 0; i < len; i += 2 ) { - def opt = directives[i] - def val = directives[i+1] + def dir = getDirectives(task) + def len = dir.size()-1 + for( int i=0; i getDirectives(TaskRun task, List initial) + abstract protected List getDirectives(TaskRun task, List initial) /** * Given a task returns a *clean* name used to submit the job to the grid engine. @@ -196,11 +194,7 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { * @param task The task instance descriptor * @return A list holding the command line */ - List getSubmitCommandLine(TaskRun task, Path scriptFile) { - getSubmitCommandLine(task, scriptFile, pipeLauncherScript()) - } - - abstract List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) + abstract List getSubmitCommandLine(TaskRun task, Path scriptFile) /** * Defines how script is run the by the grid-engine. @@ -414,14 +408,5 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { // Instead, it is the command wrapper script that is launched run within a container process. return isFusionEnabled() } - - @Override - TaskArraySubmitter createTaskArraySubmitter(List array) { - new GridTaskArraySubmitter(array, this) - } - - List getArrayDirective(int arraySize, TaskRun task) { - throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") - } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy index fb5a0e97b2..920f20afce 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BridgeExecutor.groovy @@ -44,7 +44,7 @@ class BridgeExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { String job_name = "" // parenthesis are not compatible with bridge submission commands @@ -85,9 +85,18 @@ class BridgeExecutor extends AbstractGridExecutor { String getHeaderToken() { '#MSUB' } + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - List.of('ccc_msub', scriptFile.getName()) + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + + ['ccc_msub', scriptFile.getName()] + } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy index ccb3ebcdff..03a161bb3f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CondorExecutor.groovy @@ -49,7 +49,7 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { result << "universe = vanilla" result << "executable = ${TaskRun.CMD_RUN}" @@ -88,8 +88,8 @@ class CondorExecutor extends AbstractGridExecutor { } @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - List.of('condor_submit', '--terse', CMD_CONDOR) + List getSubmitCommandLine(TaskRun task, Path scriptFile) { + return ['condor_submit', '--terse', CMD_CONDOR] } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy index 218d1b4ffa..37d7a4e1bd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/FluxExecutor.groovy @@ -41,15 +41,22 @@ class FluxExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { return result } // Flux does not require a special token or header String getHeaderToken() { null } + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { List result = ['flux', 'mini', 'submit'] result << '--setattr=cwd=' + quote(task.workDir) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy deleted file mode 100644 index 17f639f55a..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskArraySubmitter.groovy +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.executor - -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.exception.ProcessFailedException -import nextflow.fusion.FusionHelper -import nextflow.processor.TaskHandler -import nextflow.processor.TaskRun -import nextflow.processor.TaskStatus -/** - * Submit tasks as an array job for a grid executor. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class GridTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAware { - - private AbstractGridExecutor executor - - GridTaskArraySubmitter(List array, AbstractGridExecutor executor) { - super(array) - this.executor = executor - } - - @Override - AbstractGridExecutor getExecutor() { executor } - - @Override - TaskRun getTask() { array.first().getTask() } - - @Override - String stdinLauncherScript() { - def arrayDirective = executor.getArrayDirective(array.size(), task) - def directives = executor.getDirectives(task, arrayDirective) - def headers = executor.getHeaders(directives) - def arrayIndexName = executor.getArrayIndexName() - - def workDirs - def cmd - - if( fusionEnabled() ) { - final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() - headers = headers.replaceAll(remoteLog, '/dev/null') - - workDirs = array.collect { h -> FusionHelper.toContainerMount(h.task.workDir).toString() } - cmd = FusionHelper - .runWithContainer(fusionLauncher(), task.getContainerConfig(), task.getContainer(), fusionSubmitCli()) - .replaceAll(workDirs.first(), '\\$task_dir') - } - else { - workDirs = array.collect { h -> h.task.workDir.toString() } - cmd = "bash \$task_dir/${TaskRun.CMD_RUN}" - } - - final builder = new StringBuilder() - << '#!/bin/bash\n' - << headers - << "declare -a array=( ${workDirs.join(' ')} )\n" - << "task_dir=\${array[\$${arrayIndexName}]}\n" - << "${cmd}\n" - - return builder.toString() - } - - @Override - void submit() { - final jobId = submitJob(true) - - array.eachWithIndex { handler, i -> - ((GridTaskHandler)handler).setJobId(executor.getArrayTaskId(jobId, i)) - handler.setStatus(TaskStatus.SUBMITTED) - } - - log.debug "[${executor.name.toUpperCase()}] submitted array job > jobId: ${jobId}" - } - - @Override - Exception submitError(Exception e, String submitCommand) { - for( TaskHandler handler : array ) - ((GridTaskHandler)handler).submitError(e, submitCommand) - - throw new ProcessFailedException("Error submitting array job for execution", e) - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 2569c0513c..9bffa079c0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -20,23 +20,35 @@ import static nextflow.processor.TaskStatus.* import java.nio.file.Path import java.nio.file.attribute.BasicFileAttributes - +import java.time.temporal.ChronoUnit +import java.util.function.Predicate +import java.util.regex.Pattern + +import dev.failsafe.Failsafe +import dev.failsafe.RetryPolicy +import dev.failsafe.event.EventListener +import dev.failsafe.event.ExecutionAttemptedEvent +import dev.failsafe.function.CheckedSupplier +import groovy.transform.Memoized import groovy.util.logging.Slf4j import nextflow.exception.ProcessException import nextflow.exception.ProcessFailedException import nextflow.exception.ProcessNonZeroExitStatusException import nextflow.file.FileHelper +import nextflow.fusion.FusionAwareTask import nextflow.fusion.FusionHelper +import nextflow.processor.TaskArray import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.trace.TraceRecord +import nextflow.util.CmdLineHelper import nextflow.util.Duration import nextflow.util.Throttle /** * Handles a job execution in the underlying grid platform */ @Slf4j -class GridTaskHandler extends TaskHandler implements SubmitJobAware { +class GridTaskHandler extends TaskHandler implements FusionAwareTask { /** The target executor platform */ final AbstractGridExecutor executor @@ -87,13 +99,110 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { this.sanityCheckInterval = duration } - @Override - AbstractGridExecutor getExecutor() { executor } + protected ProcessBuilder createProcessBuilder() { - @Override - void prepareLauncher() { - // -- create the wrapper script - createTaskWrapper(task).build() + // -- log the qsub command + final cli = executor.getSubmitCommandLine(task, wrapperFile) + log.trace "start process ${task.name} > cli: ${cli}" + + /* + * launch 'sub' script wrapper + */ + ProcessBuilder builder = new ProcessBuilder() + .command( cli as String[] ) + .redirectErrorStream(true) + if( !fusionEnabled() ) + builder .directory(task.workDir.toFile()) + + return builder + } + + @Memoized + protected Predicate retryCondition(String reasonPattern) { + final pattern = Pattern.compile(reasonPattern) + return new Predicate() { + @Override + boolean test(Throwable failure) { + if( failure instanceof ProcessNonZeroExitStatusException ) { + final reason = failure.reason + return reason ? pattern.matcher(reason).find() : false + } + return false + } + } + } + + protected RetryPolicy retryPolicy() { + + final delay = executor.session.getConfigAttribute("executor.retry.delay", '500ms') as Duration + final maxDelay = executor.session.getConfigAttribute("executor.retry.maxDelay", '30s') as Duration + final jitter = executor.session.getConfigAttribute("executor.retry.jitter", '0.25') as double + final maxAttempts = executor.session.getConfigAttribute("executor.retry.maxAttempts", '3') as int + final reason = executor.session.getConfigAttribute("executor.submit.retry.reason", 'Socket timed out') as String + + final listener = new EventListener() { + @Override + void accept(ExecutionAttemptedEvent event) throws Throwable { + final failure = event.getLastFailure() + if( failure instanceof ProcessNonZeroExitStatusException ) { + final msg = """\ + Failed to submit process '${task.name}' + - attempt : ${event.attemptCount} + - command : ${failure.command} + - reason : ${failure.reason} + """.stripIndent(true) + log.warn msg + + } else { + log.debug("Unexpected retry failure: ${failure?.message}", failure) + } + } + } + + return RetryPolicy.builder() + .handleIf(retryCondition(reason)) + .withBackoff(delay.toMillis(), maxDelay.toMillis(), ChronoUnit.MILLIS) + .withMaxAttempts(maxAttempts) + .withJitter(jitter) + .onFailedAttempt(listener) + .build() + } + + protected T safeExecute(CheckedSupplier action) { + final policy = retryPolicy() + return Failsafe.with(policy).get(action) + } + + protected String processStart(ProcessBuilder builder, String pipeScript) { + final process = builder.start() + + try { + // -- forward the job launcher script to the command stdin if required + if( pipeScript ) { + log.trace "[${executor.name.toUpperCase()}] Submit STDIN command ${task.name} >\n${pipeScript.indent()}" + process.out << pipeScript + process.out.close() + } + + // -- wait the the process completes + final result = process.text + final exitStatus = process.waitFor() + final cmd = launchCmd0(builder,pipeScript) + + if( exitStatus ) { + throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) + } + + // -- return the process stdout + return result + } + finally { + // make sure to release all resources + process.in.closeQuietly() + process.out.closeQuietly() + process.err.closeQuietly() + process.destroy() + } } protected BashWrapperBuilder createTaskWrapper(TaskRun task) { @@ -102,8 +211,7 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { : executor.createBashWrapperBuilder(task) } - @Override - String stdinLauncherScript() { + protected String stdinLauncherScript() { return fusionEnabled() ? fusionStdinWrapper() : wrapperFile.text } @@ -126,6 +234,22 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { return result } + protected String launchCmd0(ProcessBuilder builder, String pipeScript) { + def result = CmdLineHelper.toLine(builder.command()) + if( pipeScript ) { + result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" + result += pipeScript.trim() + '\n' + result += 'LAUNCH_COMMAND_EOF\n' + } + return result + } + + @Override + void prepareLauncher() { + // -- create the wrapper script + createTaskWrapper(task).build() + } + /* * {@inheritDocs} */ @@ -136,29 +260,47 @@ class GridTaskHandler extends TaskHandler implements SubmitJobAware { return } - this.jobId = submitJob(executor.pipeLauncherScript()) - this.status = SUBMITTED + ProcessBuilder builder = null + try { + // -- start the execution and notify the event to the monitor + builder = createProcessBuilder() + // -- forward the job launcher script to the command stdin if required + final stdinScript = executor.pipeLauncherScript() ? stdinLauncherScript() : null + // -- execute with a re-triable strategy + final result = safeExecute( () -> processStart(builder, stdinScript) ) + // -- save the JobId in the + final jobId = executor.parseJobId(result) + this.onSubmit(jobId) + log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" + + } + catch( Exception e ) { + // update task exit status and message + if( e instanceof ProcessNonZeroExitStatusException ) { + task.exitStatus = e.getExitStatus() + task.stdout = e.getReason() + task.script = e.getCommand() + } + else { + task.script = builder ? CmdLineHelper.toLine(builder.command()) : null + } + status = COMPLETED + throw new ProcessFailedException("Error submitting process '${task.name}' for execution", e ) + } - log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" } - @Override - Exception submitError(Exception e, String submitCommand) { - // update task exit status and message - if( e instanceof ProcessNonZeroExitStatusException ) { - task.exitStatus = e.getExitStatus() - task.stdout = e.getReason() - task.script = e.getCommand() + void onSubmit(String jobId) { + if( task instanceof TaskArray ) { + ((TaskArray)task).children.eachWithIndex { handler, i -> + final arrayTaskId = executor.getArrayTaskId(jobId, i) + ((GridTaskHandler)handler).onSubmit(arrayTaskId) + } } else { - task.script = submitCommand + this.jobId = jobId + this.status = SUBMITTED } - this.status = COMPLETED - return new ProcessFailedException("Error submitting process '${task.name}' for execution", e ) - } - - void setJobId(String jobId) { - this.jobId = jobId } private long startedMillis diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy index b1b6be15df..0805c16609 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/HyperQueueExecutor.groovy @@ -56,7 +56,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { result << '--name' << getJobNameFor(task) result << '--log' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) @@ -81,7 +81,7 @@ class HyperQueueExecutor extends AbstractGridExecutor { } @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List getSubmitCommandLine(TaskRun task, Path scriptFile) { return TupleHelper.listOf('hq', '--output-mode=quiet', 'submit', '--directives=file', scriptFile.getName()) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index bd992f06ef..d94a7af24d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -23,6 +23,7 @@ import java.util.regex.Pattern import groovy.util.logging.Slf4j import nextflow.fusion.FusionHelper +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * Processor for LSF resource manager @@ -67,7 +68,12 @@ class LsfExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { + + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + result << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]" + } result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() @@ -113,10 +119,16 @@ class LsfExecutor extends AbstractGridExecutor { return result } + + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - List.of('bsub') - } + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { ['bsub'] } /** * @return {@code true} since BSC grid requires the script to be piped to the {@code bsub} command @@ -298,14 +310,10 @@ class LsfExecutor extends AbstractGridExecutor { return FusionHelper.isFusionEnabled(session) } - @Override - List getArrayDirective(int arraySize, TaskRun task) { - ['-J', "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]"] - } - @Override String getArrayIndexName() { 'LSB_JOBINDEX' } @Override String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy index 4a65e690e6..21c47af000 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/MoabExecutor.groovy @@ -39,7 +39,7 @@ class MoabExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives( TaskRun task, List result ) { + protected List getDirectives( TaskRun task, List result ) { assert result !=null result << '-N' << getJobNameFor(task) @@ -75,9 +75,19 @@ class MoabExecutor extends AbstractGridExecutor { return result } - @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - List.of('msub', '--xml', scriptFile.name) + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + def cmd = new ArrayList(5) + cmd << 'msub' + cmd << '--xml' + cmd << scriptFile.name + return cmd } protected String getHeaderToken() { '#MSUB' } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy index c441e6c28e..0a60e17485 100755 --- a/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/NqsiiExecutor.groovy @@ -38,7 +38,7 @@ class NqsiiExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) @@ -76,9 +76,12 @@ class NqsiiExecutor extends AbstractGridExecutor { return result } - @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - List.of('qsub', scriptFile.name) + + /* + * Prepare the 'qsub' cmdline + */ + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + return ['qsub', scriptFile.name] } protected String getHeaderToken() { '#PBS' } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy index acc381c35a..e8a9820a3b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/OarExecutor.groovy @@ -38,7 +38,7 @@ class OarExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { result << '-d' << quote(task.workDir) result << '-n' << getJobNameFor(task) @@ -90,11 +90,18 @@ class OarExecutor extends AbstractGridExecutor { String getHeaderToken() { '#OAR' } + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { // Scripts need to be executable scriptFile.setPermissions(7,0,0) - List.of("oarsub", "-S", "./${scriptFile.getName()}") + return ["oarsub", "-S", "./${scriptFile.getName()}"] } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 26f4d21171..c8bfcc6700 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -20,6 +20,7 @@ import java.nio.file.Path import java.util.regex.Pattern import groovy.util.logging.Slf4j +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * Implements a executor for PBS/Torque cluster @@ -38,9 +39,14 @@ class PbsExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives( TaskRun task, List result ) { + protected List getDirectives( TaskRun task, List result ) { assert result !=null + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + result << '-J' << "0-${arraySize - 1}" + } + result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) result << '-j' << 'oe' @@ -88,13 +94,17 @@ class PbsExecutor extends AbstractGridExecutor { name.size()>15 ? name.substring(0,15) : name } - @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { // in some PBS implementation the submit command will fail if the script name starts with a dot eg `.command.run` // add the `-N ` to fix this -- see issue #228 - pipeLauncherScript - ? List.of('qsub') - : List.of('qsub', '-N', getJobNameFor(task), scriptFile.getName()) + [ 'qsub', '-N', getJobNameFor(task), scriptFile.getName() ] } protected String getHeaderToken() { '#PBS' } @@ -169,11 +179,6 @@ class PbsExecutor extends AbstractGridExecutor { value ? OPTS_REGEX.matcher(value).find() : null } - @Override - List getArrayDirective(int arraySize, TaskRun task) { - ['-J', "0-${arraySize - 1}"] - } - @Override String getArrayIndexName() { 'PBS_ARRAY_INDEX' } @@ -181,4 +186,5 @@ class PbsExecutor extends AbstractGridExecutor { String getArrayTaskId(String jobId, int index) { jobId.replace('[]', "[$index]") } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index d6e944b9d0..6a25e3b684 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -18,6 +18,7 @@ package nextflow.executor import groovy.util.logging.Slf4j +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * Implements a executor for PBSPro cluster executor @@ -41,9 +42,14 @@ class PbsProExecutor extends PbsExecutor { * @return A {@link List} containing all directive tokens and values. */ @Override - List getDirectives(TaskRun task, List result ) { + protected List getDirectives(TaskRun task, List result ) { assert result !=null - + + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + result << '-J' << "0-${arraySize - 1}" + } + // when multiple competing directives are provided, only the first one will take effect // therefore clusterOptions is added as first to give priority over other options as expected // by the clusterOptions semantics -- see https://github.com/nextflow-io/nextflow/pull/2036 diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index c7a5a2f4ba..e1f3b6bb3a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -18,6 +18,7 @@ package nextflow.executor import java.nio.file.Path import nextflow.fusion.FusionHelper +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * Execute a task script by running it on the SGE/OGE cluster @@ -33,7 +34,12 @@ class SgeExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { + + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + result << '-t' << "0-${arraySize - 1}" + } result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) @@ -79,14 +85,16 @@ class SgeExecutor extends AbstractGridExecutor { return result } - @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { + /* + * Prepare the 'qsub' cmdline + */ + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { // The '-terse' command line control the output of the qsub command line, when // used it only return the ID of the submitted job. // NOTE: In some SGE implementations the '-terse' only works on the qsub command line // and it is ignored when used in the script job as directive, fir this reason it // should not be remove from here - return pipeLauncherScript + return pipeLauncherScript() ? List.of('qsub', '-') : List.of('qsub', '-terse', scriptFile.name) } @@ -182,14 +190,10 @@ class SgeExecutor extends AbstractGridExecutor { return FusionHelper.isFusionEnabled(session) } - @Override - List getArrayDirective(int arraySize, TaskRun task) { - ['-t', "0-${arraySize - 1}"] - } - @Override String getArrayIndexName() { 'SGE_TASK_ID' } @Override String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index d99aa504f0..53b208417a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -21,6 +21,7 @@ import java.util.regex.Pattern import groovy.util.logging.Slf4j import nextflow.fusion.FusionHelper +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * Processor for SLURM resource manager @@ -48,8 +49,12 @@ class SlurmExecutor extends AbstractGridExecutor { * @param result The {@link List} instance to which add the job directives * @return A {@link List} containing all directive tokens and values. */ - @Override - List getDirectives(TaskRun task, List result) { + protected List getDirectives(TaskRun task, List result) { + + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + result << '--array' << "0-${arraySize - 1}" + } result << '-J' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) // -o OUTFILE and no -e option => stdout and stderr merged to stdout/OUTFILE @@ -91,12 +96,18 @@ class SlurmExecutor extends AbstractGridExecutor { return result } - @Override String getHeaderToken() { '#SBATCH' } + /** + * The command line to submit this job + * + * @param task The {@link TaskRun} instance to submit for execution to the cluster + * @param scriptFile The file containing the job launcher script + * @return A list representing the submit command line + */ @Override - List getSubmitCommandLine(TaskRun task, Path scriptFile, boolean pipeLauncherScript) { - return pipeLauncherScript + List getSubmitCommandLine(TaskRun task, Path scriptFile ) { + return pipeLauncherScript() ? List.of('sbatch') : List.of('sbatch', scriptFile.getName()) } @@ -193,14 +204,10 @@ class SlurmExecutor extends AbstractGridExecutor { return FusionHelper.isFusionEnabled(session) } - @Override - List getArrayDirective(int arraySize, TaskRun task) { - ['--array', "0-${arraySize - 1}"] - } - @Override String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } @Override String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy deleted file mode 100644 index 00eaa8a7ce..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SubmitJobAware.groovy +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.executor - -import java.time.temporal.ChronoUnit -import java.util.function.Predicate -import java.util.regex.Pattern - -import dev.failsafe.Failsafe -import dev.failsafe.RetryPolicy -import dev.failsafe.event.EventListener -import dev.failsafe.event.ExecutionAttemptedEvent -import dev.failsafe.function.CheckedSupplier -import groovy.transform.Memoized -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.exception.ProcessNonZeroExitStatusException -import nextflow.fusion.FusionAwareTask -import nextflow.processor.TaskRun -import nextflow.util.CmdLineHelper -import nextflow.util.Duration -import org.slf4j.Logger -import org.slf4j.LoggerFactory -/** - * Implementation of job submission for grid executors. - * - * @author Paolo Di Tommaso - * @author Ben Sherman - */ -@CompileStatic -trait SubmitJobAware extends FusionAwareTask { - - static private Logger log = LoggerFactory.getLogger(SubmitJobAware) - - abstract AbstractGridExecutor getExecutor() - - abstract TaskRun getTask() - - String submitJob(boolean pipeLauncherScript) { - ProcessBuilder builder = null - try { - // -- create the stdin launcher script - final stdinScript = pipeLauncherScript ? stdinLauncherScript() : null - - // -- create the submit command - builder = createProcessBuilder(pipeLauncherScript) - - // -- submit the job with a retryable strategy - final result = safeExecute( () -> launchProcess(builder, stdinScript) ) - - return (String)executor.parseJobId(result) - } - catch( Exception e ) { - final submitCommand = builder ? CmdLineHelper.toLine(builder.command()) : null - throw submitError(e, submitCommand) - } - } - - abstract String stdinLauncherScript() - - abstract Exception submitError(Exception e, String submitCommand) - - ProcessBuilder createProcessBuilder(boolean pipeLauncherScript) { - - // -- log the submit command line - final cli = executor.getSubmitCommandLine(task, task.workDir.resolve(TaskRun.CMD_RUN), pipeLauncherScript) - log.trace "start process ${task.name} > cli: ${cli}" - - // -- create the submit process - ProcessBuilder builder = new ProcessBuilder() - .command( cli as String[] ) - .redirectErrorStream(true) - - if( !pipeLauncherScript ) - builder .directory(task.workDir.toFile()) - - return builder - } - - String launchProcess(ProcessBuilder builder, String pipeScript) { - final process = builder.start() - - try { - // -- forward the job launcher script to the command stdin if required - if( pipeScript ) { - log.trace "[${executor.name.toUpperCase()}] Submit STDIN command ${task.name} >\n${pipeScript.indent()}" - process.out << pipeScript - process.out.close() - } - - // -- wait the the process completes - final result = process.text - final exitStatus = process.waitFor() - final cmd = getLaunchCommand(builder,pipeScript) - - if( exitStatus ) { - throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) - } - - // -- return the process stdout - return result - } - finally { - // make sure to release all resources - process.in.closeQuietly() - process.out.closeQuietly() - process.err.closeQuietly() - process.destroy() - } - } - - String getLaunchCommand(ProcessBuilder builder, String pipeScript) { - def result = CmdLineHelper.toLine(builder.command()) - if( pipeScript ) { - result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" - result += pipeScript.trim() + '\n' - result += 'LAUNCH_COMMAND_EOF\n' - } - return result - } - - T safeExecute(CheckedSupplier action) { - Failsafe.with(retryPolicy()).get(action) - } - - RetryPolicy retryPolicy() { - - final delay = executor.session.getConfigAttribute('executor.retry.delay', '500ms') as Duration - final maxDelay = executor.session.getConfigAttribute('executor.retry.maxDelay', '30s') as Duration - final jitter = executor.session.getConfigAttribute('executor.retry.jitter', '0.25') as double - final maxAttempts = executor.session.getConfigAttribute('executor.retry.maxAttempts', '3') as int - final reason = executor.session.getConfigAttribute('executor.submit.retry.reason', 'Socket timed out') as String - - RetryPolicy.builder() - .handleIf(retryCondition(reason)) - .withBackoff(delay.toMillis(), maxDelay.toMillis(), ChronoUnit.MILLIS) - .withMaxAttempts(maxAttempts) - .withJitter(jitter) - .onFailedAttempt(new RetryListener(task: task)) - .build() - } - - static private class RetryListener extends EventListener { - TaskRun task - - @Override - void accept(ExecutionAttemptedEvent event) throws Throwable { - final failure = event.getLastFailure() - if( failure instanceof ProcessNonZeroExitStatusException ) { - final msg = """\ - Failed to submit process '${task.name}' - - attempt : ${event.attemptCount} - - command : ${failure.command} - - reason : ${failure.reason} - """.stripIndent(true) - log.warn msg - } else { - log.debug("Unexpected retry failure: ${failure?.message}", failure) - } - } - } - - @Memoized - Predicate retryCondition(String reasonPattern) { - new RetryPredicate(pattern: Pattern.compile(reasonPattern)) - } - - static private class RetryPredicate extends Predicate { - Pattern pattern - - @Override - boolean test(Throwable failure) { - if( failure instanceof ProcessNonZeroExitStatusException ) { - final reason = failure.reason - return reason ? pattern.matcher(reason).find() : false - } - return false - } - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index d69d30a2d8..f7af14cd69 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -16,6 +16,8 @@ package nextflow.executor +import java.nio.file.Path + import nextflow.processor.TaskHandler import nextflow.processor.TaskRun @@ -28,23 +30,25 @@ interface TaskArrayAware { String getName() + Path getWorkDir() + void submit( TaskRun task ) TaskHandler createTaskHandler(TaskRun task) /** - * Create a submitter for an array job. - * - * @param array + * Get the environment variable name that provides the array index of a task. */ - default TaskArraySubmitter createTaskArraySubmitter(List array) { - new TaskArraySubmitter(array) - } - default String getArrayIndexName() { throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") } + /** + * Get the job ID of an array task based on its index in the array. + * + * @param jobId + * @param index + */ default String getArrayTaskId(String jobId, int index) { throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index 32595cbd3e..4cca97141d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -16,11 +16,19 @@ package nextflow.executor +import java.nio.file.FileSystems +import java.nio.file.Files import java.util.concurrent.atomic.AtomicInteger import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import nextflow.file.FileHelper +import nextflow.processor.TaskArray +import nextflow.processor.TaskContext import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun +import nextflow.util.CacheHelper +import nextflow.util.Escape /** * Submit tasks as an array job. @@ -31,20 +39,23 @@ import nextflow.processor.TaskHandler @CompileStatic class TaskArraySubmitter { - protected List array + private List array + + private TaskArrayAware executor private AtomicInteger collected = new AtomicInteger() - TaskArraySubmitter(List array) { + TaskArraySubmitter(List array, TaskArrayAware executor) { this.array = array - - for( TaskHandler handler : array ) - handler.arraySubmitter = this + this.executor = executor } /** * Mark a task as ready to be submitted. * + * When all tasks in the array are ready, the array job + * will be submitted. + * * @param handler */ void collect(TaskHandler handler) { @@ -54,14 +65,66 @@ class TaskArraySubmitter { /** * Submit the array job. - * - * By default, this method simply submits each task individually. - * It should be overridden to submit an array job to the underlying - * executor. */ protected void submit() { - for( TaskHandler handler : array ) - handler.submit() + final tasks = array.collect( h -> h.task ) + final first = tasks.first() + + // create work directory + final hash = CacheHelper.hasher( tasks.collect( t -> t.getHash().asLong() ) ).hash() + final workDir = FileHelper.getWorkFolder(executor.getWorkDir(), hash) + + Files.createDirectories(workDir) + + // create wrapper script + final script = createWrapperScript(tasks) + + // create task handler + final arrayTask = new TaskArray( + id: first.id, + index: first.index, + processor: first.processor, + type: first.type, + config: first.processor.config.createTaskConfig(), + context: new TaskContext(first.processor), + hash: hash, + workDir: workDir, + script: script, + children: array + ) + final arrayHandler = executor.createTaskHandler(arrayTask) + + // submit array job + arrayHandler.prepareLauncher() + arrayHandler.submit() + } + + /** + * Create the wrapper script for an array job. + */ + protected String createWrapperScript(List tasks) { + // get work directory and launch command for each task + def workDirs + def cmd + + if( executor.workDir.fileSystem == FileSystems.default ) { + workDirs = tasks.collect( t -> Escape.path(t.workDir) ) + cmd = "cd \${task_dir} ; bash ${TaskRun.CMD_RUN} &> ${TaskRun.CMD_LOG}" + } + else { + workDirs = tasks.collect( t -> Escape.path(t.workDir.toUriString()) ) + cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) + cmd = cmd.replace(tasks.first().workDir.toUriString(), '\\${task_dir}') + } + + // create wrapper script + final arrayIndexName = executor.getArrayIndexName() + + """ + declare -a array=( ${workDirs.join(' ')} ) + task_dir=\${array[\$${arrayIndexName}]} + ${cmd} + """.stripIndent().trim() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy index 2e98718ced..aa0f1f4f5c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/local/LocalExecutor.groovy @@ -19,7 +19,6 @@ package nextflow.executor.local import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.executor.TaskArrayAware import nextflow.executor.Executor import nextflow.executor.SupportedScriptTypes import nextflow.fusion.FusionHelper @@ -36,7 +35,7 @@ import nextflow.script.ScriptType @Slf4j @CompileStatic @SupportedScriptTypes( [ScriptType.SCRIPTLET, ScriptType.GROOVY] ) -class LocalExecutor extends Executor implements TaskArrayAware { +class LocalExecutor extends Executor { @Override protected TaskMonitor createTaskMonitor() { diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy new file mode 100644 index 0000000000..9c66f17d62 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy @@ -0,0 +1,34 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.processor + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j + +/** + * Models a task array, which submits a collection of independent + * tasks with a single submit script. + * + * @author Ben Sherman + */ +@Slf4j +@CompileStatic +class TaskArray extends TaskRun { + + List children + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 8b72eff334..85fd30e43a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -16,14 +16,14 @@ package nextflow.processor -import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.locks.Lock import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.executor.TaskArrayAware import nextflow.executor.Executor +import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArraySubmitter /** * Task monitor that batches tasks and submits them as array jobs @@ -109,11 +109,14 @@ class TaskArrayCollector { protected void submit0(List array) { // create submitter for array job - executor.createTaskArraySubmitter(array) + final arraySubmitter = new TaskArraySubmitter(array, executor) // submit each task to the underlying monitor - for( TaskHandler handler : array ) + // each task will defer to the array job during submission + for( TaskHandler handler : array ) { + handler.arraySubmitter = arraySubmitter monitor.schedule(handler) + } } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 341b274aef..e7fae218f9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -99,6 +99,11 @@ abstract class TaskHandler { */ void prepareLauncher() {} + /** + * Get the command to execute the launcher script. + */ + List getSubmitCommand() { null } + /** * Task status attribute setter. * diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index df15954065..01a028e05a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -66,7 +66,7 @@ class GridTaskHandlerTest extends Specification { exec.pipeLauncherScript() >> false and: handler.fusionEnabled() >> false - handler.createProcessBuilder(false) >> new ProcessBuilder().command([]) + handler.createProcessBuilder() >> GroovyMock(ProcessBuilder) and: thrown(ProcessFailedException) and: @@ -132,9 +132,9 @@ class GridTaskHandlerTest extends Specification { def exec = Spy(GridTaskHandler) expect: - exec.getLaunchCommand(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' + exec.launchCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' and: - exec.getLaunchCommand(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == + exec.launchCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == '''\ cat << 'LAUNCH_COMMAND_EOF' | qsub docker run /some/file diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 405dd00640..9581232f02 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -308,11 +308,6 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar ThreadPoolHelper.await(reaper, Duration.of('60min'), waitMsg, exitMsg) } - @Override - TaskArraySubmitter createTaskArraySubmitter(List array) { - new AwsBatchTaskArraySubmitter(array, this) - } - @Override String getArrayIndexName() { 'AWS_BATCH_JOB_ARRAY_INDEX' } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy deleted file mode 100644 index 25c0c55b28..0000000000 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskArraySubmitter.groovy +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.cloud.aws.batch - -import static nextflow.fusion.FusionConfig.FUSION_PATH -import static nextflow.fusion.FusionHelper.* - -import com.amazonaws.services.batch.AWSBatch -import com.amazonaws.services.batch.model.ArrayProperties -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.executor.TaskArraySubmitter -import nextflow.processor.TaskHandler -import nextflow.processor.TaskRun -import nextflow.processor.TaskStatus -/** - * Submit tasks as an array job for a grid executor. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class AwsBatchTaskArraySubmitter extends TaskArraySubmitter implements SubmitJobAware { - - private AwsBatchExecutor executor - - private AWSBatch client - - AwsBatchTaskArraySubmitter(List array, AwsBatchExecutor executor) { - super(array) - this.executor = executor - this.client = executor.client - - if( array.size() > 10_000 ) - throw new IllegalArgumentException("Array jobs on AWS Batch may not have size greater than 10,000") - } - - @Override - TaskRun getTask() { array.first().getTask() } - - @Override - AWSBatch getClient() { executor.client } - - @Override - AwsOptions getAwsOptions() { executor.getAwsOptions() } - - @Override - List getSubmitCommand() { - // create wrapper script - final arrayIndexName = executor.getArrayIndexName() - final workDirs = fusionEnabled() - ? array.collect { h -> toContainerMount(h.task.workDir).toString() } - : array.collect { h -> h.task.workDir.toUriString() } - - def cmd = ((AwsBatchTaskHandler)array.first()) - .getSubmitCommand() - .last() - .replaceAll(workDirs.first(), '\\$task_dir') - - if( fusionEnabled() ) - cmd = "bash ${cmd}" - - cmd = """ - declare -a array=( ${workDirs.join(' ')} ) - task_dir=\${array[\$${arrayIndexName}]} - ${cmd} - """.stripIndent().trim() - - // create command line - final cli = ['bash', '-o', 'pipefail', '-c', cmd.toString()] - if( fusionEnabled() ) - cli.add(0, FUSION_PATH) - - return cli - } - - @Override - void submit() { - // -- create the submit request - final request = newSubmitRequest(getTask()) - .withArrayProperties(new ArrayProperties().withSize(array.size())) - - // -- submit the array job - final response = submitJobRequest(bypassProxy(client), request) - final jobId = response.jobId - - // -- set the job id, queue, and status of each task - array.eachWithIndex { handler, i -> - ((AwsBatchTaskHandler)handler).jobId = executor.getArrayTaskId(jobId, i) - ((AwsBatchTaskHandler)handler).queueName = request.getJobQueue() - handler.status = TaskStatus.SUBMITTED - } - - log.debug "[AWS BATCH] submitted array job > jobId: ${jobId}" - } - -} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 866fd53a85..04cd5c3a15 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -16,34 +16,66 @@ package nextflow.cloud.aws.batch +import static nextflow.cloud.aws.batch.AwsContainerOptionsMapper.* + import java.nio.file.Path +import java.nio.file.Paths +import java.time.Instant import com.amazonaws.services.batch.AWSBatch +import com.amazonaws.services.batch.model.AWSBatchException +import com.amazonaws.services.batch.model.ArrayProperties import com.amazonaws.services.batch.model.AttemptContainerDetail +import com.amazonaws.services.batch.model.ClientException +import com.amazonaws.services.batch.model.ContainerOverrides +import com.amazonaws.services.batch.model.ContainerProperties +import com.amazonaws.services.batch.model.DescribeJobDefinitionsRequest +import com.amazonaws.services.batch.model.DescribeJobDefinitionsResult import com.amazonaws.services.batch.model.DescribeJobsRequest import com.amazonaws.services.batch.model.DescribeJobsResult +import com.amazonaws.services.batch.model.EvaluateOnExit +import com.amazonaws.services.batch.model.Host +import com.amazonaws.services.batch.model.JobDefinition +import com.amazonaws.services.batch.model.JobDefinitionType import com.amazonaws.services.batch.model.JobDetail +import com.amazonaws.services.batch.model.JobTimeout +import com.amazonaws.services.batch.model.KeyValuePair +import com.amazonaws.services.batch.model.LogConfiguration +import com.amazonaws.services.batch.model.MountPoint +import com.amazonaws.services.batch.model.RegisterJobDefinitionRequest +import com.amazonaws.services.batch.model.RegisterJobDefinitionResult +import com.amazonaws.services.batch.model.ResourceRequirement +import com.amazonaws.services.batch.model.ResourceType +import com.amazonaws.services.batch.model.RetryStrategy +import com.amazonaws.services.batch.model.SubmitJobRequest +import com.amazonaws.services.batch.model.SubmitJobResult import com.amazonaws.services.batch.model.TerminateJobRequest -import groovy.transform.CompileDynamic +import com.amazonaws.services.batch.model.Volume import groovy.transform.CompileStatic +import groovy.transform.Memoized import groovy.util.logging.Slf4j +import nextflow.Const import nextflow.cloud.types.CloudMachineInfo +import nextflow.container.ContainerNameValidator +import nextflow.exception.ProcessSubmitException import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder +import nextflow.executor.res.AcceleratorResource +import nextflow.fusion.FusionAwareTask import nextflow.processor.BatchContext import nextflow.processor.BatchHandler +import nextflow.processor.TaskArray import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus import nextflow.trace.TraceRecord +import nextflow.util.CacheHelper /** - * Implements a task handler for AWS Batch jobs. - * - * @author Paolo Di Tommaso + * Implements a task handler for AWS Batch jobs */ +// note: do not declare this class as `CompileStatic` otherwise the proxy is not get invoked @Slf4j -@CompileStatic -class AwsBatchTaskHandler extends TaskHandler implements BatchHandler, SubmitJobAware { +class AwsBatchTaskHandler extends TaskHandler implements BatchHandler, FusionAwareTask { private final Path exitFile @@ -63,13 +95,19 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler environment + + final static private Map jobDefinitions = [:] /** * Batch context shared between multiple task handlers @@ -88,6 +126,8 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler job=$jobId; work-dir=${task.getWorkDirStr()}" } - void setJobId(String jobId) { - this.jobId = jobId - } - - void setQueueName(String queueName) { - this.queueName = queueName + void onSubmit(String jobId, String queueName) { + if( task instanceof TaskArray ) { + ((TaskArray)task).children.eachWithIndex { handler, i -> + final arrayTaskId = executor.getArrayTaskId(jobId, i) + ((AwsBatchTaskHandler)handler).onSubmit(arrayTaskId, queueName) + } + } + else { + this.jobId = jobId + this.queueName = queueName + this.status = TaskStatus.SUBMITTED + } } protected BashWrapperBuilder createTaskWrapper() { @@ -313,6 +352,296 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler MAX_ATTEMPTS) + throw e + + final delay = (Math.pow(DEFAULT_BACK_OFF_BASE, attempt) as long) * DEFAULT_BACK_OFF_DELAY + log.debug "Got AWS Client exception on Batch resolve job definition - message=$e.message; waiting for ${delay}ms (attempt=$attempt)" + Thread.sleep(delay) + } + } + } + + protected String resolveJobDefinition0(String container) { + final req = makeJobDefRequest(container) + final token = req.getParameters().get('nf-token') + final jobKey = "$container:$token".toString() + if( jobDefinitions.containsKey(jobKey) ) + return jobDefinitions[jobKey] + + synchronized(jobDefinitions) { + if( jobDefinitions.containsKey(jobKey) ) + return jobDefinitions[jobKey] + + def msg + def name = findJobDef(req.jobDefinitionName, token) + if( name ) { + msg = "[AWS BATCH] Found job definition name=$name; container=$container" + } + else { + name = createJobDef(req) + msg = "[AWS BATCH] Created job definition name=$name; container=$container" + } + // log the request + if( log.isTraceEnabled() ) + log.debug "[AWS BATCH] $msg; request=${req.toString().indent()}" + else + log.debug "[AWS BATCH] $msg" + + jobDefinitions[jobKey] = name + return name + } + } + + /** + * Create a Batch job definition request object for the specified Docker image + * + * @param image The Docker container image for which is required to create a Batch job definition + * @return An instance of {@link com.amazonaws.services.batch.model.RegisterJobDefinitionRequest} for the specified Docker image + */ + protected RegisterJobDefinitionRequest makeJobDefRequest(String image) { + final uniq = new ArrayList() + final result = configJobDefRequest(image, uniq) + + // create a job marker uuid + def hash = computeUniqueToken(uniq) + result.setParameters(['nf-token':hash]) + + return result + } + + protected String computeUniqueToken(List uniq) { + return CacheHelper.hasher(uniq).hash().toString() + } + + /** + * Create and configure the actual RegisterJobDefinitionRequest object + * + * @param image + * The Docker container image for which is required to create a Batch job definition + * @param hashingTokens + * A list used to collect values that should be used to create a unique job definition Id for the given job request. + * It should be used to return such values in the calling context + * @return + * An instance of {@link com.amazonaws.services.batch.model.RegisterJobDefinitionRequest} for the specified Docker image + */ + protected RegisterJobDefinitionRequest configJobDefRequest(String image, List hashingTokens) { + final name = normalizeJobDefinitionName(image) + final opts = getAwsOptions() + + final result = new RegisterJobDefinitionRequest() + result.setJobDefinitionName(name) + result.setType(JobDefinitionType.Container) + + // create the container opts based on task config + final containerOpts = task.getConfig().getContainerOptionsMap() + final container = createContainerProperties(containerOpts) + + // container definition + final _1_cpus = new ResourceRequirement().withType(ResourceType.VCPU).withValue('1') + final _1_gb = new ResourceRequirement().withType(ResourceType.MEMORY).withValue('1024') + container + .withImage(image) + .withCommand('true') + // note the actual command, memory and cpus are overridden when the job is executed + .withResourceRequirements( _1_cpus, _1_gb ) + + final jobRole = opts.getJobRole() + if( jobRole ) + container.setJobRoleArn(jobRole) + + final logsGroup = opts.getLogsGroup() + if( logsGroup ) + container.setLogConfiguration(getLogConfiguration(logsGroup, opts.getRegion())) + + if( fusionEnabled() ) + container.setPrivileged(true) + + final mountsMap = new LinkedHashMap( 10) + final awscli = opts.cliPath + if( awscli ) { + def path = Paths.get(awscli).parent.parent.toString() + mountsMap.put('aws-cli', "$path:$path:ro") + } + + int c=0 + final volumes = opts.getVolumes() + for( String vol : volumes ) { + mountsMap.put("vol-"+(++c), vol) + } + + if( mountsMap ) + addVolumeMountsToContainer(mountsMap, container) + + // finally set the container options + result.setContainerProperties(container) + + // add to this list all values that has to contribute to the + // job definition unique name creation + hashingTokens.add(name) + hashingTokens.add(container.toString()) + if( containerOpts ) + hashingTokens.add(containerOpts) + + return result + } + + @Memoized + LogConfiguration getLogConfiguration(String name, String region) { + new LogConfiguration() + .withLogDriver('awslogs') + .withOptions([ + 'awslogs-region': region, + 'awslogs-group': name + ]) + } + + protected void addVolumeMountsToContainer(Map mountsMap, ContainerProperties container) { + final mounts = new ArrayList(mountsMap.size()) + final volumes = new ArrayList(mountsMap.size()) + for( Map.Entry entry : mountsMap.entrySet() ) { + final mountName = entry.key + final parts = entry.value.tokenize(':') + final containerPath = parts[0] + final hostPath = parts.size()>1 ? parts[1] : containerPath + final readOnly = parts.size()>2 ? parts[2]=='ro' : false + if( parts.size()>3 ) + throw new IllegalArgumentException("Not a valid volume mount syntax: $entry.value") + + def mount = new MountPoint() + .withSourceVolume(mountName) + .withContainerPath(hostPath) + .withReadOnly(readOnly) + mounts << mount + + def vol = new Volume() + .withName(mountName) + .withHost(new Host() + .withSourcePath(containerPath)) + volumes << vol + } + + if( mountsMap ) { + container.setMountPoints(mounts) + container.setVolumes(volumes) + } + } + + /** + * Look for a Batch job definition in ACTIVE status for the given name and NF job definition ID + * + * @param name The Batch job definition name + * @param jobId A unique job definition ID generated by NF + * @return The fully qualified Batch job definition name eg {@code my-job-definition:3} + */ + protected String findJobDef(String name, String jobId) { + log.trace "[AWS BATCH] checking job definition with name=$name; jobid=$jobId" + final req = new DescribeJobDefinitionsRequest().withJobDefinitionName(name) + // bypass the proxy because this method is invoked during a + // job submit request that's already in a separate thread pool request + // therefore it's protected by a TooManyRequestsException + final res = describeJobDefinitions0(bypassProxy(this.client), req) + final jobs = res.getJobDefinitions() + if( jobs.size()==0 ) + return null + + def job = jobs.find { JobDefinition it -> it.status == 'ACTIVE' && it.parameters?.'nf-token' == jobId } + return job ? "$name:$job.revision" : null + } + + /** + * Create (aka register) a new Batch job definition + * + * @param req A {@link RegisterJobDefinitionRequest} representing the Batch jib definition to create + * @return The fully qualified Batch job definition name eg {@code my-job-definition:3} + */ + protected String createJobDef(RegisterJobDefinitionRequest req) { + // add nextflow tags + req.addTagsEntry('nextflow.io/createdAt', Instant.now().toString()) + req.addTagsEntry('nextflow.io/version', Const.APP_VER) + // create the job def + final res = createJobDef0(bypassProxy(client), req) // bypass the client proxy! see #1024 + return "${res.jobDefinitionName}:$res.revision" + } + + /** + * Make a name string compliant with the Batch job definition format + * + * @param name A job name + * @return A given name formatted to be used as Job definition name + */ + protected String normalizeJobDefinitionName(String name) { + if( !name ) return null + if( !ContainerNameValidator.isValidImageName(name) ) throw new IllegalArgumentException("Invalid container image name: $name") + + def result = name.replaceAll(/[^a-zA-Z0-9\-_]+/,'-') + // Batch job definition length cannot exceed 128 characters + // take first 40 chars + add a unique MD5 hash (32 chars) + if( result.length()>125 ) { + final hash = name.md5() + result = result.substring(0,40) + '-' + hash + } + + return "nf-" + result + } + protected List classicSubmitCli() { // the cmd list to launch it final opts = getAwsOptions() @@ -333,6 +662,135 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler0 ) { + // retry the job when an Ec2 instance is terminate + final cond1 = new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*') + // the exit condition prevent to retry for other reason and delegate + // instead to nextflow error strategy the handling of the error + final cond2 = new EvaluateOnExit().withAction('EXIT').withOnReason('*') + final retry = new RetryStrategy() + .withAttempts( attempts ) + .withEvaluateOnExit(cond1, cond2) + result.setRetryStrategy(retry) + } + + // set task timeout + final time = task.config.getTime() + if( time ) { + def secs = time.toSeconds() as Integer + if( secs < 60 ) { + secs = 60 // Batch minimal allowed timeout is 60 seconds + } + result.setTimeout(new JobTimeout().withAttemptDurationSeconds(secs)) + } + + // set the actual command + final resources = new ArrayList(5) + def container = new ContainerOverrides() + container.command = getSubmitCommand() + // set the task memory + if( task.config.getMemory() ) { + final mega = (int)task.config.getMemory().toMega() + if( mega >= 4 ) + resources << new ResourceRequirement().withType(ResourceType.MEMORY).withValue(mega.toString()) + else + log.warn "Ignoring task ${task.lazyName()} memory directive: ${task.config.getMemory()} -- AWS Batch job memory request cannot be lower than 4 MB" + } + // set the task cpus + if( task.config.getCpus() > 1 ) + resources << new ResourceRequirement().withType(ResourceType.VCPU).withValue(task.config.getCpus().toString()) + + if( task.config.getAccelerator() ) + resources << createGpuResource(task.config.getAccelerator()) + + if( resources ) + container.withResourceRequirements(resources) + + // set the environment + def vars = getEnvironmentVars() + if( vars ) + container.setEnvironment(vars) + + result.setContainerOverrides(container) + + // set the array properties + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).children.size() + + if( arraySize > 10_000 ) + throw new IllegalArgumentException("Array jobs on AWS Batch may not have more than 10,000 tasks") + + result.setArrayProperties(new ArrayProperties().withSize(arraySize)) + } + + return result + } + + protected ResourceRequirement createGpuResource(AcceleratorResource acc) { + final res = new ResourceRequirement() + final type = acc.type ?: 'GPU' + final count = acc.request?.toString() ?: '1' + res.setType(type.toUpperCase()) + res.setValue(count) + return res + } + + /** + * @return The list of environment variables to be defined in the Batch job execution context + */ + protected List getEnvironmentVars() { + def vars = [] + if( this.environment?.containsKey('NXF_DEBUG') ) + vars << new KeyValuePair().withName('NXF_DEBUG').withValue(this.environment['NXF_DEBUG']) + if( this.getAwsOptions().retryMode && this.getAwsOptions().retryMode in AwsOptions.VALID_RETRY_MODES) + vars << new KeyValuePair().withName('AWS_RETRY_MODE').withValue(this.getAwsOptions().retryMode) + if( this.getAwsOptions().maxTransferAttempts ) { + vars << new KeyValuePair().withName('AWS_MAX_ATTEMPTS').withValue(this.getAwsOptions().maxTransferAttempts as String) + vars << new KeyValuePair().withName('AWS_METADATA_SERVICE_NUM_ATTEMPTS').withValue(this.getAwsOptions().maxTransferAttempts as String) + } + if( fusionEnabled() ) { + for(Map.Entry it : fusionLauncher().fusionEnv()) { + vars << new KeyValuePair().withName(it.key).withValue(it.value) + } + } + return vars + } + /** * @return The launcher script file {@link Path} */ @@ -343,6 +801,18 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler128 ? result.substring(0,128) : result + } + + protected CloudMachineInfo getMachineInfo() { if( machineInfo ) return machineInfo @@ -353,7 +823,6 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler=500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to submit job: ${req.jobName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + + static private DescribeJobDefinitionsResult describeJobDefinitions0(AWSBatch client, DescribeJobDefinitionsRequest req) { + try { + client.describeJobDefinitions(req) + } + catch (AWSBatchException e) { + if( e.statusCode>=500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to describe job definitions: ${req.jobDefinitions} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + + static private RegisterJobDefinitionResult createJobDef0(AWSBatch client, RegisterJobDefinitionRequest req) { + try { + return client.registerJobDefinition(req) + } + catch (AWSBatchException e) { + if( e.statusCode>=500 ) + // raise a process exception so that nextflow can try to recover it + throw new ProcessSubmitException("Failed to register job definition: ${req.jobDefinitionName} - Reason: ${e.errorCode}", e) + else + // status code < 500 are not expected to be recoverable, just throw it again + throw e + } + } + } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy deleted file mode 100644 index ac50bc50b3..0000000000 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/SubmitJobAware.groovy +++ /dev/null @@ -1,547 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.cloud.aws.batch - -import static nextflow.cloud.aws.batch.AwsContainerOptionsMapper.* - -import java.nio.file.Paths -import java.time.Instant - -import com.amazonaws.services.batch.AWSBatch -import com.amazonaws.services.batch.model.AWSBatchException -import com.amazonaws.services.batch.model.ClientException -import com.amazonaws.services.batch.model.ContainerOverrides -import com.amazonaws.services.batch.model.ContainerProperties -import com.amazonaws.services.batch.model.DescribeJobDefinitionsRequest -import com.amazonaws.services.batch.model.DescribeJobDefinitionsResult -import com.amazonaws.services.batch.model.EvaluateOnExit -import com.amazonaws.services.batch.model.Host -import com.amazonaws.services.batch.model.JobDefinition -import com.amazonaws.services.batch.model.JobDefinitionType -import com.amazonaws.services.batch.model.JobTimeout -import com.amazonaws.services.batch.model.KeyValuePair -import com.amazonaws.services.batch.model.LogConfiguration -import com.amazonaws.services.batch.model.MountPoint -import com.amazonaws.services.batch.model.RegisterJobDefinitionRequest -import com.amazonaws.services.batch.model.RegisterJobDefinitionResult -import com.amazonaws.services.batch.model.ResourceRequirement -import com.amazonaws.services.batch.model.ResourceType -import com.amazonaws.services.batch.model.RetryStrategy -import com.amazonaws.services.batch.model.SubmitJobRequest -import com.amazonaws.services.batch.model.SubmitJobResult -import com.amazonaws.services.batch.model.Volume -import groovy.transform.CompileStatic -import groovy.transform.Memoized -import nextflow.Const -import nextflow.container.ContainerNameValidator -import nextflow.exception.ProcessSubmitException -import nextflow.exception.ProcessUnrecoverableException -import nextflow.fusion.FusionAwareTask -import nextflow.processor.TaskRun -import nextflow.util.CacheHelper -import org.slf4j.Logger -import org.slf4j.LoggerFactory -/** - * Implementation of submit job requests for tasks. - * - * @author Paolo Di Tommaso - * @author Ben Sherman - */ -@CompileStatic -trait SubmitJobAware extends FusionAwareTask { - - static private Logger log = LoggerFactory.getLogger(SubmitJobAware) - - static private Map jobDefinitions = [:] - - Map environment = System.getenv() - - AWSBatch bypassProxy(AWSBatch client) { - client instanceof AwsBatchProxy ? client.getClient() : client - } - - /** - * Get the AWS Batch client. - */ - abstract AWSBatch getClient() - - /** - * Get the Nextflow configuration options for AWS Batch. - */ - abstract AwsOptions getAwsOptions() - - /** - * Get the submit command for a job request. - */ - abstract List getSubmitCommand() - - /** - * Create a new submit job request for a task. - * - * @param task - */ - SubmitJobRequest newSubmitRequest(TaskRun task) { - - // create the request object - final request = new SubmitJobRequest() - request.setJobName(normalizeJobName(task.name)) - request.setJobQueue(getJobQueue(task)) - request.setJobDefinition(getJobDefinition(task)) - - // set task timeout - final time = task.config.getTime() - if( time ) { - def secs = time.toSeconds() as Integer - if( secs < 60 ) { - secs = 60 // Batch minimal allowed timeout is 60 seconds - } - request.setTimeout(new JobTimeout().withAttemptDurationSeconds(secs)) - } - - // set the container overrides - final container = new ContainerOverrides() - - // set the submit command - container.setCommand(getSubmitCommand()) - - // set the environment vars - final env = getEnvironmentVars() - if( env ) - container.setEnvironment(env) - - // set the task resources - final resources = new ArrayList(5) - - if( task.config.getMemory() ) { - final mega = (int)task.config.getMemory().toMega() - if( mega >= 4 ) - resources << new ResourceRequirement().withType(ResourceType.MEMORY).withValue(mega.toString()) - else - log.warn "Ignoring task ${task.lazyName()} memory directive: ${task.config.getMemory()} -- AWS Batch job memory request cannot be lower than 4 MB" - } - - if( task.config.getCpus() > 1 ) - resources << new ResourceRequirement().withType(ResourceType.VCPU).withValue(task.config.getCpus().toString()) - - final accelerator = task.config.getAccelerator() - if( accelerator ) { - final type = accelerator.type?.toUpperCase() ?: 'GPU' - final count = accelerator.request?.toString() ?: '1' - - resources << new ResourceRequirement().withType(type).withValue(count) - } - - if( resources ) - container.withResourceRequirements(resources) - - request.setContainerOverrides(container) - - // set the resource labels - final labels = task.config.getResourceLabels() - if( labels ) { - request.setTags(labels) - request.setPropagateTags(true) - } - - // set the share identifier - if( awsOptions.shareIdentifier ) { - request.setShareIdentifier(awsOptions.shareIdentifier) - request.setSchedulingPriorityOverride(awsOptions.schedulingPriority) - } - - // retry on spot reclamation - // https://aws.amazon.com/blogs/compute/introducing-retry-strategies-for-aws-batch/ - if( awsOptions.maxSpotAttempts > 0 ) { - final retryStrategy = new RetryStrategy() - .withAttempts(awsOptions.maxSpotAttempts) - .withEvaluateOnExit( - // retry the job when an EC2 instance is terminated - new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*'), - // delegate all other exit conditions to Nextflow - new EvaluateOnExit().withAction('EXIT').withOnReason('*') - ) - - request.setRetryStrategy(retryStrategy) - } - - return request - } - - /** - * Remove invalid characters from a job name. - * - * @param name - */ - String normalizeJobName(String name) { - final result = name.replaceAll(' ','_').replaceAll(/[^a-zA-Z0-9_]/,'') - result.size() > 128 ? result.substring(0,128) : result - } - - /** - * Get the Batch queue name for a task. - * - * @param task - */ - String getJobQueue(TaskRun task) { - final queue = task.config.queue?.toString() - if( !queue ) - throw new ProcessUnrecoverableException("Missing AWS Batch job queue -- provide it by using the process `queue` directive") - - return queue - } - - /** - * Get the list of environment variables for a job request. - */ - List getEnvironmentVars() { - def vars = [] as List - - if( environment.containsKey('NXF_DEBUG') ) - vars << new KeyValuePair().withName('NXF_DEBUG').withValue(environment['NXF_DEBUG']) - - if( awsOptions.retryMode && awsOptions.retryMode in AwsOptions.VALID_RETRY_MODES ) - vars << new KeyValuePair().withName('AWS_RETRY_MODE').withValue(awsOptions.retryMode) - - if( awsOptions.maxTransferAttempts ) { - vars << new KeyValuePair().withName('AWS_MAX_ATTEMPTS').withValue(awsOptions.maxTransferAttempts as String) - vars << new KeyValuePair().withName('AWS_METADATA_SERVICE_NUM_ATTEMPTS').withValue(awsOptions.maxTransferAttempts as String) - } - - if( fusionEnabled() ) { - for( Map.Entry it : fusionLauncher().fusionEnv() ) { - vars << new KeyValuePair().withName(it.key).withValue(it.value) - } - } - - return vars - } - - /** - * Get the Batch job definition name for a task. - * - * @param task - */ - String getJobDefinition(TaskRun task) { - final container = task.getContainer() - if( !container ) - throw new ProcessUnrecoverableException("Invalid AWS Batch job definition -- provide a Docker image name or a Batch job definition name") - - if( container.startsWith('job-definition://')) - return container.substring(17) - - resolveJobDefinition(container) - } - - /** - * Get the Batch job definition name associated with a Docker container image. - * - * @param container - */ - String resolveJobDefinition(String container) { - final int DEFAULT_BACK_OFF_BASE = 3 - final int DEFAULT_BACK_OFF_DELAY = 250 - final int MAX_ATTEMPTS = 5 - int attempt = 0 - while( true ) { - try { - return resolveJobDefinition0(container) - } - catch (ClientException e) { - if( e.statusCode != 404 || attempt++ > MAX_ATTEMPTS) - throw e - - final delay = (Math.pow(DEFAULT_BACK_OFF_BASE, attempt) as long) * DEFAULT_BACK_OFF_DELAY - log.debug "Got AWS Client exception on Batch resolve job definition - message=$e.message; waiting for ${delay}ms (attempt=$attempt)" - Thread.sleep(delay) - } - } - } - - private String resolveJobDefinition0(String container) { - final request = makeJobDefRequest(container) - final token = request.getParameters().get('nf-token') - final jobKey = "$container:$token".toString() - if( jobDefinitions.containsKey(jobKey) ) - return jobDefinitions[jobKey] - - synchronized (jobDefinitions) { - if( jobDefinitions.containsKey(jobKey) ) - return jobDefinitions[jobKey] - - def msg - def name = findJobDefinition(request.jobDefinitionName, token) - if( name ) { - msg = "[AWS BATCH] Found job definition name=$name; container=$container" - } - else { - name = registerJobDefinition(request) - msg = "[AWS BATCH] Created job definition name=$name; container=$container" - } - // log the request - if( log.isTraceEnabled() ) - log.debug "[AWS BATCH] $msg; request=${request.toString().indent()}" - else - log.debug "[AWS BATCH] $msg" - - jobDefinitions[jobKey] = name - return name - } - } - - /** - * Create a Batch job definition request for a container image. - * - * @param container - */ - RegisterJobDefinitionRequest makeJobDefRequest(String container) { - // create the job definition request - final hashingTokens = new ArrayList() - final request = makeJobDefRequest0(container, hashingTokens) - - // create a unique id for the job definition - final hash = computeUniqueToken(hashingTokens) - request.setParameters(['nf-token': hash]) - - return request - } - - /** - * Create a unique id from a list of hashable tokens that represent - * a unique object. - * - * @param uniq - */ - private String computeUniqueToken(List uniq) { - return CacheHelper.hasher(uniq).hash().toString() - } - - /** - * Create a job definition request for a container image. - * - * The hashing tokens collect values that should be used to create a - * unique job definition id for the job request. - * - * @param image - * @param hashingTokens - */ - private RegisterJobDefinitionRequest makeJobDefRequest0(String image, List hashingTokens) { - final name = normalizeJobDefinitionName(image) - final opts = getAwsOptions() - - final request = new RegisterJobDefinitionRequest() - request.setJobDefinitionName(name) - request.setType(JobDefinitionType.Container) - - // apply the container options from the task configuration - final containerOpts = task.getConfig().getContainerOptionsMap() - final container = createContainerProperties(containerOpts) - - // set the container configuration - // the actual command, cpus, and memory are overridden when the job is executed - container - .withImage(image) - .withCommand('true') - .withResourceRequirements( - new ResourceRequirement().withType(ResourceType.VCPU).withValue('1'), - new ResourceRequirement().withType(ResourceType.MEMORY).withValue('1024') - ) - - // set the job role - final jobRole = opts.getJobRole() - if( jobRole ) - container.setJobRoleArn(jobRole) - - // set the logs group - final logsGroup = opts.getLogsGroup() - if( logsGroup ) - container.setLogConfiguration(getLogConfiguration(logsGroup, opts.getRegion())) - - // set privilged mode if fusion is enabled - if( fusionEnabled() ) - container.setPrivileged(true) - - // add the volume mounts - final mountsMap = new LinkedHashMap( 10) - if( opts.cliPath ) { - def path = Paths.get(opts.cliPath).parent.parent.toString() - mountsMap.put('aws-cli', "$path:$path:ro") - } - - int c = 0 - final volumes = opts.getVolumes() - for( String vol : volumes ) { - mountsMap.put("vol-${++c}", vol) - } - - if( mountsMap ) - addVolumeMountsToContainer(mountsMap, container) - - // set the container options - request.setContainerProperties(container) - - // add to the hashing tokens all values that contribute to the - // uniqueness of the job definition - hashingTokens.add(name) - hashingTokens.add(container.toString()) - if( containerOpts ) - hashingTokens.add(containerOpts) - - return request - } - - /** - * Normalize a name to be compliant with the Batch job definition format. - * - * @param name - */ - String normalizeJobDefinitionName(String name) { - if( !name ) - return null - if( !ContainerNameValidator.isValidImageName(name) ) - throw new IllegalArgumentException("Invalid container image name: $name") - - def result = name.replaceAll(/[^a-zA-Z0-9\-_]+/,'-') - // Batch job definition length cannot exceed 128 characters - // take first 40 chars + a unique MD5 hash (32 chars) - if( result.length() > 125 ) { - result = result.substring(0,40) + '-' + name.md5() - } - - return "nf-" + result - } - - @Memoized - LogConfiguration getLogConfiguration(String name, String region) { - new LogConfiguration() - .withLogDriver('awslogs') - .withOptions([ - 'awslogs-region': region, - 'awslogs-group': name - ]) - } - - void addVolumeMountsToContainer(Map mountsMap, ContainerProperties container) { - final mounts = new ArrayList(mountsMap.size()) - final volumes = new ArrayList(mountsMap.size()) - for( Map.Entry entry : mountsMap.entrySet() ) { - final mountName = entry.key - final parts = entry.value.tokenize(':') - final containerPath = parts[0] - final hostPath = parts.size() > 1 ? parts[1] : containerPath - final readOnly = parts.size() > 2 ? parts[2]=='ro' : false - if( parts.size() > 3 ) - throw new IllegalArgumentException("Not a valid volume mount syntax: $entry.value") - - def mount = new MountPoint() - .withSourceVolume(mountName) - .withContainerPath(hostPath) - .withReadOnly(readOnly) - mounts << mount - - def vol = new Volume() - .withName(mountName) - .withHost(new Host() - .withSourcePath(containerPath)) - volumes << vol - } - - if( mountsMap ) { - container.setMountPoints(mounts) - container.setVolumes(volumes) - } - } - - /** - * Search for a Batch job definition in ACTIVE status for the given name and job definition id. - * - * @param name - * @param jobId - * @return The fully qualified Batch job definition name, e.g. {@code my-job-definition:3} - */ - String findJobDefinition(String name, String jobId) { - log.trace "[AWS BATCH] checking job definition with name=$name; jobid=$jobId" - final request = new DescribeJobDefinitionsRequest().withJobDefinitionName(name) - // bypass the proxy because this method is invoked during a - // job submit request that's already in a separate thread pool request - // therefore it's private by a TooManyRequestsException - final response = describeJobDefinitions0(bypassProxy(client), request) - final jobs = response.getJobDefinitions() - if( jobs.size() == 0 ) - return null - - def job = jobs.find { JobDefinition it -> it.status == 'ACTIVE' && it.parameters?.'nf-token' == jobId } - return job ? "${name}:${job.revision}" : null - } - - static private DescribeJobDefinitionsResult describeJobDefinitions0(AWSBatch client, DescribeJobDefinitionsRequest req) { - try { - client.describeJobDefinitions(req) - } - catch( AWSBatchException e ) { - if( e.statusCode >= 500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to describe job definitions: ${req.jobDefinitions} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - - /** - * Register a new Batch job definition. - * - * @param request - * @return The fully qualified Batch job definition name, e.g. {@code my-job-definition:3} - */ - String registerJobDefinition(RegisterJobDefinitionRequest request) { - // add nextflow tags - request.addTagsEntry('nextflow.io/createdAt', Instant.now().toString()) - request.addTagsEntry('nextflow.io/version', Const.APP_VER) - // create the job def - final res = registerJobDefinition0(bypassProxy(client), request) // bypass the client proxy! see #1024 - return "${res.jobDefinitionName}:$res.revision" - } - - static private RegisterJobDefinitionResult registerJobDefinition0(AWSBatch client, RegisterJobDefinitionRequest req) { - try { - return client.registerJobDefinition(req) - } - catch( AWSBatchException e ) { - if( e.statusCode >= 500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to register job definition: ${req.jobDefinitionName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - - static SubmitJobResult submitJobRequest(AWSBatch client, SubmitJobRequest request) { - try { - return client.submitJob(request) - } - catch( AWSBatchException e ) { - if( e.statusCode >= 500 ) - // raise a process exception so that nextflow can try to recover it - throw new ProcessSubmitException("Failed to submit job: ${request.jobName} - Reason: ${e.errorCode}", e) - else - // status code < 500 are not expected to be recoverable, just throw it again - throw e - } - } - -} diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy index a42c12712b..0f7199688e 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy @@ -88,21 +88,18 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - maxSpotAttempts >> 5 - } - and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.maxSpotAttempts() >> 5 + _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - and: + req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] req.getRetryStrategy() == new RetryStrategy() @@ -112,21 +109,18 @@ class AwsBatchTaskHandlerTest extends Specification { when: req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - region >> 'eu-west-1' - } - and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.maxSpotAttempts() >> 0 + _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'], region: 'eu-west-1')) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - and: + req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] req.getRetryStrategy() == null // <-- retry is managed by NF, hence this must be null @@ -145,28 +139,38 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - debug >> true - storageEncryption >> 'AES256' - schedulingPriority >> 9999 - shareIdentifier >> 'priority/high' - maxSpotAttempts >> 5 - } - and: 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.maxSpotAttempts() >> 5 + _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'],client: [storageEncryption: 'AES256'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [] - and: + req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' req.getContainerOverrides().getCommand() == ['bash', '-c', 'something'] - req.getShareIdentifier() == 'priority/high' - req.getSchedulingPriorityOverride() == 9999 + + when: + def req2 = handler.newSubmitRequest(task) + then: + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.maxSpotAttempts() >> 5 + _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws',schedulingPriority: 9999,shareIdentifier: 'priority/high'], client:[storageEncryption: 'AES256', debug: true])) } + 1 * handler.getJobQueue(task) >> 'queue1' + 1 * handler.getJobDefinition(task) >> 'job-def:1' + 1 * handler.getEnvironmentVars() >> [] + + req2.getJobName() == 'batchtask' + req2.getJobQueue() == 'queue1' + req2.getJobDefinition() == 'job-def:1' + req2.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req2.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' + req2.getContainerOverrides().getCommand() ==['bash', '-c', 'something'] + req2.getShareIdentifier() == 'priority/high' + req2.getSchedulingPriorityOverride() == 9999 } @@ -182,19 +186,19 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - region >> 'eu-west-1' - } + handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'],region: 'eu-west-1')) } and: _ * handler.fusionEnabled() >> false + 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: - req.getContainerOverrides().getResourceRequirements().size() == 3 - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '2048' - req.getContainerOverrides().getResourceRequirements().find { it.type=='GPU' }.getValue() == '2' + def res = req.getContainerOverrides().getResourceRequirements() + res.size()==3 + and: + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '2048' + req.getContainerOverrides().getResourceRequirements().find { it.type=='GPU'}.getValue() == '2' } @@ -209,9 +213,10 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig() - handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } + handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: _ * handler.fusionEnabled() >> false + 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: @@ -225,9 +230,10 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '5 sec') - handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } + handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: _ * handler.fusionEnabled() >> false + 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue2' 1 * handler.getJobDefinition(task) >> 'job-def:2' and: @@ -237,20 +243,23 @@ class AwsBatchTaskHandlerTest extends Specification { // minimal allowed timeout is 60 seconds req.getTimeout().getAttemptDurationSeconds() == 60 + when: req = handler.newSubmitRequest(task) then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '1 hour') - handler.getAwsOptions() >> Mock(AwsOptions) { cliPath >> '/bin/aws' } + handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: _ * handler.fusionEnabled() >> false + 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue3' 1 * handler.getJobDefinition(task) >> 'job-def:3' and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue3' req.getJobDefinition() == 'job-def:3' + // minimal allowed timeout is 60 seconds req.getTimeout().getAttemptDurationSeconds() == 3600 } @@ -270,22 +279,18 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - retryMode >> 'adaptive' - maxTransferAttempts >> 10 - maxSpotAttempts >> 3 - } + handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws', retryMode: 'adaptive', maxTransferAttempts: 10])) } and: _ * handler.fusionEnabled() >> false 1 * handler.getSubmitCommand() >> ['bash','-c','foo'] + 1 * handler.maxSpotAttempts() >> 3 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' and: req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - // no error `retry` error strategy is defined by NF, use `maxRetries` to set Batch attempts + // no error `retry` error strategy is defined by NF, use `maxRetries` to se Batch attempts req.getRetryStrategy() == new RetryStrategy() .withAttempts(3) .withEvaluateOnExit( new EvaluateOnExit().withAction('RETRY').withOnStatusReason('Host EC2*'), new EvaluateOnExit().withOnReason('*').withAction('EXIT') ) @@ -410,16 +415,16 @@ class AwsBatchTaskHandlerTest extends Specification { handler.resolveJobDefinition(IMAGE) then: 1 * handler.makeJobDefRequest(IMAGE) >> req - 1 * handler.findJobDefinition(JOB_NAME, JOB_ID) >> null - 1 * handler.registerJobDefinition(req) >> null + 1 * handler.findJobDef(JOB_NAME, JOB_ID) >> null + 1 * handler.createJobDef(req) >> null when: handler.resolveJobDefinition(IMAGE) then: // second time are not invoked for the same image 1 * handler.makeJobDefRequest(IMAGE) >> req - 0 * handler.findJobDefinition(JOB_NAME, JOB_ID) >> null - 0 * handler.registerJobDefinition(req) >> null + 0 * handler.findJobDef(JOB_NAME, JOB_ID) >> null + 0 * handler.createJobDef(req) >> null } @@ -429,42 +434,41 @@ class AwsBatchTaskHandlerTest extends Specification { def JOB_NAME = 'foo-bar-1-0' def JOB_ID = '123' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> client - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = client def req = new DescribeJobDefinitionsRequest().withJobDefinitionName(JOB_NAME) def res = Mock(DescribeJobDefinitionsResult) def job = Mock(JobDefinition) when: - def result = handler.findJobDefinition(JOB_NAME, JOB_ID) + def result = handler.findJobDef(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [] result == null when: - result = handler.findJobDefinition(JOB_NAME, JOB_ID) + result = handler.findJobDef(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] 1 * job.getStatus() >> 'ACTIVE' - _ * job.getParameters() >> ['nf-token': JOB_ID] + 1 * job.getParameters() >> ['nf-token': JOB_ID] 1 * job.getRevision() >> 3 result == "$JOB_NAME:3" when: - result = handler.findJobDefinition(JOB_NAME, JOB_ID) + result = handler.findJobDef(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] 1 * job.getStatus() >> 'ACTIVE' - _ * job.getParameters() >> [:] + 1 * job.getParameters() >> [:] result == null when: - result = handler.findJobDefinition(JOB_NAME, JOB_ID) + result = handler.findJobDef(JOB_NAME, JOB_ID) then: 1 * client.describeJobDefinitions(req) >> res 1 * res.getJobDefinitions() >> [job] @@ -474,20 +478,19 @@ class AwsBatchTaskHandlerTest extends Specification { } - def 'should register job definition' () { + def 'should create job definition existence' () { given: def JOB_NAME = 'foo-bar-1-0' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> client - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = client def req = new RegisterJobDefinitionRequest() def res = Mock(RegisterJobDefinitionResult) when: - def result = handler.registerJobDefinition(req) + def result = handler.createJobDef(req) then: 1 * client.registerJobDefinition(req) >> res 1 * res.getJobDefinitionName() >> JOB_NAME @@ -588,7 +591,7 @@ class AwsBatchTaskHandlerTest extends Specification { given: def IMAGE = 'foo/bar:1.0' def JOB_NAME = 'nf-foo-bar-1-0' - def handler = Spy(AwsBatchTaskHandler) { + AwsBatchTaskHandler handler = Spy(AwsBatchTaskHandler) { getTask() >> Mock(TaskRun) { getConfig() >> Mock(TaskConfig) } fusionEnabled() >> true } @@ -692,9 +695,8 @@ class AwsBatchTaskHandlerTest extends Specification { given: def JOB_ID = 'job-2' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> client - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = client def JOB1 = new JobDetail().withJobId('job-1') def JOB2 = new JobDetail().withJobId('job-2') @@ -717,9 +719,8 @@ class AwsBatchTaskHandlerTest extends Specification { def collector = Mock(BatchContext) def JOB_ID = 'job-1' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> client - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = client handler.@jobId = JOB_ID handler.batch(collector) @@ -746,9 +747,8 @@ class AwsBatchTaskHandlerTest extends Specification { def collector = Mock(BatchContext) def JOB_ID = 'job-1' def client = Mock(AWSBatch) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> client - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = client handler.@jobId = JOB_ID handler.batch(collector) @@ -771,10 +771,9 @@ class AwsBatchTaskHandlerTest extends Specification { def task = Mock(TaskRun) def client = Mock(AWSBatch) def proxy = Mock(AwsBatchProxy) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> proxy - getTask() >> task - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = proxy + handler.task = task def req = Mock(SubmitJobRequest) def resp = Mock(SubmitJobResult) @@ -822,9 +821,8 @@ class AwsBatchTaskHandlerTest extends Specification { task.getProcessor() >> processor task.getConfig() >> GroovyMock(TaskConfig) def proxy = Mock(AwsBatchProxy) - def handler = Spy(AwsBatchTaskHandler) { - getClient() >> proxy - } + def handler = Spy(AwsBatchTaskHandler) + handler.@client = proxy handler.task = task handler.@jobId = 'xyz-123' @@ -887,21 +885,18 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - handler.getAwsOptions() >> Mock(AwsOptions) { - cliPath >> '/bin/aws' - maxSpotAttempts >> 5 - } - and: 1 * handler.getSubmitCommand() >> ['sh','-c','hello'] + 1 * handler.maxSpotAttempts() >> 5 + 1 * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' 1 * handler.getEnvironmentVars() >> [VAR_FOO, VAR_BAR] - and: + req.getJobName() == 'batchtask' req.getJobQueue() == 'queue1' req.getJobDefinition() == 'job-def:1' - req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU' }.getValue() == '4' - req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY' }.getValue() == '8192' + req.getContainerOverrides().getResourceRequirements().find { it.type=='VCPU'}.getValue() == '4' + req.getContainerOverrides().getResourceRequirements().find { it.type=='MEMORY'}.getValue() == '8192' req.getContainerOverrides().getEnvironment() == [VAR_FOO, VAR_BAR] req.getContainerOverrides().getCommand() == ['sh', '-c','hello'] req.getRetryStrategy() == new RetryStrategy() @@ -910,8 +905,7 @@ class AwsBatchTaskHandlerTest extends Specification { req.getTags() == [a:'b'] req.getPropagateTags() == true } - - def 'should create fusion submit command' () { + def 'get fusion submit command' () { given: def handler = Spy(AwsBatchTaskHandler) { fusionEnabled() >> true From ffc6d312781c17f86db85f97011495ab8b4115fd Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 2 May 2023 20:26:06 -0500 Subject: [PATCH 25/97] Fix missing method error Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/processor/TaskRun.groovy | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index a45382a144..2b2d8f0331 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -448,7 +448,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('outputFileNames', (it)-> getOutputFilesNames0()) } - private List getOutputFilesNames0() { + protected List getOutputFilesNames0() { def result = [] for( FileOutParam param : getOutputsByType(FileOutParam).keySet() ) { @@ -589,7 +589,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('condaEnv', (it)-> getCondaEnv0()) } - private Path getCondaEnv0() { + protected Path getCondaEnv0() { if( !config.conda || !processor.session.getCondaConfig().isEnabled() ) return null @@ -601,7 +601,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('spackEnv', (it)-> getSpackEnv0()) } - private Path getSpackEnv0() { + protected Path getSpackEnv0() { if( !config.spack || !processor.session.getSpackConfig().isEnabled() ) return null @@ -615,7 +615,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('containerInfo', (it)-> containerInfo0()) } - private ContainerInfo containerInfo0() { + protected ContainerInfo containerInfo0() { // fetch the container image from the config def configImage = config.getContainer() // the boolean `false` literal can be provided From 69454500a0934ffdbf100fbda97fb7cb8bb7b49f Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 3 May 2023 01:49:47 -0500 Subject: [PATCH 26/97] Minor changes Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/LsfExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/PbsExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/PbsProExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/SlurmExecutor.groovy | 2 +- .../main/groovy/nextflow/executor/TaskArraySubmitter.groovy | 2 ++ .../src/main/groovy/nextflow/processor/TaskArray.groovy | 4 ++++ .../main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy | 2 +- 8 files changed, 12 insertions(+), 6 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index d94a7af24d..64d296e6e8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -71,7 +71,7 @@ class LsfExecutor extends AbstractGridExecutor { protected List getDirectives(TaskRun task, List result) { if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).children.size() + final arraySize = ((TaskArray)task).getArraySize() result << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index c8bfcc6700..cb47331296 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -43,7 +43,7 @@ class PbsExecutor extends AbstractGridExecutor { assert result !=null if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).children.size() + final arraySize = ((TaskArray)task).getArraySize() result << '-J' << "0-${arraySize - 1}" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 6a25e3b684..6d3e4ba286 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -46,7 +46,7 @@ class PbsProExecutor extends PbsExecutor { assert result !=null if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).children.size() + final arraySize = ((TaskArray)task).getArraySize() result << '-J' << "0-${arraySize - 1}" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index e1f3b6bb3a..1ce68dad7d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -37,7 +37,7 @@ class SgeExecutor extends AbstractGridExecutor { protected List getDirectives(TaskRun task, List result) { if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).children.size() + final arraySize = ((TaskArray)task).getArraySize() result << '-t' << "0-${arraySize - 1}" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 53b208417a..bbbc3250c4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -52,7 +52,7 @@ class SlurmExecutor extends AbstractGridExecutor { protected List getDirectives(TaskRun task, List result) { if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).children.size() + final arraySize = ((TaskArray)task).getArraySize() result << '--array' << "0-${arraySize - 1}" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index 4cca97141d..4cc0f31d4a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -97,6 +97,8 @@ class TaskArraySubmitter { // submit array job arrayHandler.prepareLauncher() arrayHandler.submit() + + log.trace "Submitted array job ${arrayTask.name} > workDir: ${arrayTask.workDir}" } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy index 9c66f17d62..8a3b50ef16 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy @@ -31,4 +31,8 @@ class TaskArray extends TaskRun { List children + int getArraySize() { + children.size() + } + } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 04cd5c3a15..48d527fab4 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -750,7 +750,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler 10_000 ) throw new IllegalArgumentException("Array jobs on AWS Batch may not have more than 10,000 tasks") From f95c7d0b14f5e6736872f236d738211a0d6cbd86 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 3 May 2023 01:53:34 -0500 Subject: [PATCH 27/97] Fix bugs in array submit script Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/executor/TaskArraySubmitter.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index 4cc0f31d4a..f5bd6d3690 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -116,7 +116,7 @@ class TaskArraySubmitter { else { workDirs = tasks.collect( t -> Escape.path(t.workDir.toUriString()) ) cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) - cmd = cmd.replace(tasks.first().workDir.toUriString(), '\\${task_dir}') + cmd = cmd.replaceAll(tasks.first().workDir.toUriString(), '\\${task_dir}') } // create wrapper script @@ -124,7 +124,7 @@ class TaskArraySubmitter { """ declare -a array=( ${workDirs.join(' ')} ) - task_dir=\${array[\$${arrayIndexName}]} + export task_dir=\${array[\$${arrayIndexName}]} ${cmd} """.stripIndent().trim() } From 44f6825c6afa7db5c12712e19470a790b71935fb Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 3 May 2023 02:16:51 -0500 Subject: [PATCH 28/97] Fix Fusion support Signed-off-by: Ben Sherman --- .../groovy/nextflow/executor/TaskArrayAware.groovy | 2 ++ .../nextflow/executor/TaskArraySubmitter.groovy | 11 +++++++---- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index f7af14cd69..1856fbb63c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -36,6 +36,8 @@ interface TaskArrayAware { TaskHandler createTaskHandler(TaskRun task) + boolean isFusionEnabled() + /** * Get the environment variable name that provides the array index of a task. */ diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy index f5bd6d3690..e44e003945 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.file.FileHelper +import nextflow.fusion.FusionHelper import nextflow.processor.TaskArray import nextflow.processor.TaskContext import nextflow.processor.TaskHandler @@ -110,20 +111,22 @@ class TaskArraySubmitter { def cmd if( executor.workDir.fileSystem == FileSystems.default ) { - workDirs = tasks.collect( t -> Escape.path(t.workDir) ) + workDirs = tasks.collect( t -> t.workDir.toString() ) cmd = "cd \${task_dir} ; bash ${TaskRun.CMD_RUN} &> ${TaskRun.CMD_LOG}" } else { - workDirs = tasks.collect( t -> Escape.path(t.workDir.toUriString()) ) + workDirs = executor.isFusionEnabled() + ? tasks.collect( t -> FusionHelper.toContainerMount(t.workDir).toString() ) + : tasks.collect( t -> t.workDir.toUriString() ) cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) - cmd = cmd.replaceAll(tasks.first().workDir.toUriString(), '\\${task_dir}') + cmd = cmd.replaceAll(workDirs.first(), '\\${task_dir}') } // create wrapper script final arrayIndexName = executor.getArrayIndexName() """ - declare -a array=( ${workDirs.join(' ')} ) + declare -a array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) export task_dir=\${array[\$${arrayIndexName}]} ${cmd} """.stripIndent().trim() From 12612a36eaaf2c0c98b9e570e8aa160763294cf8 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 3 May 2023 11:03:21 -0500 Subject: [PATCH 29/97] Replace array submitter with array handling logic in the task polling monitor Signed-off-by: Ben Sherman --- .../nextflow/executor/GridTaskHandler.groovy | 5 - .../executor/TaskArraySubmitter.groovy | 135 ------------------ .../processor/TaskArrayCollector.groovy | 83 +++++++++-- .../nextflow/processor/TaskHandler.groovy | 6 - .../processor/TaskPollingMonitor.groovy | 32 +++-- .../cloud/aws/batch/AwsBatchExecutor.groovy | 1 - .../aws/batch/AwsBatchTaskHandler.groovy | 5 - 7 files changed, 95 insertions(+), 172 deletions(-) delete mode 100644 modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 9bffa079c0..527286cb9b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -255,11 +255,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { */ @Override void submit() { - if( arraySubmitter ) { - arraySubmitter.collect(this) - return - } - ProcessBuilder builder = null try { // -- start the execution and notify the event to the monitor diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy deleted file mode 100644 index e44e003945..0000000000 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArraySubmitter.groovy +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Copyright 2013-2023, Seqera Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package nextflow.executor - -import java.nio.file.FileSystems -import java.nio.file.Files -import java.util.concurrent.atomic.AtomicInteger - -import groovy.transform.CompileStatic -import groovy.util.logging.Slf4j -import nextflow.file.FileHelper -import nextflow.fusion.FusionHelper -import nextflow.processor.TaskArray -import nextflow.processor.TaskContext -import nextflow.processor.TaskHandler -import nextflow.processor.TaskRun -import nextflow.util.CacheHelper -import nextflow.util.Escape - -/** - * Submit tasks as an array job. - * - * @author Ben Sherman - */ -@Slf4j -@CompileStatic -class TaskArraySubmitter { - - private List array - - private TaskArrayAware executor - - private AtomicInteger collected = new AtomicInteger() - - TaskArraySubmitter(List array, TaskArrayAware executor) { - this.array = array - this.executor = executor - } - - /** - * Mark a task as ready to be submitted. - * - * When all tasks in the array are ready, the array job - * will be submitted. - * - * @param handler - */ - void collect(TaskHandler handler) { - if( collected.incrementAndGet() == array.size() ) - submit() - } - - /** - * Submit the array job. - */ - protected void submit() { - final tasks = array.collect( h -> h.task ) - final first = tasks.first() - - // create work directory - final hash = CacheHelper.hasher( tasks.collect( t -> t.getHash().asLong() ) ).hash() - final workDir = FileHelper.getWorkFolder(executor.getWorkDir(), hash) - - Files.createDirectories(workDir) - - // create wrapper script - final script = createWrapperScript(tasks) - - // create task handler - final arrayTask = new TaskArray( - id: first.id, - index: first.index, - processor: first.processor, - type: first.type, - config: first.processor.config.createTaskConfig(), - context: new TaskContext(first.processor), - hash: hash, - workDir: workDir, - script: script, - children: array - ) - final arrayHandler = executor.createTaskHandler(arrayTask) - - // submit array job - arrayHandler.prepareLauncher() - arrayHandler.submit() - - log.trace "Submitted array job ${arrayTask.name} > workDir: ${arrayTask.workDir}" - } - - /** - * Create the wrapper script for an array job. - */ - protected String createWrapperScript(List tasks) { - // get work directory and launch command for each task - def workDirs - def cmd - - if( executor.workDir.fileSystem == FileSystems.default ) { - workDirs = tasks.collect( t -> t.workDir.toString() ) - cmd = "cd \${task_dir} ; bash ${TaskRun.CMD_RUN} &> ${TaskRun.CMD_LOG}" - } - else { - workDirs = executor.isFusionEnabled() - ? tasks.collect( t -> FusionHelper.toContainerMount(t.workDir).toString() ) - : tasks.collect( t -> t.workDir.toUriString() ) - cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) - cmd = cmd.replaceAll(workDirs.first(), '\\${task_dir}') - } - - // create wrapper script - final arrayIndexName = executor.getArrayIndexName() - - """ - declare -a array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export task_dir=\${array[\$${arrayIndexName}]} - ${cmd} - """.stripIndent().trim() - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 85fd30e43a..4cad4758fe 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -16,6 +16,8 @@ package nextflow.processor +import java.nio.file.FileSystems +import java.nio.file.Files import java.util.concurrent.locks.Lock import java.util.concurrent.locks.ReentrantLock @@ -23,7 +25,10 @@ import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.executor.Executor import nextflow.executor.TaskArrayAware -import nextflow.executor.TaskArraySubmitter +import nextflow.file.FileHelper +import nextflow.fusion.FusionHelper +import nextflow.util.CacheHelper +import nextflow.util.Escape /** * Task monitor that batches tasks and submits them as array jobs @@ -37,8 +42,6 @@ class TaskArrayCollector { private TaskArrayAware executor - private TaskMonitor monitor - private int arraySize private Lock sync = new ReentrantLock() @@ -52,7 +55,6 @@ class TaskArrayCollector { throw new IllegalArgumentException("Executor '${executor.name}' does not support array jobs") this.executor = (TaskArrayAware)executor - this.monitor = executor.monitor this.arraySize = arraySize this.array = new ArrayList<>(arraySize) } @@ -108,15 +110,72 @@ class TaskArrayCollector { } protected void submit0(List array) { - // create submitter for array job - final arraySubmitter = new TaskArraySubmitter(array, executor) - - // submit each task to the underlying monitor - // each task will defer to the array job during submission - for( TaskHandler handler : array ) { - handler.arraySubmitter = arraySubmitter - monitor.schedule(handler) + executor.submit(createTaskArray(array)) + } + + /** + * Create the task run for an array job. + * + * @param array + */ + protected TaskRun createTaskArray(List array) { + final tasks = array.collect( h -> h.task ) + final first = tasks.first() + + // create work directory + final hash = CacheHelper.hasher( tasks.collect( t -> t.getHash().asLong() ) ).hash() + final workDir = FileHelper.getWorkFolder(executor.getWorkDir(), hash) + + Files.createDirectories(workDir) + + // create wrapper script + final script = createTaskArrayScript(tasks) + + // create task handler + return new TaskArray( + id: first.id, + index: first.index, + processor: first.processor, + type: first.type, + config: first.processor.config.createTaskConfig(), + context: new TaskContext(first.processor), + hash: hash, + workDir: workDir, + script: script, + children: array + ) + } + + /** + * Create the wrapper script for an array job. + * + * @param tasks + */ + protected String createTaskArrayScript(List tasks) { + // get work directory and launch command for each task + def workDirs + def cmd + + if( executor.workDir.fileSystem == FileSystems.default ) { + workDirs = tasks.collect( t -> t.workDir.toString() ) + cmd = "cd \${task_dir} ; bash ${TaskRun.CMD_RUN} &> ${TaskRun.CMD_LOG}" } + else { + workDirs = executor.isFusionEnabled() + ? tasks.collect( t -> FusionHelper.toContainerMount(t.workDir).toString() ) + : tasks.collect( t -> t.workDir.toUriString() ) + cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) + cmd = cmd.replaceAll(workDirs.first(), '\\${task_dir}') + } + + // create wrapper script + final arrayIndexName = executor.getArrayIndexName() + + """ + declare -a array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) + export task_dir=\${array[\$${arrayIndexName}]} + ${cmd} + """.stripIndent().trim() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index e7fae218f9..c766868559 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -21,7 +21,6 @@ import static nextflow.processor.TaskStatus.* import java.nio.file.NoSuchFileException import groovy.util.logging.Slf4j -import nextflow.executor.TaskArraySubmitter import nextflow.trace.TraceRecord /** * Actions to handle the underlying job running the user task. @@ -52,11 +51,6 @@ abstract class TaskHandler { */ TaskRun getTask() { task } - /** - * Submitter for array jobs - */ - TaskArraySubmitter arraySubmitter - /** * Task current status */ diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index f634f8c99a..8b8be6336d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -192,14 +192,30 @@ class TaskPollingMonitor implements TaskMonitor { * A {@link TaskHandler} instance representing the task to be submitted for execution */ protected void submit(TaskHandler handler) { - // submit the job execution -- throws a ProcessException when submit operation fail - handler.prepareLauncher() - handler.submit() - // note: add the 'handler' into the polling queue *after* the submit operation, - // this guarantees that in the queue are only jobs successfully submitted - runningQueue.add(handler) - // notify task submission - session.notifyTaskSubmit(handler) + if( handler.task instanceof TaskArray ) { + // prepare launcher for each child task + final children = ((TaskArray)handler.task).children + for( TaskHandler h0 : children ) + h0.prepareLauncher() + // submit task array + handler.prepareLauncher() + handler.submit() + // add each child task to the running queue + for( TaskHandler h0 : children ) { + runningQueue.add(h0) + session.notifyTaskSubmit(h0) + } + } + else { + // submit the job execution -- throws a ProcessException when submit operation fail + handler.prepareLauncher() + handler.submit() + // note: add the 'handler' into the polling queue *after* the submit operation, + // this guarantees that in the queue are only jobs successfully submitted + runningQueue.add(handler) + // notify task submission + session.notifyTaskSubmit(handler) + } } /** diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 9581232f02..d1b6eef811 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -33,7 +33,6 @@ import nextflow.cloud.aws.config.AwsConfig import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException import nextflow.executor.TaskArrayAware -import nextflow.executor.TaskArraySubmitter import nextflow.executor.Executor import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 48d527fab4..cb2e27fa7d 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -311,11 +311,6 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler Date: Wed, 3 May 2023 11:03:38 -0500 Subject: [PATCH 30/97] Fix task reporting in log observer Signed-off-by: Ben Sherman --- .../processor/ParallelPollingMonitor.groovy | 2 +- .../processor/TaskPollingMonitor.groovy | 20 +++++++++++++++++-- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy index 9c2f687ffb..239e8d4b57 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy @@ -86,7 +86,7 @@ class ParallelPollingMonitor extends TaskPollingMonitor { if( !session.success ) return // ignore error when the session has been interrupted handleException(handler, e) - session.notifyTaskComplete(handler) + notifyTaskComplete0(handler) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 8b8be6336d..451cbadc58 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -243,7 +243,7 @@ class TaskPollingMonitor implements TaskMonitor { try{ pendingQueue << handler taskAvail.signal() // signal that a new task is available for execution - session.notifyTaskPending(handler) + notifyTaskPending0(handler) log.trace "Scheduled task > $handler" } finally { @@ -580,7 +580,7 @@ class TaskPollingMonitor implements TaskMonitor { } catch ( Throwable e ) { handleException(handler, e) - session.notifyTaskComplete(handler) + notifyTaskComplete0(handler) } // remove processed handler either on successful submit or failed one (managed by catch section) // when `canSubmit` return false the handler should be retained to be tried in a following iteration @@ -694,5 +694,21 @@ class TaskPollingMonitor implements TaskMonitor { return pendingQueue } + protected void notifyTaskPending0(TaskHandler handler) { + if( handler.task instanceof TaskArray ) + for( TaskHandler h0 : ((TaskArray)handler.task).children ) + session.notifyTaskPending(h0) + else + session.notifyTaskPending(handler) + } + + protected void notifyTaskComplete0(TaskHandler handler) { + if( handler.task instanceof TaskArray ) + for( TaskHandler h0 : ((TaskArray)handler.task).children ) + session.notifyTaskComplete(h0) + else + session.notifyTaskComplete(handler) + } + } From 7ec397fd262ce0186603cae6f7921966c9e44d55 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 24 May 2023 17:53:40 +0200 Subject: [PATCH 31/97] Add support for AWS SSE env variables Signed-off-by: Paolo Di Tommaso --- .../nextflow/cloud/aws/config/AwsS3Config.groovy | 4 ++-- .../cloud/aws/config/AwsS3ConfigTest.groovy | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy index e479925314..5ff28aac75 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy @@ -50,8 +50,8 @@ class AwsS3Config { this.debug = opts.debug as Boolean this.endpoint = opts.endpoint ?: SysEnv.get('AWS_S3_ENDPOINT') this.storageClass = parseStorageClass((opts.storageClass ?: opts.uploadStorageClass) as String) // 'uploadStorageClass' is kept for legacy purposes - this.storageEncryption = parseStorageEncryption(opts.storageEncryption as String) - this.storageKmsKeyId = opts.storageKmsKeyId + this.storageEncryption = parseStorageEncryption(opts.storageEncryption as String) ?: SysEnv.get('NXF_AWS_SSE_MODE') + this.storageKmsKeyId = opts.storageKmsKeyId ?: SysEnv.get('NXF_AWS_SSE_KMS_KEY_ID') this.pathStyleAccess = opts.s3PathStyleAccess as Boolean this.s3Acl = parseS3Acl(opts.s3Acl as String) } diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy index 116a64456b..0f5cd1f643 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy @@ -121,4 +121,18 @@ class AwsS3ConfigTest extends Specification { SysEnv.pop() } + + def 'should set storage encryption via env variable' () { + given: + SysEnv.push([NXF_AWS_SSE_MODE: 'aws:kms', NXF_AWS_SSE_KMS_KEY_ID: 'xyz1']) + + when: + def client = new AwsS3Config([:]) + then: + client.storageKmsKeyId == 'xyz1' + client.storageEncryption == 'aws:kms' + + cleanup: + SysEnv.pop() + } } From 324681bcff958cffc9bb88ebbc60532284b8e9d4 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 26 May 2023 11:34:22 -0500 Subject: [PATCH 32/97] Add support for google batch Signed-off-by: Ben Sherman --- docs/process.md | 1 + .../executor/AbstractGridExecutor.groovy | 5 + .../nextflow/executor/GridTaskHandler.groovy | 36 ++++-- .../nextflow/executor/TaskArrayAware.groovy | 10 -- .../processor/TaskArrayCollector.groovy | 41 +++---- .../nextflow/processor/TaskHandler.groovy | 5 + .../processor/TaskPollingMonitor.groovy | 6 +- .../executor/GridTaskHandlerTest.groovy | 2 +- .../cloud/aws/batch/AwsBatchExecutor.groovy | 5 +- .../aws/batch/AwsBatchTaskHandler.groovy | 9 +- .../google/batch/GoogleBatchExecutor.groovy | 7 +- .../batch/GoogleBatchFusionAdapter.groovy | 6 + .../batch/GoogleBatchLauncherSpec.groovy | 7 ++ .../batch/GoogleBatchScriptLauncher.groovy | 4 +- .../batch/GoogleBatchTaskHandler.groovy | 110 ++++++++++++------ .../google/batch/client/BatchClient.groovy | 33 ++++-- .../batch/GoogleBatchLauncherSpecMock.groovy | 5 + .../batch/GoogleBatchTaskHandlerTest.groovy | 9 +- .../batch/logging/BatchLoggingTest.groovy | 4 +- 19 files changed, 198 insertions(+), 107 deletions(-) diff --git a/docs/process.md b/docs/process.md index cd17d263a0..3b3a6f87c4 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1280,6 +1280,7 @@ process cpu_task { Nextflow currently supports array jobs for the following executors: - {ref}`awsbatch-executor` +- {ref}`google-batch-executor` - {ref}`lsf-executor` - {ref}`pbs-executor` - {ref}`pbspro-executor` diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 0365f69f6e..caea8e91c3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -408,5 +408,10 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { // Instead, it is the command wrapper script that is launched run within a container process. return isFusionEnabled() } + + String getArrayTaskId(String jobId, int index) { + throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 527286cb9b..8e4df76c5c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -99,6 +99,22 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { this.sanityCheckInterval = duration } + @Override + String getWorkDir() { + fusionEnabled() + ? FusionHelper.toContainerMount(task.workDir).toString() + : task.workDir.toString() + } + + @Override + List getSubmitCommand() { + final cmd = fusionEnabled() + ? fusionSubmitCommand() + : "cd ${task.workDir} ; bash ${TaskRun.CMD_RUN} | tee ${TaskRun.CMD_LOG}" + + List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) + } + protected ProcessBuilder createProcessBuilder() { // -- log the qsub command @@ -216,15 +232,14 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { } protected String fusionStdinWrapper() { - final submit = fusionSubmitCli() - final launcher = fusionLauncher() - final config = task.getContainerConfig() - final cmd = FusionHelper.runWithContainer(launcher, config, task.getContainer(), submit) - // create an inline script to launch the job execution - return '#!/bin/bash\n' + submitDirective(task) + cmd + '\n' + final builder = new StringBuilder() + << '#!/bin/bash\n' + << fusionSubmitDirective(task) + << fusionSubmitCommand(task) << '\n' + return builder.toString() } - protected String submitDirective(TaskRun task) { + protected String fusionSubmitDirective(TaskRun task) { final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() // replaces the log file with a null file because the cluster submit tool // cannot write to a file hosted in a remote object storage @@ -234,6 +249,13 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } + protected String fusionSubmitCommand(TaskRun task) { + final launcher = fusionLauncher() + final config = task.getContainerConfig() + final submit = fusionSubmitCli() + return FusionHelper.runWithContainer(launcher, config, task.getContainer(), submit) + } + protected String launchCmd0(ProcessBuilder builder, String pipeScript) { def result = CmdLineHelper.toLine(builder.command()) if( pipeScript ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index 1856fbb63c..db9a695fd9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -45,14 +45,4 @@ interface TaskArrayAware { throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") } - /** - * Get the job ID of an array task based on its index in the array. - * - * @param jobId - * @param index - */ - default String getArrayTaskId(String jobId, int index) { - throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 4cad4758fe..592dbb6302 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -16,7 +16,6 @@ package nextflow.processor -import java.nio.file.FileSystems import java.nio.file.Files import java.util.concurrent.locks.Lock import java.util.concurrent.locks.ReentrantLock @@ -26,7 +25,6 @@ import groovy.util.logging.Slf4j import nextflow.executor.Executor import nextflow.executor.TaskArrayAware import nextflow.file.FileHelper -import nextflow.fusion.FusionHelper import nextflow.util.CacheHelper import nextflow.util.Escape @@ -110,6 +108,11 @@ class TaskArrayCollector { } protected void submit0(List array) { + // prepare child job launcher scripts + for( TaskHandler handler : array ) + handler.prepareLauncher() + + // submit array job executor.submit(createTaskArray(array)) } @@ -129,7 +132,7 @@ class TaskArrayCollector { Files.createDirectories(workDir) // create wrapper script - final script = createTaskArrayScript(tasks) + final script = createTaskArrayScript(array) // create task handler return new TaskArray( @@ -149,33 +152,21 @@ class TaskArrayCollector { /** * Create the wrapper script for an array job. * - * @param tasks + * @param array */ - protected String createTaskArrayScript(List tasks) { + protected String createTaskArrayScript(List array) { // get work directory and launch command for each task - def workDirs - def cmd - - if( executor.workDir.fileSystem == FileSystems.default ) { - workDirs = tasks.collect( t -> t.workDir.toString() ) - cmd = "cd \${task_dir} ; bash ${TaskRun.CMD_RUN} &> ${TaskRun.CMD_LOG}" - } - else { - workDirs = executor.isFusionEnabled() - ? tasks.collect( t -> FusionHelper.toContainerMount(t.workDir).toString() ) - : tasks.collect( t -> t.workDir.toUriString() ) - cmd = Escape.cli(array.first().getSubmitCommand().toArray() as String[]) - cmd = cmd.replaceAll(workDirs.first(), '\\${task_dir}') - } + final workDirs = array.collect( handler -> handler.getWorkDir() ) + final args = array.first().getSubmitCommand().toArray() as String[] + final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${task_dir}') // create wrapper script final arrayIndexName = executor.getArrayIndexName() - - """ - declare -a array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export task_dir=\${array[\$${arrayIndexName}]} - ${cmd} - """.stripIndent().trim() + final builder = new StringBuilder() + << "array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} )\n" + << "export task_dir=\${array[${arrayIndexName}]}\n" + << cmd << '\n' + return builder.toString() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index c766868559..3cfa0fd03c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -93,6 +93,11 @@ abstract class TaskHandler { */ void prepareLauncher() {} + /** + * Get the work directory as it will be seen from the launcher script. + */ + String getWorkDir() { null } + /** * Get the command to execute the launcher script. */ diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 451cbadc58..edf5bd6535 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -193,15 +193,11 @@ class TaskPollingMonitor implements TaskMonitor { */ protected void submit(TaskHandler handler) { if( handler.task instanceof TaskArray ) { - // prepare launcher for each child task - final children = ((TaskArray)handler.task).children - for( TaskHandler h0 : children ) - h0.prepareLauncher() // submit task array handler.prepareLauncher() handler.submit() // add each child task to the running queue - for( TaskHandler h0 : children ) { + for( TaskHandler h0 : ((TaskArray)handler.task).children ) { runningQueue.add(h0) session.notifyTaskSubmit(h0) } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 01a028e05a..2d2a7ec13c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -86,7 +86,7 @@ class GridTaskHandlerTest extends Specification { def handler = Spy(new GridTaskHandler(task, exec)) when: - def result = handler.submitDirective(task) + def result = handler.fusionSubmitDirective(task) then: 1 * exec.getHeaders(task) >> "#FOO this\n#BAR that\n#OUT file=${WORK_DIR}/.command.log\n" diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index d1b6eef811..d5b4080d80 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -32,8 +32,8 @@ import nextflow.cloud.aws.AwsClientFactory import nextflow.cloud.aws.config.AwsConfig import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException -import nextflow.executor.TaskArrayAware import nextflow.executor.Executor +import nextflow.executor.TaskArrayAware import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx import nextflow.processor.ParallelPollingMonitor @@ -310,9 +310,6 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar @Override String getArrayIndexName() { 'AWS_BATCH_JOB_ARRAY_INDEX' } - @Override - String getArrayTaskId(String jobId, int index) { "${jobId}:${index}" } - } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index cb2e27fa7d..4295f5abd4 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -330,7 +330,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler - final arrayTaskId = executor.getArrayTaskId(jobId, i) + final arrayTaskId = "${jobId}:${i}" ((AwsBatchTaskHandler)handler).onSubmit(arrayTaskId, queueName) } } @@ -657,6 +657,13 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler getContainerMounts() { return List.of() diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy index fc34f7f1bf..aa9ee78a0f 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy @@ -16,6 +16,8 @@ package nextflow.cloud.google.batch +import java.nio.file.Path + import com.google.cloud.batch.v1.Volume /** @@ -25,6 +27,11 @@ import com.google.cloud.batch.v1.Volume */ interface GoogleBatchLauncherSpec { + /** + * @return The equvalent container mount for a remote path. + */ + Path toContainerMount(Path path) + /** * @return * A list of string representing the container mounts. Each mount uses the docker diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy index ae4dc6b5f6..01d48126f3 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy @@ -59,6 +59,8 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc // map bean work and target dirs to container mount // this is needed to create the command launcher using container local file paths + toContainerMount(bean.workDir.parent.parent) + bean.workDir = toContainerMount(bean.workDir) bean.targetDir = toContainerMount(bean.targetDir) @@ -96,7 +98,7 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc return result } - protected Path toContainerMount(Path path, boolean parent=false) { + Path toContainerMount(Path path, boolean parent=false) { if( path instanceof CloudStoragePath ) { buckets.add(path.bucket()) pathTrie.add( (parent ? "/${path.bucket()}${path.parent}" : "/${path.bucket()}${path}").toString() ) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 5b8f3b698d..8a55ed1699 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -17,10 +17,6 @@ package nextflow.cloud.google.batch -import nextflow.cloud.types.CloudMachineInfo -import nextflow.cloud.types.PriceModel -import nextflow.processor.TaskConfig - import java.nio.file.Path import com.google.cloud.batch.v1.AllocationPolicy @@ -37,10 +33,15 @@ import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.util.logging.Slf4j import nextflow.cloud.google.batch.client.BatchClient +import nextflow.cloud.types.CloudMachineInfo +import nextflow.cloud.types.PriceModel import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder import nextflow.fusion.FusionAwareTask +import nextflow.fusion.FusionHelper import nextflow.fusion.FusionScriptLauncher +import nextflow.processor.TaskArray +import nextflow.processor.TaskConfig import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus @@ -66,20 +67,27 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { private BatchClient client + private BashWrapperBuilder launcher + /** * Job Id assigned by Nextflow */ private String jobId + /** + * Task id assigned by Google Batch service + */ + private String taskId + /** * Job unique id assigned by Google Batch service */ private String uid /** - * Job state assigned by Google Batch service + * Task state assigned by Google Batch service */ - private String jobState + private String taskState private volatile CloudMachineInfo machineInfo @@ -120,24 +128,49 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { } @Override - void submit() { - /* - * create the task runner script - */ - final launcher = createTaskWrapper() + void prepareLauncher() { + launcher = createTaskWrapper() launcher.build() + } + + @Override + String getWorkDir() { + spec0(launcher).toContainerMount(task.workDir).toString() + } + @Override + List getSubmitCommand() { + spec0(launcher).launchCommand() + } + + @Override + void submit() { /* * create submit request */ final req = newSubmitRequest(task, spec0(launcher)) log.trace "[GOOGLE BATCH] new job request > $req" final resp = client.submitJob(jobId, req) - this.uid = resp.getUid() - this.status = TaskStatus.SUBMITTED + final uid = resp.getUid() + this.onSubmit(jobId, '0', uid) log.debug "[GOOGLE BATCH] submitted > job=$jobId; uid=$uid; work-dir=${task.getWorkDirStr()}" } + void onSubmit(String jobId, String taskId, String uid) { + if( task instanceof TaskArray ) { + ((TaskArray)task).children.eachWithIndex { handler, i -> + final arrayTaskId = i.toString() + ((GoogleBatchTaskHandler)handler).onSubmit(jobId, arrayTaskId, uid) + } + } + else { + this.jobId = jobId + this.taskId = taskId + this.uid = uid + this.status = TaskStatus.SUBMITTED + } + } + protected Job newSubmitRequest(TaskRun task, GoogleBatchLauncherSpec launcher) { // resource requirements final taskSpec = TaskSpec.newBuilder() @@ -288,12 +321,18 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { allocationPolicy.putAllLabels(task.config.getResourceLabels()) + // task group + final taskGroup = TaskGroup.newBuilder() + .setTaskSpec(taskSpec) + + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).getArraySize() + taskGroup.setTaskCount(arraySize) + } + // create the job return Job.newBuilder() - .addTaskGroups( - TaskGroup.newBuilder() - .setTaskSpec(taskSpec) - ) + .addTaskGroups(taskGroup) .setAllocationPolicy(allocationPolicy) .setLogsPolicy( LogsPolicy.newBuilder() @@ -303,39 +342,43 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { } /** - * @return Retrieve the submitted job state + * @return Retrieve the submitted task state */ - protected String getJobState() { + protected String getTaskState() { + final tasks = client.listTasks(jobId) + if( !tasks.iterator().hasNext() ) + return 'PENDING' + final now = System.currentTimeMillis() final delta = now - timestamp; - if( !jobState || delta >= 1_000) { - final status = client.getJobStatus(jobId) + 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 state=$newState" - jobState = newState + log.trace "[GOOGLE BATCH] Get job=$jobId task=$taskId state=$newState" + taskState = newState timestamp = now } - if( newState == 'SCHEDULED' ) { + 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()}" } } - return jobState + return taskState } - private List RUNNING_AND_TERMINATED = ['RUNNING', 'SUCCEEDED', 'FAILED', 'DELETION_IN_PROGRESS'] + static private final List RUNNING_OR_COMPLETED = ['RUNNING', 'SUCCEEDED', 'FAILED'] - private List TERMINATED = ['SUCCEEDED', 'FAILED', 'DELETION_IN_PROGRESS'] + static private final List COMPLETED = ['SUCCEEDED', 'FAILED'] @Override boolean checkIfRunning() { if(isSubmitted()) { // include `terminated` state to allow the handler status to progress - if (getJobState() in RUNNING_AND_TERMINATED) { + if( getTaskState() in RUNNING_OR_COMPLETED ) { status = TaskStatus.RUNNING return true } @@ -345,9 +388,9 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { @Override boolean checkIfCompleted() { - final state = getJobState() - if( state in TERMINATED ) { - log.debug "[GOOGLE BATCH] Terminated job=$jobId; state=$state" + final state = getTaskState() + if( state in COMPLETED ) { + log.debug "[GOOGLE BATCH] Terminated job=$jobId; task=$taskId; state=$state" // finalize the task task.exitStatus = readExitFile() if( state == 'FAILED' ) { @@ -393,9 +436,8 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { @Override TraceRecord getTraceRecord() { def result = super.getTraceRecord() - if( jobId && uid ) { - result.put('native_id', "$jobId/$uid") - } + if( jobId && uid ) + result.put('native_id', "$jobId/$taskId/$uid") result.machineInfo = getMachineInfo() return result } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy index d2db757ba6..c4db64292b 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy @@ -23,9 +23,11 @@ import com.google.cloud.batch.v1.BatchServiceClient import com.google.cloud.batch.v1.BatchServiceSettings import com.google.cloud.batch.v1.Job import com.google.cloud.batch.v1.JobName -import com.google.cloud.batch.v1.JobStatus import com.google.cloud.batch.v1.LocationName +import com.google.cloud.batch.v1.Task import com.google.cloud.batch.v1.TaskGroupName +import com.google.cloud.batch.v1.TaskName +import com.google.cloud.batch.v1.TaskStatus import groovy.transform.CompileStatic import groovy.util.logging.Slf4j /** @@ -85,31 +87,38 @@ class BatchClient { return batchServiceClient.createJob(parent, job, jobId) } - Job describeJob(String jobId) { - final name = JobName.of(projectId, location, jobId) + Iterable listTasks(String jobId) { + final parent = TaskGroupName.of(projectId, location, jobId, 'group0') - return batchServiceClient.getJob(name) + return batchServiceClient.listTasks(parent).iterateAll() } - Iterable listTasks(String jobId) { - final parent = TaskGroupName.of(projectId, location, jobId, 'group0') + Task describeTask(String jobId, String taskId) { + final name = TaskName.of(projectId, location, jobId, 'group0', taskId) - return batchServiceClient.listTasks(parent).iterateAll() + return batchServiceClient.getTask(name) } + static private Set deletedJobs = [] as Set + void deleteJob(String jobId) { + // prevent duplicate delete requests on the same job + if( jobId in deletedJobs ) + return + else + deletedJobs.add(jobId) + final name = JobName.of(projectId, location, jobId).toString() batchServiceClient.deleteJobAsync(name) } - JobStatus getJobStatus(String jobId) { - final job = describeJob(jobId) - return job.getStatus() + TaskStatus getTaskStatus(String jobId, String taskId) { + return describeTask(jobId, taskId).getStatus() } - String getJobState(String jobId) { - final status = getJobStatus(jobId) + String getTaskState(String jobId, String taskId) { + final status = getTaskStatus(jobId, taskId) return status ? status.getState().toString() : null } diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy index 6ba5681b92..cebc81705b 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy @@ -17,6 +17,8 @@ package nextflow.cloud.google.batch +import java.nio.file.Path + import com.google.cloud.batch.v1.Volume import groovy.transform.Canonical @@ -32,6 +34,9 @@ class GoogleBatchLauncherSpecMock implements GoogleBatchLauncherSpec { List volumes = List.of() Map environment = Map.of() + @Override + Path toContainerMount(Path path) { path } + @Override List getContainerMounts() { return containerMounts diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy index 9c434a3195..0d8d946088 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy @@ -42,7 +42,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def 'should create submit request with minimal spec' () { given: def GCS_VOL = Volume.newBuilder().setGcs(GCS.newBuilder().setRemotePath('foo').build() ).build() - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') def CONTAINER_IMAGE = 'debian:latest' def exec = Mock(GoogleBatchExecutor) { getConfig() >> Mock(BatchConfig) @@ -105,7 +105,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def 'should create submit request with maximal spec' () { given: - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') and: def ACCELERATOR = new AcceleratorResource(request: 1, type: 'nvidia-tesla-v100') def BOOT_DISK = MemoryUnit.of('10 GB') @@ -220,6 +220,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def handler = Spy(GoogleBatchTaskHandler) handler.task = task handler.@jobId = 'xyz-123' + handler.@taskId = '0' handler.@uid = '789' when: @@ -227,14 +228,14 @@ class GoogleBatchTaskHandlerTest extends Specification { then: handler.isCompleted() >> false and: - trace.native_id == 'xyz-123/789' + trace.native_id == 'xyz-123/0/789' trace.executorName == 'google-batch' } def 'should create submit request with fusion enabled' () { given: def GCS_VOL = Volume.newBuilder().setGcs(GCS.newBuilder().setRemotePath('foo').build() ).build() - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') def CONTAINER_IMAGE = 'debian:latest' def exec = Mock(GoogleBatchExecutor) { getConfig() >> Mock(BatchConfig) diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy index 376a643bc4..218672d9aa 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy @@ -130,8 +130,8 @@ class BatchLoggingTest extends Specification { when: def state=null do { - state = batchClient.getJobState(jobId) - log.debug "Test job state=$state" + state = batchClient.getTaskState(jobId, '0') + log.debug "Test task state=$state" sleep 10_000 } while( state !in ['SUCCEEDED', 'FAILED'] ) then: From d3d037739003a71bff9b3e4a3b9bba8b06346bdb Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 26 May 2023 14:18:09 -0500 Subject: [PATCH 33/97] Fix static compilation errors Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/GridTaskHandler.groovy | 4 ++-- .../src/main/groovy/nextflow/executor/LsfExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/PbsExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/PbsProExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 2 +- .../src/main/groovy/nextflow/executor/SlurmExecutor.groovy | 2 +- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 044952dd23..7f9bcdae82 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -111,7 +111,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { @Override List getSubmitCommand() { final cmd = fusionEnabled() - ? fusionSubmitCommand() + ? fusionSubmitCommand(task) : "cd ${task.workDir} ; bash ${TaskRun.CMD_RUN} | tee ${TaskRun.CMD_LOG}" List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) @@ -289,7 +289,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { // -- execute with a re-triable strategy final result = safeExecute( () -> processStart(builder, stdinScript) ) // -- save the JobId in the - final jobId = executor.parseJobId(result) + final jobId = (String)executor.parseJobId(result) this.onSubmit(jobId) log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index d5fa968d35..5b5b52b93b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -74,7 +74,7 @@ class LsfExecutor extends AbstractGridExecutor { if( task instanceof TaskArray ) { final arraySize = ((TaskArray)task).getArraySize() - result << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]" + result << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]".toString() } result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 95f41d32f0..30c79e8e9a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -46,7 +46,7 @@ class PbsExecutor extends AbstractGridExecutor { if( task instanceof TaskArray ) { final arraySize = ((TaskArray)task).getArraySize() - result << '-J' << "0-${arraySize - 1}" + result << '-J' << "0-${arraySize - 1}".toString() } result << '-N' << getJobNameFor(task) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 70531b2485..7aa566721a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -48,7 +48,7 @@ class PbsProExecutor extends PbsExecutor { if( task instanceof TaskArray ) { final arraySize = ((TaskArray)task).getArraySize() - result << '-J' << "0-${arraySize - 1}" + result << '-J' << "0-${arraySize - 1}".toString() } // when multiple competing directives are provided, only the first one will take effect diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 3ec09e63f5..98c50a6b35 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -40,7 +40,7 @@ class SgeExecutor extends AbstractGridExecutor { if( task instanceof TaskArray ) { final arraySize = ((TaskArray)task).getArraySize() - result << '-t' << "0-${arraySize - 1}" + result << '-t' << "0-${arraySize - 1}".toString() } result << '-N' << getJobNameFor(task) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 276451aa4e..154542dc4d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -55,7 +55,7 @@ class SlurmExecutor extends AbstractGridExecutor { if( task instanceof TaskArray ) { final arraySize = ((TaskArray)task).getArraySize() - result << '--array' << "0-${arraySize - 1}" + result << '--array' << "0-${arraySize - 1}".toString() } result << '-J' << getJobNameFor(task) From acea09f98c52c88f3ff0c84902e7bf4a26f2ab67 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 26 May 2023 14:37:33 -0500 Subject: [PATCH 34/97] Delete entire AWS Batch array job on workflow termination Signed-off-by: Ben Sherman --- .../cloud/aws/batch/AwsBatchExecutor.groovy | 23 +++++++++++++++++++ .../aws/batch/AwsBatchTaskHandler.groovy | 12 +--------- 2 files changed, 24 insertions(+), 11 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index d5b4080d80..9236b384a1 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import com.amazonaws.services.batch.AWSBatch import com.amazonaws.services.batch.model.AWSBatchException +import com.amazonaws.services.batch.model.TerminateJobRequest import com.amazonaws.services.ecs.model.AccessDeniedException import com.amazonaws.services.logs.model.ResourceNotFoundException import nextflow.cloud.aws.nio.S3Path @@ -83,6 +84,8 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar private AwsOptions awsOptions + private Set deletedJobs = [] as Set + AwsOptions getAwsOptions() { awsOptions } /** @@ -264,6 +267,26 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar @PackageScope ThrottlingExecutor getReaper() { reaper } + synchronized void killTask(String jobId) { + // extract array job id + if( jobId.contains(':') ) + jobId = jobId.split(':')[0] + + // prevent duplicate delete requests on the same job + if( jobId in deletedJobs ) + return + else + deletedJobs.add(jobId) + + // submit terminate request + reaper.submit({ + final req = new TerminateJobRequest() + .withJobId(jobId) + .withReason('Job killed by NF') + final resp = client.terminateJob(req) + log.debug "[AWS BATCH] killing job=$jobId; response=$resp" + }) + } CloudMachineInfo getMachineInfoByQueueAndTaskArn(String queue, String taskArn) { try { diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 3e67294900..3bcb19f997 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -49,7 +49,6 @@ import com.amazonaws.services.batch.model.ResourceType import com.amazonaws.services.batch.model.RetryStrategy import com.amazonaws.services.batch.model.SubmitJobRequest import com.amazonaws.services.batch.model.SubmitJobResult -import com.amazonaws.services.batch.model.TerminateJobRequest import com.amazonaws.services.batch.model.Volume import groovy.transform.CompileStatic import groovy.transform.Memoized @@ -288,17 +287,8 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler Date: Mon, 29 May 2023 16:12:59 -0500 Subject: [PATCH 35/97] Fix failing test Signed-off-by: Ben Sherman --- .../cloud/aws/batch/AwsBatchExecutor.groovy | 16 ++++++++------ .../aws/batch/AwsBatchTaskHandlerTest.groovy | 8 +++---- .../executor/AwsBatchExecutorTest.groovy | 22 +++++++++++++++++++ 3 files changed, 34 insertions(+), 12 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 9236b384a1..3ec384d294 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -279,13 +279,15 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar deletedJobs.add(jobId) // submit terminate request - reaper.submit({ - final req = new TerminateJobRequest() - .withJobId(jobId) - .withReason('Job killed by NF') - final resp = client.terminateJob(req) - log.debug "[AWS BATCH] killing job=$jobId; response=$resp" - }) + reaper.submit({ killTask0(jobId) }) + } + + protected void killTask0(String jobId) { + final req = new TerminateJobRequest() + .withJobId(jobId) + .withReason('Job killed by NF') + final resp = client.terminateJob(req) + log.debug "[AWS BATCH] killing job=$jobId; response=$resp" } CloudMachineInfo getMachineInfoByQueueAndTaskArn(String queue, String taskArn) { diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy index 0f7199688e..2f6363cfe1 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy @@ -794,19 +794,17 @@ class AwsBatchTaskHandlerTest extends Specification { def 'should kill a job' () { given: def JOB_ID = '54321' + def executor = Mock(AwsBatchExecutor) def task = Mock(TaskRun) def handler = Spy(AwsBatchTaskHandler) + handler.@executor = executor handler.task = task handler.@jobId = JOB_ID - def req = Mock(TerminateJobRequest) - req.getJobId() >> JOB_ID - req.getReason() >> 'Job killed by NF' - when: handler.kill() then: - 1 * handler.terminateJob(req) >> null + 1 * executor.killTask(JOB_ID) } diff --git a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy index af11ae542a..711e70012e 100644 --- a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy @@ -10,6 +10,7 @@ package nextflow.executor import nextflow.Session import nextflow.SysEnv import nextflow.cloud.aws.batch.AwsBatchExecutor +import nextflow.util.ThrottlingExecutor import spock.lang.Specification /** @@ -43,4 +44,25 @@ class AwsBatchExecutorTest extends Specification { } + def 'should kill tasks' () { + given: + def reaper = Mock(ThrottlingExecutor) { + submit(_) >> { args -> args[0]() } + } + def executor = Spy(AwsBatchExecutor) + executor.@reaper = reaper + + when: + executor.killTask('job-id') + executor.killTask('job-id') + then: + 1 * executor.killTask0('job-id') >> null + + when: + executor.killTask('array-job-id:0') + executor.killTask('array-job-id:1') + then: + 1 * executor.killTask0('array-job-id') >> null + } + } From 75dc40443c6797d2e940f4aa8752e576d4941da3 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 11:45:32 -0500 Subject: [PATCH 36/97] Fix issues with google batch array jobs Signed-off-by: Ben Sherman --- .../google/batch/GoogleBatchScriptLauncher.groovy | 2 -- .../cloud/google/batch/GoogleBatchTaskHandler.groovy | 10 ++++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy index 01d48126f3..8d05bbee78 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy @@ -59,8 +59,6 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc // map bean work and target dirs to container mount // this is needed to create the command launcher using container local file paths - toContainerMount(bean.workDir.parent.parent) - bean.workDir = toContainerMount(bean.workDir) bean.targetDir = toContainerMount(bean.targetDir) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index a01f4fe360..d17fb8bbcb 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -17,6 +17,7 @@ package nextflow.cloud.google.batch + import java.nio.file.Path import com.google.cloud.batch.v1.AllocationPolicy @@ -143,6 +144,10 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { spec0(launcher).launchCommand() } + List getContainerMounts() { + spec0(launcher).getContainerMounts() + } + @Override void submit() { /* @@ -225,6 +230,11 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { if( containerOptions ) container.setOptions( containerOptions ) + // add child container mounts if task is an array + if( task instanceof TaskArray ) + for( TaskHandler handler : ((TaskArray)task).children ) + container.addAllVolumes( ((GoogleBatchTaskHandler)handler).getContainerMounts() ) + // task spec final env = Environment .newBuilder() From 7c7a7b3cb7919cf5dbd2dbc19c28f1eb33de5d91 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 12:50:14 -0500 Subject: [PATCH 37/97] Rename getSubmitCommand() to getLaunchCommand() Signed-off-by: Ben Sherman --- .../nextflow/executor/GridTaskHandler.groovy | 8 ++++---- .../processor/TaskArrayCollector.groovy | 2 +- .../nextflow/processor/TaskHandler.groovy | 2 +- .../cloud/aws/batch/AwsBatchTaskHandler.groovy | 4 ++-- .../aws/batch/AwsBatchTaskHandlerTest.groovy | 18 +++++++++--------- .../google/batch/GoogleBatchTaskHandler.groovy | 2 +- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 7f9bcdae82..57eaa0f9d8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -109,9 +109,9 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { } @Override - List getSubmitCommand() { + List getLaunchCommand() { final cmd = fusionEnabled() - ? fusionSubmitCommand(task) + ? fusionLaunchCommand(task) : "cd ${task.workDir} ; bash ${TaskRun.CMD_RUN} | tee ${TaskRun.CMD_LOG}" List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) @@ -238,7 +238,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { final builder = new StringBuilder() << '#!/bin/bash\n' << fusionSubmitDirective(task) - << fusionSubmitCommand(task) << '\n' + << fusionLaunchCommand(task) << '\n' return builder.toString() } @@ -252,7 +252,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } - protected String fusionSubmitCommand(TaskRun task) { + protected String fusionLaunchCommand(TaskRun task) { final launcher = fusionLauncher() final config = task.getContainerConfig() final submit = fusionSubmitCli() diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 592dbb6302..f3d0f2a241 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -157,7 +157,7 @@ class TaskArrayCollector { protected String createTaskArrayScript(List array) { // get work directory and launch command for each task final workDirs = array.collect( handler -> handler.getWorkDir() ) - final args = array.first().getSubmitCommand().toArray() as String[] + final args = array.first().getLaunchCommand().toArray() as String[] final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${task_dir}') // create wrapper script diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 1a884a6ca9..81b1d02bd5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -104,7 +104,7 @@ abstract class TaskHandler { /** * Get the command to execute the launcher script. */ - List getSubmitCommand() { null } + List getLaunchCommand() { null } /** * Task status attribute setter. diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index f2209b0213..01a1c9bd41 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -640,7 +640,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler getSubmitCommand() { + List getLaunchCommand() { // final launcher command return fusionEnabled() ? fusionSubmitCli() @@ -714,7 +714,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler(5) def container = new ContainerOverrides() - container.command = getSubmitCommand() + container.command = getLaunchCommand() // set the task memory if( task.config.getMemory() ) { final mega = (int)task.config.getMemory().toMega() diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy index 2f6363cfe1..d3803b7665 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy @@ -88,7 +88,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -109,7 +109,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: req = handler.newSubmitRequest(task) then: - 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 0 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'], region: 'eu-west-1')) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -139,7 +139,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'],client: [storageEncryption: 'AES256'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -156,7 +156,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req2 = handler.newSubmitRequest(task) then: - 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] + 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws',schedulingPriority: 9999,shareIdentifier: 'priority/high'], client:[storageEncryption: 'AES256', debug: true])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -282,7 +282,7 @@ class AwsBatchTaskHandlerTest extends Specification { handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws', retryMode: 'adaptive', maxTransferAttempts: 10])) } and: _ * handler.fusionEnabled() >> false - 1 * handler.getSubmitCommand() >> ['bash','-c','foo'] + 1 * handler.getLaunchCommand() >> ['bash','-c','foo'] 1 * handler.maxSpotAttempts() >> 3 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' @@ -845,7 +845,7 @@ class AwsBatchTaskHandlerTest extends Specification { } when: - def result = handler.getSubmitCommand() + def result = handler.getLaunchCommand() then: handler.getAwsOptions() >> Mock(AwsOptions) { getAwsCli() >> 'aws' } handler.getLogFile() >> Paths.get('/work/log') @@ -854,7 +854,7 @@ class AwsBatchTaskHandlerTest extends Specification { result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors .command.log s3://work/log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors s3://work/run - | bash 2>&1 | tee .command.log' when: - result = handler.getSubmitCommand() + result = handler.getLaunchCommand() then: handler.getAwsOptions() >> Mock(AwsOptions) { getAwsCli() >> 'aws'; @@ -883,7 +883,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getSubmitCommand() >> ['sh','-c','hello'] + 1 * handler.getLaunchCommand() >> ['sh','-c','hello'] 1 * handler.maxSpotAttempts() >> 5 1 * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -914,7 +914,7 @@ class AwsBatchTaskHandlerTest extends Specification { } when: - def result = handler.getSubmitCommand() + def result = handler.getLaunchCommand() then: result.join(' ') == '/usr/bin/fusion bash /fusion/s3/my-bucket/work/dir/.command.run' } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index d17fb8bbcb..7b8433bbdf 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -140,7 +140,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { } @Override - List getSubmitCommand() { + List getLaunchCommand() { spec0(launcher).launchCommand() } From 52f3195802af3631142a753be440c2054c2740df Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 14:06:08 -0500 Subject: [PATCH 38/97] Fix SLURM+Fusion+array jobs, cleanup Signed-off-by: Ben Sherman --- docs/process.md | 10 +++---- .../nextflow/executor/GridTaskHandler.groovy | 26 +++++++------------ .../executor/GridTaskHandlerTest.groovy | 2 +- .../cloud/aws/batch/AwsBatchExecutor.groovy | 2 +- .../google/batch/GoogleBatchExecutor.groovy | 13 ++++++++++ .../batch/GoogleBatchFusionAdapter.groovy | 6 ----- .../batch/GoogleBatchLauncherSpec.groovy | 7 ----- .../batch/GoogleBatchScriptLauncher.groovy | 2 +- .../batch/GoogleBatchTaskHandler.groovy | 4 +-- .../google/batch/client/BatchClient.groovy | 8 ------ .../batch/GoogleBatchLauncherSpecMock.groovy | 5 ---- .../batch/GoogleBatchTaskHandlerTest.groovy | 6 ++--- 12 files changed, 35 insertions(+), 56 deletions(-) diff --git a/docs/process.md b/docs/process.md index e8439eed5e..1013df70bd 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1252,19 +1252,17 @@ Examples of values for the architecture `target` option are `cascadelake`, `icel ## array -:::{note} -This feature requires Nextflow version `23.05.0-edge` or later. +:::{versionadded} 23.06.0-edge ::: -:::{warning} -This feature is experimental and may change in a future release. +:::{warning} *Experimental: may change in a future release.* ::: The `array` directive allows you to submit tasks as *array jobs* for executors that support it. An array job is a collection of jobs with the same resource requirements and the same script (parameterized by an index). Array jobs incur significantly less scheduling overhead compared to individual jobs, and as a result they are preferred by HPC schedulers where possible. -It should be specified with a given array size, as well as an executor that supports array jobs. For example: +The directive should be specified with a given array size, along with an executor that supports array jobs. For example: ```groovy process cpu_task { @@ -1303,7 +1301,7 @@ The following directives must be uniform across all tasks in a process that uses - {ref}`process-resourcelabels` - {ref}`process-time` -For cloud-based executors like AWS Batch, the following additional directives must be uniform: +For cloud-based executors like AWS Batch, or when using Fusion with any executor, the following additional directives must be uniform: - {ref}`process-container` - {ref}`process-containerOptions` diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 57eaa0f9d8..b78a4e8d4c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -44,6 +44,7 @@ import nextflow.processor.TaskRun import nextflow.trace.TraceRecord import nextflow.util.CmdLineHelper import nextflow.util.Duration +import nextflow.util.Escape import nextflow.util.Throttle /** * Handles a job execution in the underlying grid platform @@ -110,9 +111,8 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { @Override List getLaunchCommand() { - final cmd = fusionEnabled() - ? fusionLaunchCommand(task) - : "cd ${task.workDir} ; bash ${TaskRun.CMD_RUN} | tee ${TaskRun.CMD_LOG}" + final workDir = Escape.path(getWorkDir()) + final cmd = "bash ${workDir}/${TaskRun.CMD_RUN} 2>&1 | tee ${workDir}/${TaskRun.CMD_LOG}" List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) } @@ -235,14 +235,15 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { } protected String fusionStdinWrapper() { - final builder = new StringBuilder() - << '#!/bin/bash\n' - << fusionSubmitDirective(task) - << fusionLaunchCommand(task) << '\n' - return builder.toString() + final submit = fusionSubmitCli() + final launcher = fusionLauncher() + final config = task.getContainerConfig() + final cmd = FusionHelper.runWithContainer(launcher, config, task.getContainer(), submit) + // create an inline script to launch the job execution + return '#!/bin/bash\n' + submitDirective(task) + cmd + '\n' } - protected String fusionSubmitDirective(TaskRun task) { + protected String submitDirective(TaskRun task) { final remoteLog = task.workDir.resolve(TaskRun.CMD_LOG).toString() // replaces the log file with a null file because the cluster submit tool // cannot write to a file hosted in a remote object storage @@ -252,13 +253,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } - protected String fusionLaunchCommand(TaskRun task) { - final launcher = fusionLauncher() - final config = task.getContainerConfig() - final submit = fusionSubmitCli() - return FusionHelper.runWithContainer(launcher, config, task.getContainer(), submit) - } - protected String launchCmd0(ProcessBuilder builder, String pipeScript) { def result = CmdLineHelper.toLine(builder.command()) if( pipeScript ) { diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 2d2a7ec13c..01a028e05a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -86,7 +86,7 @@ class GridTaskHandlerTest extends Specification { def handler = Spy(new GridTaskHandler(task, exec)) when: - def result = handler.fusionSubmitDirective(task) + def result = handler.submitDirective(task) then: 1 * exec.getHeaders(task) >> "#FOO this\n#BAR that\n#OUT file=${WORK_DIR}/.command.log\n" diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 3ec384d294..5de43bc82c 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -267,7 +267,7 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar @PackageScope ThrottlingExecutor getReaper() { reaper } - synchronized void killTask(String jobId) { + void killTask(String jobId) { // extract array job id if( jobId.contains(':') ) jobId = jobId.split(':')[0] diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy index c8f60336bf..5bb486e789 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy @@ -51,6 +51,8 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayA private Path remoteBinDir private BatchLogging logging + private Set deletedJobs = [] as Set + BatchClient getClient() { return client } BatchConfig getConfig() { return config } Path getRemoteBinDir() { return remoteBinDir } @@ -126,6 +128,17 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayA return FusionHelper.isFusionEnabled(session) } + void killTask(String jobId) { + // prevent duplicate delete requests on the same job + if( jobId in deletedJobs ) + return + else + deletedJobs.add(jobId) + + // delete job + client.deleteJob(jobId) + } + @Override String getArrayIndexName() { 'BATCH_TASK_INDEX' } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchFusionAdapter.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchFusionAdapter.groovy index cc9975468d..84ddd86486 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchFusionAdapter.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchFusionAdapter.groovy @@ -17,7 +17,6 @@ package nextflow.cloud.google.batch -import java.nio.file.Path import com.google.cloud.batch.v1.Volume import groovy.transform.CompileStatic @@ -41,11 +40,6 @@ class GoogleBatchFusionAdapter implements GoogleBatchLauncherSpec { this.launcher = launcher } - @Override - Path toContainerMount(Path path) { - return launcher.toContainerMount(path) - } - @Override List getContainerMounts() { return List.of() diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy index aa9ee78a0f..fc34f7f1bf 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchLauncherSpec.groovy @@ -16,8 +16,6 @@ package nextflow.cloud.google.batch -import java.nio.file.Path - import com.google.cloud.batch.v1.Volume /** @@ -27,11 +25,6 @@ import com.google.cloud.batch.v1.Volume */ interface GoogleBatchLauncherSpec { - /** - * @return The equvalent container mount for a remote path. - */ - Path toContainerMount(Path path) - /** * @return * A list of string representing the container mounts. Each mount uses the docker diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy index 8d05bbee78..ae4dc6b5f6 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy @@ -96,7 +96,7 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc return result } - Path toContainerMount(Path path, boolean parent=false) { + protected Path toContainerMount(Path path, boolean parent=false) { if( path instanceof CloudStoragePath ) { buckets.add(path.bucket()) pathTrie.add( (parent ? "/${path.bucket()}${path.parent}" : "/${path.bucket()}${path}").toString() ) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 7b8433bbdf..5b210509ff 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -136,7 +136,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { @Override String getWorkDir() { - spec0(launcher).toContainerMount(task.workDir).toString() + launcher.workDir.toString() } @Override @@ -463,7 +463,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { void kill() { if( isSubmitted() ) { log.trace "[GOOGLE BATCH] Process `${task.lazyName()}` - deleting job name=$jobId" - client.deleteJob(jobId) + executor.killTask(jobId) } else { log.debug "[GOOGLE BATCH] Process `${task.lazyName()}` - invalid delete action" diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy index c4db64292b..fb7170accb 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/client/BatchClient.groovy @@ -99,15 +99,7 @@ class BatchClient { return batchServiceClient.getTask(name) } - static private Set deletedJobs = [] as Set - void deleteJob(String jobId) { - // prevent duplicate delete requests on the same job - if( jobId in deletedJobs ) - return - else - deletedJobs.add(jobId) - final name = JobName.of(projectId, location, jobId).toString() batchServiceClient.deleteJobAsync(name) diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy index cebc81705b..6ba5681b92 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchLauncherSpecMock.groovy @@ -17,8 +17,6 @@ package nextflow.cloud.google.batch -import java.nio.file.Path - import com.google.cloud.batch.v1.Volume import groovy.transform.Canonical @@ -34,9 +32,6 @@ class GoogleBatchLauncherSpecMock implements GoogleBatchLauncherSpec { List volumes = List.of() Map environment = Map.of() - @Override - Path toContainerMount(Path path) { path } - @Override List getContainerMounts() { return containerMounts diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy index 3ca4b4b51c..9aa19b8fb8 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy @@ -45,7 +45,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def 'should create submit request with minimal spec' () { given: def GCS_VOL = Volume.newBuilder().setGcs(GCS.newBuilder().setRemotePath('foo').build() ).build() - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') def CONTAINER_IMAGE = 'debian:latest' def exec = Mock(GoogleBatchExecutor) { getConfig() >> Mock(BatchConfig) @@ -109,7 +109,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def 'should create submit request with maximal spec' () { given: - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') and: def ACCELERATOR = new AcceleratorResource(request: 1, type: 'nvidia-tesla-v100') def BOOT_DISK = MemoryUnit.of('10 GB') @@ -250,7 +250,7 @@ class GoogleBatchTaskHandlerTest extends Specification { def 'should create submit request with fusion enabled' () { given: - def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch/01/23456789abcdef') + def WORK_DIR = CloudStorageFileSystem.forBucket('foo').getPath('/scratch') def CONTAINER_IMAGE = 'debian:latest' def exec = Mock(GoogleBatchExecutor) { getConfig() >> Mock(BatchConfig) From c5e8b76ff770b9b43713a1bc853e8367f1c4098e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 14:37:44 -0500 Subject: [PATCH 39/97] Remove unrelated changes Signed-off-by: Ben Sherman --- .../nextflow/cloud/aws/config/AwsS3Config.groovy | 4 ++-- .../cloud/aws/config/AwsS3ConfigTest.groovy | 14 -------------- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy index 5ff28aac75..e479925314 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/config/AwsS3Config.groovy @@ -50,8 +50,8 @@ class AwsS3Config { this.debug = opts.debug as Boolean this.endpoint = opts.endpoint ?: SysEnv.get('AWS_S3_ENDPOINT') this.storageClass = parseStorageClass((opts.storageClass ?: opts.uploadStorageClass) as String) // 'uploadStorageClass' is kept for legacy purposes - this.storageEncryption = parseStorageEncryption(opts.storageEncryption as String) ?: SysEnv.get('NXF_AWS_SSE_MODE') - this.storageKmsKeyId = opts.storageKmsKeyId ?: SysEnv.get('NXF_AWS_SSE_KMS_KEY_ID') + this.storageEncryption = parseStorageEncryption(opts.storageEncryption as String) + this.storageKmsKeyId = opts.storageKmsKeyId this.pathStyleAccess = opts.s3PathStyleAccess as Boolean this.s3Acl = parseS3Acl(opts.s3Acl as String) } diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy index 0f5cd1f643..116a64456b 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/config/AwsS3ConfigTest.groovy @@ -121,18 +121,4 @@ class AwsS3ConfigTest extends Specification { SysEnv.pop() } - - def 'should set storage encryption via env variable' () { - given: - SysEnv.push([NXF_AWS_SSE_MODE: 'aws:kms', NXF_AWS_SSE_KMS_KEY_ID: 'xyz1']) - - when: - def client = new AwsS3Config([:]) - then: - client.storageKmsKeyId == 'xyz1' - client.storageEncryption == 'aws:kms' - - cleanup: - SysEnv.pop() - } } From 14b2455afc712a5798c25394cd148833390b5de4 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 18:04:19 -0500 Subject: [PATCH 40/97] Submit retried tasks directly Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index f3d0f2a241..1f26c8cb32 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -68,7 +68,8 @@ class TaskArrayCollector { try { // submit task directly if the collector is closed - if( closed ) { + // or if the task is retried (since it might have dynamic resources) + if( closed || task.config.getAttempt() > 1 ) { executor.submit(task) return } From 0de3c849434e672d1550e2520fc667133ea9d478 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 May 2023 19:45:15 -0500 Subject: [PATCH 41/97] Add unit tests Signed-off-by: Ben Sherman --- .../groovy/nextflow/executor/Executor.groovy | 2 +- .../nextflow/executor/GridTaskHandler.groovy | 8 +- .../processor/TaskArrayCollector.groovy | 2 +- .../executor/GridTaskHandlerTest.groovy | 12 +- .../processor/TaskArrayCollectorTest.groovy | 138 ++++++++++++++++++ .../processor/TaskPollingMonitorTest.groovy | 23 +++ .../executor/AwsBatchExecutorTest.groovy | 2 +- .../batch/GoogleBatchExecutorTest.groovy | 13 ++ 8 files changed, 187 insertions(+), 13 deletions(-) create mode 100644 modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy index 18742637ba..44ccbaef76 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/Executor.groovy @@ -81,7 +81,7 @@ abstract class Executor { * * @param task A {@code TaskRun} instance */ - final void submit( TaskRun task ) { + void submit( TaskRun task ) { log.trace "Scheduling process: ${task}" if( session.isTerminated() ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index b78a4e8d4c..f05ef843ae 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -206,7 +206,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { // -- wait the the process completes final result = process.text final exitStatus = process.waitFor() - final cmd = launchCmd0(builder,pipeScript) + final cmd = submitCmd0(builder,pipeScript) if( exitStatus ) { throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) @@ -253,12 +253,12 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } - protected String launchCmd0(ProcessBuilder builder, String pipeScript) { + protected String submitCmd0(ProcessBuilder builder, String pipeScript) { def result = CmdLineHelper.toLine(builder.command()) if( pipeScript ) { - result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" + result = "cat << 'SUBMIT_COMMAND_EOF' | ${result}\n" result += pipeScript.trim() + '\n' - result += 'LAUNCH_COMMAND_EOF\n' + result += 'SUBMIT_COMMAND_EOF\n' } return result } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 1f26c8cb32..11e4c2781a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -157,7 +157,7 @@ class TaskArrayCollector { */ protected String createTaskArrayScript(List array) { // get work directory and launch command for each task - final workDirs = array.collect( handler -> handler.getWorkDir() ) + final workDirs = array.collect( h -> h.getWorkDir() ) final args = array.first().getLaunchCommand().toArray() as String[] final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${task_dir}') diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 01a028e05a..2fae32ae30 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -127,18 +127,18 @@ class GridTaskHandlerTest extends Specification { '''.stripIndent(true) } - def 'should create launch command' () { + def 'should get submit command' () { given: - def exec = Spy(GridTaskHandler) + def handler = Spy(GridTaskHandler) expect: - exec.launchCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' + handler.submitCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' and: - exec.launchCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == + handler.submitCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == '''\ - cat << 'LAUNCH_COMMAND_EOF' | qsub + cat << 'SUBMIT_COMMAND_EOF' | qsub docker run /some/file - LAUNCH_COMMAND_EOF + SUBMIT_COMMAND_EOF '''.stripIndent() } } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy new file mode 100644 index 0000000000..ccdff4a1e6 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -0,0 +1,138 @@ +/* + * Copyright 2013-2023, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package nextflow.processor + +import java.nio.file.Paths + +import com.google.common.hash.HashCode +import nextflow.executor.Executor +import nextflow.executor.TaskArrayAware +import nextflow.script.BodyDef +import nextflow.script.ProcessConfig +import spock.lang.Specification +import test.TestHelper +/** + * + * @author Ben Sherman + */ +class TaskArrayCollectorTest extends Specification { + + static class DummyExecutor extends Executor implements TaskArrayAware { + TaskMonitor createTaskMonitor() { null } + TaskHandler createTaskHandler(TaskRun task) { null } + } + + def 'should submit tasks as array jobs' () { + given: + def executor = Mock(DummyExecutor) + def handler = Mock(TaskHandler) + def taskArray = [:] as TaskArray + def collector = Spy(new TaskArrayCollector(executor, 5)) { + createTaskArray(_) >> taskArray + } + and: + def task = Mock(TaskRun) { + getConfig() >> Mock(TaskConfig) { + getAttempt() >> 1 + } + } + + // collect tasks into array job + when: + collector.collect(task) + collector.collect(task) + collector.collect(task) + collector.collect(task) + then: + 4 * executor.createTaskHandler(task) >> handler + 0 * executor.submit(_) + + // submit array job when it is ready + when: + collector.collect(task) + then: + 1 * executor.createTaskHandler(task) >> handler + 5 * handler.prepareLauncher() + 1 * executor.submit(taskArray) + + // submit partial array job when closed + when: + collector.collect(task) + collector.close() + then: + 1 * executor.createTaskHandler(task) >> handler + 1 * handler.prepareLauncher() + 1 * executor.submit(taskArray) + + // submit tasks directly once closed + when: + collector.collect(task) + then: + 1 * executor.submit(task) + } + + def 'should submit retried tasks directly' () { + given: + def executor = Mock(DummyExecutor) + def collector = Spy(new TaskArrayCollector(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 executor = Mock(DummyExecutor) { + getWorkDir() >> TestHelper.createInMemTempDir() + getArrayIndexName() >> 'ARRAY_JOB_INDEX' + } + def collector = Spy(new TaskArrayCollector(executor, 5)) + and: + def task = Mock(TaskRun) { + processor >> Mock(TaskProcessor) { + config >> Mock(ProcessConfig) + getTaskBody() >> { new BodyDef(null, 'source') } + } + getHash() >> HashCode.fromString('0123456789abcdef') + } + def handler = Mock(TaskHandler) { + getTask() >> task + getWorkDir() >> Paths.get('/work/foo') + getLaunchCommand() >> ['bash', '-o', 'pipefail', '-c', 'bash /work/foo/.command.run 2>&1 | tee /work/foo/.command.log'] + } + + when: + def taskArray = collector.createTaskArray([handler, handler, handler]) + then: + taskArray.config == task.config + taskArray.processor == task.processor + taskArray.script == ''' + array=( /work/foo /work/foo /work/foo ) + export task_dir=${array[ARRAY_JOB_INDEX]} + bash -o pipefail -c 'bash ${task_dir}/.command.run 2>&1 | tee ${task_dir}/.command.log' + '''.stripIndent().leftTrim() + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy index efe83cb4a3..f9f350218a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy @@ -124,5 +124,28 @@ class TaskPollingMonitorTest extends Specification { 0 * session.notifyTaskComplete(handler) >> null } + def 'should submit an array job' () { + given: + def session = Mock(Session) + def monitor = Spy(new TaskPollingMonitor(name: 'foo', session: session, pollInterval: Duration.of('1min'))) + and: + def handler = Mock(TaskHandler) { + getTask() >> Mock(TaskRun) + } + def arrayHandler = Mock(TaskHandler) { + getTask() >> Mock(TaskArray) { + children >> (1..3).collect( i -> handler ) + } + } + + when: + monitor.submit(arrayHandler) + then: + 1 * arrayHandler.prepareLauncher() + 1 * arrayHandler.submit() + 0 * handler.prepareLauncher() + 0 * handler.submit() + 3 * session.notifyTaskSubmit(handler) + } } diff --git a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy index 711e70012e..19800e3879 100644 --- a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy @@ -47,7 +47,7 @@ class AwsBatchExecutorTest extends Specification { def 'should kill tasks' () { given: def reaper = Mock(ThrottlingExecutor) { - submit(_) >> { args -> args[0]() } + submit(_) >> { Closure cl -> cl() } } def executor = Spy(AwsBatchExecutor) executor.@reaper = reaper diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy index 9e0c803bac..7bf1a28e5f 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy @@ -9,6 +9,7 @@ package nextflow.cloud.google.batch import nextflow.Session import nextflow.SysEnv +import nextflow.cloud.google.batch.client.BatchClient import spock.lang.Specification /** * @@ -41,4 +42,16 @@ class GoogleBatchExecutorTest extends Specification { } + def 'should kill tasks' () { + given: + def client = Mock(BatchClient) + def executor = new GoogleBatchExecutor(client: client) + + when: + executor.killTask('job-id') + executor.killTask('job-id') + then: + 1 * client.deleteJob('job-id') + } + } From 55d794126ad4f479f5cbfda71869750cffd8aa2b Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 13 Jun 2023 09:11:19 -0500 Subject: [PATCH 42/97] Update docs [ci fast] Signed-off-by: Ben Sherman --- docs/process.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/process.md b/docs/process.md index 1013df70bd..6c6ba94381 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1287,7 +1287,7 @@ Nextflow currently supports array jobs for the following executors: A process using array jobs will collect tasks and submit each batch as an array job when it is ready. Any "leftover" tasks will be submitted as a partial array job. -Once an array job is submitted, each "child" task is executed as an independent job. Any tasks that fail (and can be retried) will be retried in another array job without interfering with the tasks that succeeded. +Once an array job is submitted, each "child" task is executed as an independent job. Any tasks that fail (and can be retried) will be retried without interfering with the tasks that succeeded. Retried tasks are executed directly rather than through an array job, in order to allow for the use of [dynamic resources](#dynamic-computing-resources). The following directives must be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: From eeeb12d77ce0b2f10596c608ee272fcffa5598ba Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 13 Jun 2023 15:29:28 -0500 Subject: [PATCH 43/97] Fix issues with LSF array jobs Signed-off-by: Ben Sherman --- .../nextflow/executor/LsfExecutor.groovy | 21 ++++++++++--------- .../nextflow/executor/TaskArrayAware.groovy | 7 +++++++ .../processor/TaskArrayCollector.groovy | 9 ++++++-- 3 files changed, 25 insertions(+), 12 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 5b5b52b93b..4b0463e657 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -18,7 +18,6 @@ package nextflow.executor import java.nio.file.Path import java.nio.file.Paths -import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern import groovy.transform.CompileStatic @@ -44,8 +43,6 @@ class LsfExecutor extends AbstractGridExecutor { static private Pattern QUOTED_STRING_REGEX = ~/"((?:[^"\\]|\\.)*)"(\s*#.*)?/ - static private AtomicInteger taskArrayCount = new AtomicInteger() - private boolean perJobMemLimit private boolean perTaskReserve @@ -72,11 +69,6 @@ class LsfExecutor extends AbstractGridExecutor { */ protected List getDirectives(TaskRun task, List result) { - if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() - result << '-J' << "nf-array-${taskArrayCount.getAndIncrement()}[0-${arraySize - 1}]".toString() - } - result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() // add other parameters (if any) @@ -113,7 +105,13 @@ class LsfExecutor extends AbstractGridExecutor { } // -- the job name - result << '-J' << getJobNameFor(task) + if( task instanceof TaskArray ) { + final arraySize = ((TaskArray)task).getArraySize() + result << '-J' << "${getJobNameFor(task)}[1-${arraySize}]".toString() + } + else { + result << '-J' << getJobNameFor(task) + } // -- at the end append the command script wrapped file name result.addAll( task.config.getClusterOptionsAsList() ) @@ -318,6 +316,9 @@ class LsfExecutor extends AbstractGridExecutor { String getArrayIndexName() { 'LSB_JOBINDEX' } @Override - String getArrayTaskId(String jobId, int index) { "${jobId}[${index}]" } + int getArrayIndexStart() { 1 } + + @Override + String getArrayTaskId(String jobId, int index) { "${jobId}[${index + 1}]" } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index db9a695fd9..90ac81f2be 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -45,4 +45,11 @@ interface TaskArrayAware { throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") } + /** + * Get the start of the array job index range. + */ + default int getArrayIndexStart() { + return 0 + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 11e4c2781a..0388aef542 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -162,10 +162,15 @@ class TaskArrayCollector { final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${task_dir}') // create wrapper script - final arrayIndexName = executor.getArrayIndexName() + final indexName = executor.getArrayIndexName() + final indexStart = executor.getArrayIndexStart() + final arrayIndex = indexStart > 0 + ? "${indexName} + ${indexStart}" + : "${indexName}" + final builder = new StringBuilder() << "array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} )\n" - << "export task_dir=\${array[${arrayIndexName}]}\n" + << "export task_dir=\${array[${arrayIndex}]}\n" << cmd << '\n' return builder.toString() } From fd10361ff3f363b679733b34d98af04fccde98f8 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 14 Jun 2023 15:10:43 -0500 Subject: [PATCH 44/97] Add array index name to env whitelist Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/TaskArray.groovy | 12 ++++++++++++ .../nextflow/processor/TaskArrayCollectorTest.groovy | 6 ++++++ 2 files changed, 18 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy index 8a3b50ef16..9c199bc768 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy @@ -18,6 +18,8 @@ package nextflow.processor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import nextflow.container.ContainerConfig +import nextflow.executor.TaskArrayAware /** * Models a task array, which submits a collection of independent @@ -35,4 +37,14 @@ class TaskArray extends TaskRun { children.size() } + @Override + ContainerConfig getContainerConfig() { + final config = super.getContainerConfig() + final envWhitelist = config.getEnvWhitelist() ?: [] + final executor = (TaskArrayAware)processor.getExecutor() + envWhitelist << executor.getArrayIndexName() + config.put('envWhitelist', envWhitelist) + return config + } + } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index ccdff4a1e6..5503966af6 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -19,6 +19,7 @@ package nextflow.processor import java.nio.file.Paths import com.google.common.hash.HashCode +import nextflow.Session import nextflow.executor.Executor import nextflow.executor.TaskArrayAware import nextflow.script.BodyDef @@ -113,6 +114,8 @@ class TaskArrayCollectorTest extends Specification { def task = Mock(TaskRun) { processor >> Mock(TaskProcessor) { config >> Mock(ProcessConfig) + getExecutor() >> executor + getSession() >> Mock(Session) getTaskBody() >> { new BodyDef(null, 'source') } } getHash() >> HashCode.fromString('0123456789abcdef') @@ -133,6 +136,9 @@ class TaskArrayCollectorTest extends Specification { export task_dir=${array[ARRAY_JOB_INDEX]} bash -o pipefail -c 'bash ${task_dir}/.command.run 2>&1 | tee ${task_dir}/.command.log' '''.stripIndent().leftTrim() + and: + taskArray.getArraySize() == 3 + taskArray.getContainerConfig().getEnvWhitelist() == [ 'ARRAY_JOB_INDEX' ] } } From 782dea2ec2ef51c90ad61ad8197323f9fa475167 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 15 Jun 2023 10:52:34 -0500 Subject: [PATCH 45/97] Disable non-native container run for task array Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/processor/TaskArray.groovy | 5 +++++ .../groovy/nextflow/processor/TaskArrayCollectorTest.groovy | 1 + 2 files changed, 6 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy index 9c199bc768..6a2fa0a6d4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy @@ -47,4 +47,9 @@ class TaskArray extends TaskRun { return config } + @Override + boolean isContainerEnabled() { + return false + } + } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 5503966af6..1238b4d71d 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -139,6 +139,7 @@ class TaskArrayCollectorTest extends Specification { and: taskArray.getArraySize() == 3 taskArray.getContainerConfig().getEnvWhitelist() == [ 'ARRAY_JOB_INDEX' ] + taskArray.isContainerEnabled() == false } } From eb450697ecf824c7fa645b5c370e1483848c2774 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 21 Jun 2023 14:43:20 -0500 Subject: [PATCH 46/97] Fix incorrect index offset in array job script Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 0388aef542..9b126430dd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -165,7 +165,7 @@ class TaskArrayCollector { final indexName = executor.getArrayIndexName() final indexStart = executor.getArrayIndexStart() final arrayIndex = indexStart > 0 - ? "${indexName} + ${indexStart}" + ? "${indexName} - ${indexStart}" : "${indexName}" final builder = new StringBuilder() From 50c300c0ad80e40d838cf3fd983e8f176238042e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 22 Jun 2023 00:03:04 -0500 Subject: [PATCH 47/97] Change SGE array index start to 1 Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 98c50a6b35..6579847c0f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -195,6 +195,9 @@ class SgeExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'SGE_TASK_ID' } + @Override + int getArrayIndexStart() { 1 } + @Override String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } From c206a5cef5336c87b5410065b5a25e26f4198dc8 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 23 Jun 2023 13:10:32 -0500 Subject: [PATCH 48/97] minor edits Signed-off-by: Ben Sherman --- docs/process.md | 2 +- .../executor/AbstractGridExecutor.groovy | 1 - .../nextflow/executor/GridTaskHandler.groovy | 16 ++++++++-------- .../groovy/nextflow/executor/LsfExecutor.groovy | 2 +- .../groovy/nextflow/executor/PbsExecutor.groovy | 2 +- .../nextflow/executor/PbsProExecutor.groovy | 2 +- .../groovy/nextflow/executor/SgeExecutor.groovy | 2 +- .../nextflow/executor/SlurmExecutor.groovy | 2 +- .../nextflow/processor/TaskArrayCollector.groovy | 10 +++++----- .../nextflow/processor/TaskPollingMonitor.groovy | 6 +++--- .../cloud/aws/batch/AwsBatchTaskHandler.groovy | 4 ++-- .../google/batch/GoogleBatchTaskHandler.groovy | 6 +++--- 12 files changed, 27 insertions(+), 28 deletions(-) diff --git a/docs/process.md b/docs/process.md index 6c6ba94381..9b25767a98 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1252,7 +1252,7 @@ Examples of values for the architecture `target` option are `cascadelake`, `icel ## array -:::{versionadded} 23.06.0-edge +:::{versionadded} 23.07.0-edge ::: :::{warning} *Experimental: may change in a future release.* diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index caea8e91c3..e426e425e8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -21,7 +21,6 @@ import java.nio.file.Path import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.util.logging.Slf4j -import nextflow.processor.TaskHandler import nextflow.processor.TaskMonitor import nextflow.processor.TaskPollingMonitor import nextflow.processor.TaskProcessor diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index f05ef843ae..315e88fb47 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -102,6 +102,12 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { this.sanityCheckInterval = duration } + @Override + void prepareLauncher() { + // -- create the wrapper script + createTaskWrapper(task).build() + } + @Override String getWorkDir() { fusionEnabled() @@ -263,12 +269,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } - @Override - void prepareLauncher() { - // -- create the wrapper script - createTaskWrapper(task).build() - } - /* * {@inheritDocs} */ @@ -282,7 +282,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { final stdinScript = executor.pipeLauncherScript() ? stdinLauncherScript() : null // -- execute with a re-triable strategy final result = safeExecute( () -> processStart(builder, stdinScript) ) - // -- save the JobId in the + // -- save the job id final jobId = (String)executor.parseJobId(result) this.onSubmit(jobId) log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" @@ -306,7 +306,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { void onSubmit(String jobId) { if( task instanceof TaskArray ) { - ((TaskArray)task).children.eachWithIndex { handler, i -> + task.children.eachWithIndex { handler, i -> final arrayTaskId = executor.getArrayTaskId(jobId, i) ((GridTaskHandler)handler).onSubmit(arrayTaskId) } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 4b0463e657..e5e32b61c5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -106,7 +106,7 @@ class LsfExecutor extends AbstractGridExecutor { // -- the job name if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() result << '-J' << "${getJobNameFor(task)}[1-${arraySize}]".toString() } else { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 30c79e8e9a..fc4b588c8d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -45,7 +45,7 @@ class PbsExecutor extends AbstractGridExecutor { assert result !=null if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() result << '-J' << "0-${arraySize - 1}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 7aa566721a..aab164fa85 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -47,7 +47,7 @@ class PbsProExecutor extends PbsExecutor { assert result !=null if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() result << '-J' << "0-${arraySize - 1}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 6579847c0f..02b2312fa0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -39,7 +39,7 @@ class SgeExecutor extends AbstractGridExecutor { protected List getDirectives(TaskRun task, List result) { if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() result << '-t' << "0-${arraySize - 1}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 154542dc4d..81a621038f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -54,7 +54,7 @@ class SlurmExecutor extends AbstractGridExecutor { protected List getDirectives(TaskRun task, List result) { if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() result << '--array' << "0-${arraySize - 1}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 9b126430dd..f42fbe64db 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -168,11 +168,11 @@ class TaskArrayCollector { ? "${indexName} - ${indexStart}" : "${indexName}" - final builder = new StringBuilder() - << "array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} )\n" - << "export task_dir=\${array[${arrayIndex}]}\n" - << cmd << '\n' - return builder.toString() + """ + array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) + export task_dir=\${array[${arrayIndex}]} + ${cmd} + """.stripIndent().leftTrim() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index edf5bd6535..d259af06ba 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -197,7 +197,7 @@ class TaskPollingMonitor implements TaskMonitor { handler.prepareLauncher() handler.submit() // add each child task to the running queue - for( TaskHandler h0 : ((TaskArray)handler.task).children ) { + for( TaskHandler h0 : handler.task.children ) { runningQueue.add(h0) session.notifyTaskSubmit(h0) } @@ -692,7 +692,7 @@ class TaskPollingMonitor implements TaskMonitor { protected void notifyTaskPending0(TaskHandler handler) { if( handler.task instanceof TaskArray ) - for( TaskHandler h0 : ((TaskArray)handler.task).children ) + for( TaskHandler h0 : handler.task.children ) session.notifyTaskPending(h0) else session.notifyTaskPending(handler) @@ -700,7 +700,7 @@ class TaskPollingMonitor implements TaskMonitor { protected void notifyTaskComplete0(TaskHandler handler) { if( handler.task instanceof TaskArray ) - for( TaskHandler h0 : ((TaskArray)handler.task).children ) + for( TaskHandler h0 : handler.task.children ) session.notifyTaskComplete(h0) else session.notifyTaskComplete(handler) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 01a1c9bd41..12bd8869de 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -319,7 +319,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler + task.children.eachWithIndex { handler, i -> final arrayTaskId = "${jobId}:${i}" ((AwsBatchTaskHandler)handler).onSubmit(arrayTaskId, queueName) } @@ -742,7 +742,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler 10_000 ) throw new IllegalArgumentException("Array jobs on AWS Batch may not have more than 10,000 tasks") diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 5b210509ff..bf2a01468b 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -163,7 +163,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { void onSubmit(String jobId, String taskId, String uid) { if( task instanceof TaskArray ) { - ((TaskArray)task).children.eachWithIndex { handler, i -> + task.children.eachWithIndex { handler, i -> final arrayTaskId = i.toString() ((GoogleBatchTaskHandler)handler).onSubmit(jobId, arrayTaskId, uid) } @@ -232,7 +232,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { // add child container mounts if task is an array if( task instanceof TaskArray ) - for( TaskHandler handler : ((TaskArray)task).children ) + for( TaskHandler handler : task.children ) container.addAllVolumes( ((GoogleBatchTaskHandler)handler).getContainerMounts() ) // task spec @@ -366,7 +366,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { .setTaskSpec(taskSpec) if( task instanceof TaskArray ) { - final arraySize = ((TaskArray)task).getArraySize() + final arraySize = task.getArraySize() taskGroup.setTaskCount(arraySize) } From d4ab783dd354807f4889aceda87f2a010fb07932 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 13 Jul 2023 00:15:21 -0500 Subject: [PATCH 49/97] Change SGE array index start to 1 (for real this time) Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 02b2312fa0..e6a621d743 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -40,7 +40,7 @@ class SgeExecutor extends AbstractGridExecutor { if( task instanceof TaskArray ) { final arraySize = task.getArraySize() - result << '-t' << "0-${arraySize - 1}".toString() + result << '-t' << "1-${arraySize}".toString() } result << '-N' << getJobNameFor(task) From 786a0b34ab915ca7ed381817fe8eb978f2bb7bdd Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 27 Jul 2023 09:36:47 -0500 Subject: [PATCH 50/97] Use Bolts.withLock Signed-off-by: Ben Sherman --- .../nextflow/processor/TaskArrayCollector.groovy | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index f42fbe64db..b5cd597c36 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -64,9 +64,7 @@ class TaskArrayCollector { * @param task */ void collect(TaskRun task) { - sync.lock() - - try { + sync.withLock { // 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 ) { @@ -86,26 +84,18 @@ class TaskArrayCollector { array = new ArrayList<>(arraySize) } } - finally { - sync.unlock() - } } /** * Close the collector, submitting any remaining tasks as a partial array job. */ void close() { - sync.lock() - - try { + sync.withLock { if( array.size() > 0 ) submit0(array) closed = true } - finally { - sync.unlock() - } } protected void submit0(List array) { From 28dbe47e70ed817c56e3e6ba0f0a72e604f8bf03 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 14 Sep 2023 21:16:45 -0500 Subject: [PATCH 51/97] Add array jobs to CRG executor Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/CrgExecutor.groovy | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy index 0b12dd9c8a..aa5ee7171b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy @@ -18,6 +18,7 @@ package nextflow.executor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import nextflow.processor.TaskArray import nextflow.processor.TaskRun /** * An executor specialised for CRG cluster @@ -41,6 +42,11 @@ class CrgExecutor extends SgeExecutor { task.config.penv = 'smp' } + if( task instanceof TaskArray ) { + final arraySize = task.getArraySize() + result << '-t' << "1-${arraySize}".toString() + } + result << '-N' << getJobNameFor(task) result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) result << '-j' << 'y' From 6c7494aa85bf5c991057f9e836b3995bd7946dff Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 18 Sep 2023 09:47:55 -0500 Subject: [PATCH 52/97] Add array job to SGE parseJobId Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index e6a621d743..f167422e0b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -120,8 +120,14 @@ class SgeExecutor extends AbstractGridExecutor { if( entry.toString().isLong() ) return entry + if( (id=entry.tokenize('.').get(0)).isLong() ) + return id + if( entry.startsWith('Your job') && entry.endsWith('has been submitted') && (id=entry.tokenize().get(2)) ) return id + + if( entry.startsWith('Your job array') && entry.endsWith('has been submitted') && (id=entry.tokenize().get(3)) ) + return id.tokenize('.').get(0) } throw new IllegalStateException("Invalid SGE submit response:\n$text\n\n") From 1929773d69a1c6b8957f5a9956ae4d697689dc30 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 24 Jan 2024 10:17:34 -0600 Subject: [PATCH 53/97] Refactor "array job" -> "job array" Signed-off-by: Ben Sherman --- docs/process.md | 16 ++++++++-------- .../executor/AbstractGridExecutor.groovy | 2 +- .../nextflow/executor/TaskArrayAware.groovy | 6 +++--- .../nextflow/processor/TaskArrayCollector.groovy | 14 +++++++------- .../processor/TaskArrayCollectorTest.groovy | 8 ++++---- .../processor/TaskPollingMonitorTest.groovy | 2 +- .../cloud/aws/batch/AwsBatchExecutor.groovy | 2 +- .../cloud/aws/batch/AwsBatchTaskHandler.groovy | 2 +- 8 files changed, 26 insertions(+), 26 deletions(-) diff --git a/docs/process.md b/docs/process.md index 78a286edc4..d53081c748 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1353,17 +1353,17 @@ Examples of values for the architecture `target` option are `cascadelake`, `icel ## array -:::{versionadded} 23.07.0-edge +:::{versionadded} 24.02.0-edge ::: :::{warning} *Experimental: may change in a future release.* ::: -The `array` directive allows you to submit tasks as *array jobs* for executors that support it. +The `array` directive allows you to submit tasks as *job arrays* for executors that support it. -An array job is a collection of jobs with the same resource requirements and the same script (parameterized by an index). Array jobs incur significantly less scheduling overhead compared to individual jobs, and as a result they are preferred by HPC schedulers where possible. +A job array is a collection of jobs with the same resource requirements and the same script (parameterized by an index). Job arrays incur significantly less scheduling overhead compared to individual jobs, and as a result they are preferred by HPC schedulers where possible. -The directive should be specified with a given array size, along with an executor that supports array jobs. For example: +The directive should be specified with a given array size, along with an executor that supports job arrays. For example: ```groovy process cpu_task { @@ -1376,7 +1376,7 @@ process cpu_task { } ``` -Nextflow currently supports array jobs for the following executors: +Nextflow currently supports job arrays for the following executors: - {ref}`awsbatch-executor` - {ref}`google-batch-executor` @@ -1386,11 +1386,11 @@ Nextflow currently supports array jobs for the following executors: - {ref}`sge-executor` - {ref}`slurm-executor` -A process using array jobs will collect tasks and submit each batch as an array job when it is ready. Any "leftover" tasks will be submitted as a partial array job. +A process using job arrays will collect tasks and submit each batch as a job array when it is ready. Any "leftover" tasks will be submitted as a partial job array. -Once an array job is submitted, each "child" task is executed as an independent job. Any tasks that fail (and can be retried) will be retried without interfering with the tasks that succeeded. Retried tasks are executed directly rather than through an array job, in order to allow for the use of [dynamic resources](#dynamic-computing-resources). +Once a job array is submitted, each "child" task is executed as an independent job. Any tasks that fail (and can be retried) will be retried without interfering with the tasks that succeeded. Retried tasks are submitted individually rather than through a job array, in order to allow for the use of [dynamic resources](#dynamic-computing-resources). -The following directives must be uniform across all tasks in a process that uses array jobs, because these directives are specified once for the entire array job: +The following directives must be uniform across all tasks in a process that uses job arrays, because these directives are specified once for the entire job array: - {ref}`process-accelerator` - {ref}`process-clusterOptions` diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index e426e425e8..32ac9f11d9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -409,7 +409,7 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { } String getArrayTaskId(String jobId, int index) { - throw new UnsupportedOperationException("Executor '${name}' does not support array jobs") + throw new UnsupportedOperationException("Executor '${name}' does not support job arrays") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy index 90ac81f2be..100358f909 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy @@ -22,7 +22,7 @@ import nextflow.processor.TaskHandler import nextflow.processor.TaskRun /** - * Interface for executors that support array jobs. + * Interface for executors that support job arrays. * * @author Ben Sherman */ @@ -42,11 +42,11 @@ interface TaskArrayAware { * Get the environment variable name that provides the array index of a task. */ default String getArrayIndexName() { - throw new UnsupportedOperationException("Executor '${getName()}' does not support array jobs") + throw new UnsupportedOperationException("Executor '${getName()}' does not support job arrays") } /** - * Get the start of the array job index range. + * Get the start of the job array index range. */ default int getArrayIndexStart() { return 0 diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index b5cd597c36..8f7bc0c16f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -29,7 +29,7 @@ import nextflow.util.CacheHelper import nextflow.util.Escape /** - * Task monitor that batches tasks and submits them as array jobs + * Task monitor that batches tasks and submits them as job arrays * to an underlying task monitor. * * @author Ben Sherman @@ -50,7 +50,7 @@ class TaskArrayCollector { TaskArrayCollector(Executor executor, int arraySize) { if( executor !instanceof TaskArrayAware ) - throw new IllegalArgumentException("Executor '${executor.name}' does not support array jobs") + throw new IllegalArgumentException("Executor '${executor.name}' does not support job arrays") this.executor = (TaskArrayAware)executor this.arraySize = arraySize @@ -78,7 +78,7 @@ class TaskArrayCollector { // add task to the array array << handler - // submit array job when it is ready + // submit job array when it is ready if( array.size() == arraySize ) { submit0(array) array = new ArrayList<>(arraySize) @@ -87,7 +87,7 @@ class TaskArrayCollector { } /** - * Close the collector, submitting any remaining tasks as a partial array job. + * Close the collector, submitting any remaining tasks as a partial job array. */ void close() { sync.withLock { @@ -103,12 +103,12 @@ class TaskArrayCollector { for( TaskHandler handler : array ) handler.prepareLauncher() - // submit array job + // submit job array executor.submit(createTaskArray(array)) } /** - * Create the task run for an array job. + * Create the task run for a job array. * * @param array */ @@ -141,7 +141,7 @@ class TaskArrayCollector { } /** - * Create the wrapper script for an array job. + * Create the wrapper script for a job array. * * @param array */ diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 1238b4d71d..c526921cbd 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -37,7 +37,7 @@ class TaskArrayCollectorTest extends Specification { TaskHandler createTaskHandler(TaskRun task) { null } } - def 'should submit tasks as array jobs' () { + def 'should submit tasks as job arrays' () { given: def executor = Mock(DummyExecutor) def handler = Mock(TaskHandler) @@ -52,7 +52,7 @@ class TaskArrayCollectorTest extends Specification { } } - // collect tasks into array job + // collect tasks into job array when: collector.collect(task) collector.collect(task) @@ -62,7 +62,7 @@ class TaskArrayCollectorTest extends Specification { 4 * executor.createTaskHandler(task) >> handler 0 * executor.submit(_) - // submit array job when it is ready + // submit job array when it is ready when: collector.collect(task) then: @@ -70,7 +70,7 @@ class TaskArrayCollectorTest extends Specification { 5 * handler.prepareLauncher() 1 * executor.submit(taskArray) - // submit partial array job when closed + // submit partial job array when closed when: collector.collect(task) collector.close() diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy index f9f350218a..8646b7f2a4 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy @@ -124,7 +124,7 @@ class TaskPollingMonitorTest extends Specification { 0 * session.notifyTaskComplete(handler) >> null } - def 'should submit an array job' () { + def 'should submit a job array' () { given: def session = Mock(Session) def monitor = Spy(new TaskPollingMonitor(name: 'foo', session: session, pollInterval: Duration.of('1min'))) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 101102fbd9..916c915f03 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -268,7 +268,7 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar ThrottlingExecutor getReaper() { reaper } void killTask(String jobId) { - // extract array job id + // extract job array id if( jobId.contains(':') ) jobId = jobId.split(':')[0] diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index bdb3578208..5aaf601f02 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -793,7 +793,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler 10_000 ) - throw new IllegalArgumentException("Array jobs on AWS Batch may not have more than 10,000 tasks") + throw new IllegalArgumentException("Job arrays on AWS Batch may not have more than 10,000 tasks") result.setArrayProperties(new ArrayProperties().withSize(arraySize)) } From 50c8f6f83859bac7d275195f5203605f4d84a78d Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Sun, 21 Apr 2024 16:50:14 +0200 Subject: [PATCH 54/97] Fix intellij compiler errors [ci fast] Signed-off-by: Paolo Di Tommaso --- .../processor/TaskPollingMonitor.groovy | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 132efb53f6..b5dd1e53f8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -197,9 +197,10 @@ class TaskPollingMonitor implements TaskMonitor { handler.prepareLauncher() handler.submit() // add each child task to the running queue - for( TaskHandler h0 : handler.task.children ) { - runningQueue.add(h0) - session.notifyTaskSubmit(h0) + final task = handler.task as TaskArray + for( TaskHandler it : task.children ) { + runningQueue.add(it) + session.notifyTaskSubmit(it) } } else { @@ -702,19 +703,25 @@ class TaskPollingMonitor implements TaskMonitor { } protected void notifyTaskPending0(TaskHandler handler) { - if( handler.task instanceof TaskArray ) - for( TaskHandler h0 : handler.task.children ) - session.notifyTaskPending(h0) - else + if( handler.task instanceof TaskArray ) { + final task = handler.task as TaskArray + for( TaskHandler it : task.children ) + session.notifyTaskPending(it) + } + else { session.notifyTaskPending(handler) + } } protected void notifyTaskComplete0(TaskHandler handler) { - if( handler.task instanceof TaskArray ) - for( TaskHandler h0 : handler.task.children ) - session.notifyTaskComplete(h0) - else + if( handler.task instanceof TaskArray ) { + final task = handler.task as TaskArray + for( TaskHandler it : task.children ) + session.notifyTaskComplete(it) + } + else { session.notifyTaskComplete(handler) + } } } From c5a12eceaa07b1999b77ea259e067f8b22d97d54 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 22 Apr 2024 13:00:09 -0500 Subject: [PATCH 55/97] Apply suggestions from review Signed-off-by: Ben Sherman --- docs/process.md | 3 ++- .../executor/AbstractGridExecutor.groovy | 7 +------ .../groovy/nextflow/executor/CrgExecutor.groovy | 4 ++-- .../nextflow/executor/GridTaskHandler.groovy | 6 +++--- .../groovy/nextflow/executor/LsfExecutor.groovy | 6 +++--- .../groovy/nextflow/executor/PbsExecutor.groovy | 9 ++++++--- .../nextflow/executor/PbsProExecutor.groovy | 4 ++-- .../groovy/nextflow/executor/SgeExecutor.groovy | 6 +++--- .../nextflow/executor/SlurmExecutor.groovy | 9 ++++++--- ...rayAware.groovy => TaskArrayExecutor.groovy} | 16 +++++++++------- .../processor/TaskArrayCollector.groovy | 10 +++++----- .../{TaskArray.groovy => TaskArrayRun.groovy} | 6 +++--- .../processor/TaskPollingMonitor.groovy | 12 ++++++------ .../processor/TaskArrayCollectorTest.groovy | 6 +++--- .../processor/TaskPollingMonitorTest.groovy | 2 +- .../cloud/aws/batch/AwsBatchExecutor.groovy | 17 +++++++---------- .../cloud/aws/batch/AwsBatchTaskHandler.groovy | 8 ++++---- .../google/batch/GoogleBatchExecutor.groovy | 10 ++++++++-- .../google/batch/GoogleBatchTaskHandler.groovy | 10 +++++----- 19 files changed, 79 insertions(+), 72 deletions(-) rename modules/nextflow/src/main/groovy/nextflow/executor/{TaskArrayAware.groovy => TaskArrayExecutor.groovy} (82%) rename modules/nextflow/src/main/groovy/nextflow/processor/{TaskArray.groovy => TaskArrayRun.groovy} (90%) diff --git a/docs/process.md b/docs/process.md index 0e4124e178..f7a0379289 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1335,7 +1335,7 @@ Examples of values for the architecture `target` option are `cascadelake`, `icel ## array -:::{versionadded} 24.02.0-edge +:::{versionadded} 24.04.0 ::: :::{warning} *Experimental: may change in a future release.* @@ -1379,6 +1379,7 @@ The following directives must be uniform across all tasks in a process that uses - {ref}`process-cpus` - {ref}`process-disk` - {ref}`process-machineType` +- {ref}`process-maxsubmitawait` - {ref}`process-memory` - {ref}`process-queue` - {ref}`process-resourcelabels` diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy index 4b47f1ba30..2502818366 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/AbstractGridExecutor.groovy @@ -36,7 +36,7 @@ import org.apache.commons.lang.StringUtils */ @Slf4j @CompileStatic -abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { +abstract class AbstractGridExecutor extends Executor { protected Duration queueInterval @@ -407,10 +407,5 @@ abstract class AbstractGridExecutor extends Executor implements TaskArrayAware { // Instead, it is the command wrapper script that is launched run within a container process. return isFusionEnabled() } - - String getArrayTaskId(String jobId, int index) { - throw new UnsupportedOperationException("Executor '${name}' does not support job arrays") - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy index d23a67dd45..8a395b297c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy @@ -18,7 +18,7 @@ package nextflow.executor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskRun /** * An executor specialised for CRG cluster @@ -42,7 +42,7 @@ class CrgExecutor extends SgeExecutor { task.config.penv = 'smp' } - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '-t' << "1-${arraySize}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index c9621316fa..5bc3635951 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -38,7 +38,7 @@ import nextflow.exception.ProcessNonZeroExitStatusException import nextflow.file.FileHelper import nextflow.fusion.FusionAwareTask import nextflow.fusion.FusionHelper -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.trace.TraceRecord @@ -306,9 +306,9 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { } void onSubmit(String jobId) { - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { task.children.eachWithIndex { handler, i -> - final arrayTaskId = executor.getArrayTaskId(jobId, i) + final arrayTaskId = ((TaskArrayExecutor)executor).getArrayTaskId(jobId, i) ((GridTaskHandler)handler).onSubmit(arrayTaskId) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index dc36d3be7a..b25f59d7b8 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -23,7 +23,7 @@ import java.util.regex.Pattern import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.fusion.FusionHelper -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskRun /** * Processor for LSF resource manager @@ -37,7 +37,7 @@ import nextflow.processor.TaskRun */ @Slf4j @CompileStatic -class LsfExecutor extends AbstractGridExecutor { +class LsfExecutor extends AbstractGridExecutor implements TaskArrayExecutor { static private Pattern KEY_REGEX = ~/^[A-Z_0-9]+=.*/ @@ -105,7 +105,7 @@ class LsfExecutor extends AbstractGridExecutor { } // -- the job name - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '-J' << "${getJobNameFor(task)}[1-${arraySize}]".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index d5ac6e314d..8589d8b65a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -21,7 +21,7 @@ import java.util.regex.Pattern import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskRun /** * Implements a executor for PBS/Torque cluster @@ -30,7 +30,7 @@ import nextflow.processor.TaskRun */ @Slf4j @CompileStatic -class PbsExecutor extends AbstractGridExecutor { +class PbsExecutor extends AbstractGridExecutor implements TaskArrayExecutor { private static Pattern OPTS_REGEX = ~/(?:^|\s)-l.+/ @@ -44,7 +44,7 @@ class PbsExecutor extends AbstractGridExecutor { protected List getDirectives( TaskRun task, List result ) { assert result !=null - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '-J' << "0-${arraySize - 1}".toString() } @@ -184,6 +184,9 @@ class PbsExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'PBS_ARRAY_INDEX' } + @Override + int getArrayIndexStart() { 0 } + @Override String getArrayTaskId(String jobId, int index) { jobId.replace('[]', "[$index]") diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 5c7d3e78bb..5d044ceaea 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -18,7 +18,7 @@ package nextflow.executor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskRun /** * Implements a executor for PBSPro cluster executor @@ -46,7 +46,7 @@ class PbsProExecutor extends PbsExecutor { protected List getDirectives(TaskRun task, List result ) { assert result !=null - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '-J' << "0-${arraySize - 1}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 3ae4ea92f7..e456cd32e6 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -19,7 +19,7 @@ import java.nio.file.Path import groovy.transform.CompileStatic import nextflow.fusion.FusionHelper -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskRun /** * Execute a task script by running it on the SGE/OGE cluster @@ -27,7 +27,7 @@ import nextflow.processor.TaskRun * @author Paolo Di Tommaso */ @CompileStatic -class SgeExecutor extends AbstractGridExecutor { +class SgeExecutor extends AbstractGridExecutor implements TaskArrayExecutor { /** * Gets the directives to submit the specified task to the cluster for execution @@ -38,7 +38,7 @@ class SgeExecutor extends AbstractGridExecutor { */ protected List getDirectives(TaskRun task, List result) { - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '-t' << "1-${arraySize}".toString() } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index ec6185d619..beb7777009 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -22,7 +22,7 @@ import java.util.regex.Pattern import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.fusion.FusionHelper -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskRun /** @@ -35,7 +35,7 @@ import nextflow.processor.TaskRun */ @Slf4j @CompileStatic -class SlurmExecutor extends AbstractGridExecutor { +class SlurmExecutor extends AbstractGridExecutor implements TaskArrayExecutor { static private Pattern SUBMIT_REGEX = ~/Submitted batch job (\d+)/ @@ -56,7 +56,7 @@ class SlurmExecutor extends AbstractGridExecutor { */ protected List getDirectives(TaskRun task, List result) { - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() result << '--array' << "0-${arraySize - 1}".toString() } @@ -222,6 +222,9 @@ class SlurmExecutor extends AbstractGridExecutor { @Override String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } + @Override + int getArrayIndexStart() { 0 } + @Override String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy similarity index 82% rename from modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy rename to modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy index 100358f909..cbd06fa71f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayAware.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy @@ -26,7 +26,7 @@ import nextflow.processor.TaskRun * * @author Ben Sherman */ -interface TaskArrayAware { +interface TaskArrayExecutor { String getName() @@ -41,15 +41,17 @@ interface TaskArrayAware { /** * Get the environment variable name that provides the array index of a task. */ - default String getArrayIndexName() { - throw new UnsupportedOperationException("Executor '${getName()}' does not support job arrays") - } + String getArrayIndexName() /** * Get the start of the job array index range. */ - default int getArrayIndexStart() { - return 0 - } + int getArrayIndexStart() + + /** + * Get the name of a child job based on the array job name + * and child index. + */ + String getArrayTaskId(String jobId, int index) } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 8f7bc0c16f..93bad8dad4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -23,7 +23,7 @@ import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.executor.Executor -import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArrayExecutor import nextflow.file.FileHelper import nextflow.util.CacheHelper import nextflow.util.Escape @@ -38,7 +38,7 @@ import nextflow.util.Escape @CompileStatic class TaskArrayCollector { - private TaskArrayAware executor + private TaskArrayExecutor executor private int arraySize @@ -49,10 +49,10 @@ class TaskArrayCollector { private boolean closed = false TaskArrayCollector(Executor executor, int arraySize) { - if( executor !instanceof TaskArrayAware ) + if( executor !instanceof TaskArrayExecutor ) throw new IllegalArgumentException("Executor '${executor.name}' does not support job arrays") - this.executor = (TaskArrayAware)executor + this.executor = (TaskArrayExecutor)executor this.arraySize = arraySize this.array = new ArrayList<>(arraySize) } @@ -126,7 +126,7 @@ class TaskArrayCollector { final script = createTaskArrayScript(array) // create task handler - return new TaskArray( + return new TaskArrayRun( id: first.id, index: first.index, processor: first.processor, diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayRun.groovy similarity index 90% rename from modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy rename to modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayRun.groovy index 6a2fa0a6d4..6d03af83f0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArray.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayRun.groovy @@ -19,7 +19,7 @@ package nextflow.processor import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.container.ContainerConfig -import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArrayExecutor /** * Models a task array, which submits a collection of independent @@ -29,7 +29,7 @@ import nextflow.executor.TaskArrayAware */ @Slf4j @CompileStatic -class TaskArray extends TaskRun { +class TaskArrayRun extends TaskRun { List children @@ -41,7 +41,7 @@ class TaskArray extends TaskRun { ContainerConfig getContainerConfig() { final config = super.getContainerConfig() final envWhitelist = config.getEnvWhitelist() ?: [] - final executor = (TaskArrayAware)processor.getExecutor() + final executor = (TaskArrayExecutor)processor.getExecutor() envWhitelist << executor.getArrayIndexName() config.put('envWhitelist', envWhitelist) return config diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index b5dd1e53f8..7dd14daa97 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -192,12 +192,12 @@ class TaskPollingMonitor implements TaskMonitor { * A {@link TaskHandler} instance representing the task to be submitted for execution */ protected void submit(TaskHandler handler) { - if( handler.task instanceof TaskArray ) { + if( handler.task instanceof TaskArrayRun ) { // submit task array handler.prepareLauncher() handler.submit() // add each child task to the running queue - final task = handler.task as TaskArray + final task = handler.task as TaskArrayRun for( TaskHandler it : task.children ) { runningQueue.add(it) session.notifyTaskSubmit(it) @@ -703,8 +703,8 @@ class TaskPollingMonitor implements TaskMonitor { } protected void notifyTaskPending0(TaskHandler handler) { - if( handler.task instanceof TaskArray ) { - final task = handler.task as TaskArray + if( handler.task instanceof TaskArrayRun ) { + final task = handler.task as TaskArrayRun for( TaskHandler it : task.children ) session.notifyTaskPending(it) } @@ -714,8 +714,8 @@ class TaskPollingMonitor implements TaskMonitor { } protected void notifyTaskComplete0(TaskHandler handler) { - if( handler.task instanceof TaskArray ) { - final task = handler.task as TaskArray + if( handler.task instanceof TaskArrayRun ) { + final task = handler.task as TaskArrayRun for( TaskHandler it : task.children ) session.notifyTaskComplete(it) } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index c526921cbd..1967621a48 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -21,7 +21,7 @@ import java.nio.file.Paths import com.google.common.hash.HashCode import nextflow.Session import nextflow.executor.Executor -import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArrayExecutor import nextflow.script.BodyDef import nextflow.script.ProcessConfig import spock.lang.Specification @@ -32,7 +32,7 @@ import test.TestHelper */ class TaskArrayCollectorTest extends Specification { - static class DummyExecutor extends Executor implements TaskArrayAware { + static class DummyExecutor extends Executor implements TaskArrayExecutor { TaskMonitor createTaskMonitor() { null } TaskHandler createTaskHandler(TaskRun task) { null } } @@ -41,7 +41,7 @@ class TaskArrayCollectorTest extends Specification { given: def executor = Mock(DummyExecutor) def handler = Mock(TaskHandler) - def taskArray = [:] as TaskArray + def taskArray = [:] as TaskArrayRun def collector = Spy(new TaskArrayCollector(executor, 5)) { createTaskArray(_) >> taskArray } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy index 34d5da314a..23f5098131 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskPollingMonitorTest.groovy @@ -133,7 +133,7 @@ class TaskPollingMonitorTest extends Specification { getTask() >> Mock(TaskRun) } def arrayHandler = Mock(TaskHandler) { - getTask() >> Mock(TaskArray) { + getTask() >> Mock(TaskArrayRun) { children >> (1..3).collect( i -> handler ) } } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 14cbfcbb0c..13631b7b9e 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -34,7 +34,7 @@ import nextflow.cloud.aws.config.AwsConfig import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException import nextflow.executor.Executor -import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArrayExecutor import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx import nextflow.processor.ParallelPollingMonitor @@ -56,7 +56,7 @@ import org.pf4j.ExtensionPoint @Slf4j @ServiceName('awsbatch') @CompileStatic -class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAware { +class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExecutor { /** * Proxy to throttle AWS batch client requests @@ -337,13 +337,10 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAwar @Override String getArrayIndexName() { 'AWS_BATCH_JOB_ARRAY_INDEX' } -} - - - - - - - + @Override + int getArrayIndexStart() { 0 } + @Override + String getArrayTaskId(String jobId, int index) { "${jobId}:${index}" } +} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index dcee4c2bfd..b07c9ff3fc 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -68,7 +68,7 @@ import nextflow.executor.BashWrapperBuilder import nextflow.fusion.FusionAwareTask import nextflow.processor.BatchContext import nextflow.processor.BatchHandler -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import nextflow.processor.TaskStatus @@ -327,9 +327,9 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler - final arrayTaskId = "${jobId}:${i}" + final arrayTaskId = executor.getArrayTaskId(jobId, i) ((AwsBatchTaskHandler)handler).onSubmit(arrayTaskId, queueName) } } @@ -789,7 +789,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler 10_000 ) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy index c2f2650696..11a5cd9586 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy @@ -27,7 +27,7 @@ import nextflow.cloud.google.batch.client.BatchClient import nextflow.cloud.google.batch.logging.BatchLogging import nextflow.exception.AbortOperationException import nextflow.executor.Executor -import nextflow.executor.TaskArrayAware +import nextflow.executor.TaskArrayExecutor import nextflow.extension.FilesEx import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler @@ -45,7 +45,7 @@ import org.pf4j.ExtensionPoint @Slf4j @ServiceName(value='google-batch') @CompileStatic -class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayAware { +class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExecutor { private BatchClient client private BatchConfig config @@ -146,4 +146,10 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayA @Override String getArrayIndexName() { 'BATCH_TASK_INDEX' } + + @Override + int getArrayIndexStart() { 0 } + + @Override + String getArrayTaskId(String jobId, int index) { index.toString() } } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 259f5b8ce8..140d824261 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -43,7 +43,7 @@ import nextflow.executor.res.DiskResource import nextflow.fusion.FusionAwareTask import nextflow.fusion.FusionHelper import nextflow.fusion.FusionScriptLauncher -import nextflow.processor.TaskArray +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor @@ -185,9 +185,9 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { } void onSubmit(String jobId, String taskId, String uid) { - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { task.children.eachWithIndex { handler, i -> - final arrayTaskId = i.toString() + final arrayTaskId = executor.getArrayTaskId(jobId, i) ((GoogleBatchTaskHandler)handler).onSubmit(jobId, arrayTaskId, uid) } } @@ -254,7 +254,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { container.setOptions( containerOptions ) // add child container mounts if task is an array - if( task instanceof TaskArray ) + if( task instanceof TaskArrayRun ) for( TaskHandler handler : task.children ) container.addAllVolumes( ((GoogleBatchTaskHandler)handler).getContainerMounts() ) @@ -431,7 +431,7 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { final taskGroup = TaskGroup.newBuilder() .setTaskSpec(taskSpec) - if( task instanceof TaskArray ) { + if( task instanceof TaskArrayRun ) { final arraySize = task.getArraySize() taskGroup.setTaskCount(arraySize) } From 9897a6c03f5db5bd79b38fdfdad1deb0a856b6e7 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 23 Apr 2024 09:53:57 -0500 Subject: [PATCH 56/97] Fix failing tests Signed-off-by: Ben Sherman --- .../groovy/nextflow/processor/TaskArrayCollectorTest.groovy | 4 ++++ .../cloud/google/batch/logging/BatchLoggingTest.groovy | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 1967621a48..236b7c7051 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -35,6 +35,10 @@ class TaskArrayCollectorTest extends Specification { static class DummyExecutor extends Executor implements TaskArrayExecutor { TaskMonitor createTaskMonitor() { null } TaskHandler createTaskHandler(TaskRun task) { null } + + String getArrayIndexName() { null } + int getArrayIndexStart() { 0 } + String getArrayTaskId(String jobId, int index) { null } } def 'should submit tasks as job arrays' () { diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy index 9ef150e525..a4a4478172 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy @@ -130,7 +130,10 @@ class BatchLoggingTest extends Specification { when: def state=null do { - state = batchClient.getTaskState(jobId, '0') + if( batchClient.listTasks(jobId).iterator().hasNext() ) + state = batchClient.getTaskState(jobId, '0') + else + state = 'PENDING' log.debug "Test task state=$state" sleep 10_000 } while( state !in ['SUCCEEDED', 'FAILED'] ) From c15e5ecd0e9338c597966a5478b1a3892aefec1e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 23 Apr 2024 13:33:56 -0500 Subject: [PATCH 57/97] Stability improvements Signed-off-by: Ben Sherman --- .../processor/TaskArrayCollector.groovy | 55 ++++++++++--------- .../nextflow/processor/TaskProcessor.groovy | 2 +- .../processor/TaskArrayCollectorTest.groovy | 36 ++++++------ .../aws/batch/AwsBatchTaskHandler.groovy | 1 + 4 files changed, 49 insertions(+), 45 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 93bad8dad4..7c908d6f99 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -38,20 +38,23 @@ import nextflow.util.Escape @CompileStatic class TaskArrayCollector { + private TaskProcessor processor + private TaskArrayExecutor executor private int arraySize private Lock sync = new ReentrantLock() - private List array + private List array private boolean closed = false - TaskArrayCollector(Executor executor, int arraySize) { + TaskArrayCollector(TaskProcessor processor, Executor executor, int arraySize) { if( executor !instanceof TaskArrayExecutor ) throw new IllegalArgumentException("Executor '${executor.name}' does not support job arrays") + this.processor = processor this.executor = (TaskArrayExecutor)executor this.arraySize = arraySize this.array = new ArrayList<>(arraySize) @@ -72,11 +75,8 @@ class TaskArrayCollector { return } - // create task handler - final handler = executor.createTaskHandler(task) - // add task to the array - array << handler + array << task // submit job array when it is ready if( array.size() == arraySize ) { @@ -91,30 +91,29 @@ class TaskArrayCollector { */ void close() { sync.withLock { - if( array.size() > 0 ) + if( array.size() == 1 ) + executor.submit(array.first()) + else if( array.size() > 0 ) submit0(array) closed = true } } - protected void submit0(List array) { - // prepare child job launcher scripts - for( TaskHandler handler : array ) - handler.prepareLauncher() - - // submit job array - executor.submit(createTaskArray(array)) + protected void submit0(List tasks) { + executor.submit(createTaskArray(tasks)) } /** * Create the task run for a job array. * - * @param array + * @param tasks */ - protected TaskRun createTaskArray(List array) { - final tasks = array.collect( h -> h.task ) - final first = tasks.first() + protected TaskRun createTaskArray(List tasks) { + // prepare child job launcher scripts + final handlers = tasks.collect( t -> executor.createTaskHandler(t) ) + for( TaskHandler handler : handlers ) + handler.prepareLauncher() // create work directory final hash = CacheHelper.hasher( tasks.collect( t -> t.getHash().asLong() ) ).hash() @@ -123,21 +122,25 @@ class TaskArrayCollector { Files.createDirectories(workDir) // create wrapper script - final script = createTaskArrayScript(array) + final script = createTaskArrayScript(handlers) - // create task handler - return new TaskArrayRun( + // create job array + final first = tasks.min( t -> t.index ) + final taskArray = new TaskArrayRun( id: first.id, index: first.index, - processor: first.processor, - type: first.type, - config: first.processor.config.createTaskConfig(), - context: new TaskContext(first.processor), + processor: processor, + type: processor.taskBody.type, + config: processor.config.createTaskConfig(), + context: new TaskContext(processor), hash: hash, workDir: workDir, script: script, - children: array + children: handlers ) + taskArray.config.remove('tag') + + return taskArray } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index c7c4435ed6..b35209e567 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -313,7 +313,7 @@ class TaskProcessor { this.isFair0 = config.getFair() final arraySize = config.getArray() - this.arrayCollector = arraySize > 0 ? new TaskArrayCollector(executor, arraySize) : null + this.arrayCollector = arraySize > 0 ? new TaskArrayCollector(this, executor, arraySize) : null } /** diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 236b7c7051..9c8d9574f1 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -46,7 +46,7 @@ class TaskArrayCollectorTest extends Specification { def executor = Mock(DummyExecutor) def handler = Mock(TaskHandler) def taskArray = [:] as TaskArrayRun - def collector = Spy(new TaskArrayCollector(executor, 5)) { + def collector = Spy(new TaskArrayCollector(null, executor, 5)) { createTaskArray(_) >> taskArray } and: @@ -63,24 +63,20 @@ class TaskArrayCollectorTest extends Specification { collector.collect(task) collector.collect(task) then: - 4 * executor.createTaskHandler(task) >> handler 0 * executor.submit(_) // submit job array when it is ready when: collector.collect(task) then: - 1 * executor.createTaskHandler(task) >> handler - 5 * handler.prepareLauncher() 1 * executor.submit(taskArray) // submit partial job array when closed when: collector.collect(task) + collector.collect(task) collector.close() then: - 1 * executor.createTaskHandler(task) >> handler - 1 * handler.prepareLauncher() 1 * executor.submit(taskArray) // submit tasks directly once closed @@ -93,7 +89,7 @@ class TaskArrayCollectorTest extends Specification { def 'should submit retried tasks directly' () { given: def executor = Mock(DummyExecutor) - def collector = Spy(new TaskArrayCollector(executor, 5)) + def collector = Spy(new TaskArrayCollector(null, executor, 5)) and: def task = Mock(TaskRun) { getConfig() >> Mock(TaskConfig) { @@ -109,19 +105,21 @@ class TaskArrayCollectorTest extends Specification { def 'should create task array' () { given: - def executor = Mock(DummyExecutor) { + def exec = Mock(DummyExecutor) { getWorkDir() >> TestHelper.createInMemTempDir() getArrayIndexName() >> 'ARRAY_JOB_INDEX' } - def collector = Spy(new TaskArrayCollector(executor, 5)) + def proc = Mock(TaskProcessor) { + config >> Mock(ProcessConfig) { + createTaskConfig() >> Mock(TaskConfig) + } + getExecutor() >> exec + getSession() >> Mock(Session) + getTaskBody() >> { new BodyDef(null, 'source') } + } + def collector = Spy(new TaskArrayCollector(proc, exec, 5)) and: def task = Mock(TaskRun) { - processor >> Mock(TaskProcessor) { - config >> Mock(ProcessConfig) - getExecutor() >> executor - getSession() >> Mock(Session) - getTaskBody() >> { new BodyDef(null, 'source') } - } getHash() >> HashCode.fromString('0123456789abcdef') } def handler = Mock(TaskHandler) { @@ -131,10 +129,12 @@ class TaskArrayCollectorTest extends Specification { } when: - def taskArray = collector.createTaskArray([handler, handler, handler]) + def taskArray = collector.createTaskArray([task, task, task]) then: - taskArray.config == task.config - taskArray.processor == task.processor + 3 * exec.createTaskHandler(task) >> handler + 3 * handler.prepareLauncher() + and: + taskArray.processor == proc taskArray.script == ''' array=( /work/foo /work/foo /work/foo ) export task_dir=${array[ARRAY_JOB_INDEX]} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index b07c9ff3fc..e2915a9e5b 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -66,6 +66,7 @@ import nextflow.exception.ProcessSubmitException import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder import nextflow.fusion.FusionAwareTask +import nextflow.fusion.FusionHelper import nextflow.processor.BatchContext import nextflow.processor.BatchHandler import nextflow.processor.TaskArrayRun From ba176e0f9c7a30827b7cd341c497d3179fa82acd Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 23 Apr 2024 21:23:28 -0500 Subject: [PATCH 58/97] Improve construction of job array Signed-off-by: Ben Sherman --- docs/process.md | 2 +- .../processor/TaskArrayCollector.groovy | 31 +++++++++++++++++-- .../nextflow/processor/TaskProcessor.groovy | 2 ++ .../processor/TaskArrayCollectorTest.groovy | 16 ++++++++-- 4 files changed, 45 insertions(+), 6 deletions(-) diff --git a/docs/process.md b/docs/process.md index f7a0379289..931f5e88d8 100644 --- a/docs/process.md +++ b/docs/process.md @@ -1379,10 +1379,10 @@ The following directives must be uniform across all tasks in a process that uses - {ref}`process-cpus` - {ref}`process-disk` - {ref}`process-machineType` -- {ref}`process-maxsubmitawait` - {ref}`process-memory` - {ref}`process-queue` - {ref}`process-resourcelabels` +- {ref}`process-resourcelimits` - {ref}`process-time` For cloud-based executors like AWS Batch, or when using Fusion with any executor, the following additional directives must be uniform: diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 7c908d6f99..15c1c58546 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -38,6 +38,26 @@ import nextflow.util.Escape @CompileStatic class TaskArrayCollector { + /** + * The set of directives which are used by the job array. + */ + private static final List ARRAY_DIRECTIVES = [ + 'accelerator', + 'arch', + 'clusterOptions', + 'cpus', + 'disk', + 'machineType', + 'memory', + 'queue', + 'resourceLabels', + 'resourceLimits', + 'time', + // only needed for container-native executors and/or Fusion + 'container', + 'containerOptions', + ] + private TaskProcessor processor private TaskArrayExecutor executor @@ -124,6 +144,14 @@ class TaskArrayCollector { // create wrapper script final script = createTaskArrayScript(handlers) + // create config for job array + final rawConfig = new HashMap(ARRAY_DIRECTIVES.size()) + for( final key : ARRAY_DIRECTIVES ) { + final value = processor.config.get(key) + if( value != null ) + rawConfig[key] = value + } + // create job array final first = tasks.min( t -> t.index ) final taskArray = new TaskArrayRun( @@ -131,14 +159,13 @@ class TaskArrayCollector { index: first.index, processor: processor, type: processor.taskBody.type, - config: processor.config.createTaskConfig(), + config: new TaskConfig(rawConfig), context: new TaskContext(processor), hash: hash, workDir: workDir, script: script, children: handlers ) - taskArray.config.remove('tag') return taskArray } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index b35209e567..83a7707500 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -382,6 +382,8 @@ class TaskProcessor { return provider } + boolean isSingleton() { singleton } + /** * Create a "preview" for a task run. This method is only meant for the creation of "mock" task run * to allow the access for the associated {@link TaskConfig} during a pipeline "preview" execution. diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 9c8d9574f1..da4ceb33de 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -22,6 +22,7 @@ import com.google.common.hash.HashCode import nextflow.Session import nextflow.executor.Executor import nextflow.executor.TaskArrayExecutor +import nextflow.script.BaseScript import nextflow.script.BodyDef import nextflow.script.ProcessConfig import spock.lang.Specification @@ -109,17 +110,23 @@ class TaskArrayCollectorTest extends Specification { getWorkDir() >> TestHelper.createInMemTempDir() getArrayIndexName() >> 'ARRAY_JOB_INDEX' } + def config = Spy(ProcessConfig, constructorArgs: [Mock(BaseScript), 'PROC']) { + createTaskConfig() >> Mock(TaskConfig) + get('cpus') >> 4 + get('tag') >> 'foo' + } def proc = Mock(TaskProcessor) { - config >> Mock(ProcessConfig) { - createTaskConfig() >> Mock(TaskConfig) - } + getConfig() >> config getExecutor() >> exec + getName() >> 'PROC' getSession() >> Mock(Session) + isSingleton() >> false getTaskBody() >> { new BodyDef(null, 'source') } } def collector = Spy(new TaskArrayCollector(proc, exec, 5)) and: def task = Mock(TaskRun) { + index >> 1 getHash() >> HashCode.fromString('0123456789abcdef') } def handler = Mock(TaskHandler) { @@ -134,6 +141,9 @@ class TaskArrayCollectorTest extends Specification { 3 * exec.createTaskHandler(task) >> handler 3 * handler.prepareLauncher() and: + taskArray.name == 'PROC (1)' + taskArray.config.cpus == 4 + taskArray.config.tag == null taskArray.processor == proc taskArray.script == ''' array=( /work/foo /work/foo /work/foo ) From 584c961169de869221713ddd54f3f41c99f482cc Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 24 Apr 2024 12:58:05 -0500 Subject: [PATCH 59/97] Add process directives to wrapper script for Fusion Signed-off-by: Ben Sherman --- .../executor/BashWrapperBuilder.groovy | 29 +++++++++++++++- .../groovy/nextflow/processor/TaskBean.groovy | 12 +++++++ .../nextflow/executor/command-run.txt | 1 + .../executor/BashWrapperBuilderTest.groovy | 33 +++++++++++++++++++ .../fusion/FusionHelperTest.groovy | 4 +-- .../fusion/FusionScriptLauncherTest.groovy | 3 +- .../nextflow/k8s/K8sWrapperBuilderTest.groovy | 1 + 7 files changed, 77 insertions(+), 6 deletions(-) rename modules/nextflow/src/test/groovy/nextflow/{executor => }/fusion/FusionHelperTest.groovy (96%) rename modules/nextflow/src/test/groovy/nextflow/{executor => }/fusion/FusionScriptLauncherTest.groovy (97%) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index f836dfd46a..58094650bf 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -92,7 +92,7 @@ class BashWrapperBuilder { ScriptFileCopyStrategy copyStrategy @Delegate - private TaskBean bean + protected TaskBean bean private boolean runWithContainer @@ -307,6 +307,7 @@ class BashWrapperBuilder { final binding = new HashMap(20) binding.header_script = headerScript + binding.process_directives = getProcessDirectives() binding.task_name = name binding.helpers_script = getHelpersScript() @@ -461,6 +462,32 @@ class BashWrapperBuilder { } } + protected String getProcessDirectives() { + final lines = [] + lines << '---' + + if( arrayIndexName ) { + lines << 'array:' + lines << " index-name: ${arrayIndexName}" + lines << " index-start: ${arrayIndexStart}" + } + + if( containerConfig?.isEnabled() ) + lines << "container: '${containerImage}'" + + if( outputFiles.size() > 0 ) { + lines << 'outputs:' + for( final output : outputFiles ) + lines << "- '${output}'" + } + + if( lines.size() == 1 ) + return '' + + lines << '...' + return lines.collect( line -> '### ' + line ).join('\n') + } + protected String getHelpersScript() { def result = new StringBuilder() diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy index c1ef240388..eff207c5a5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy @@ -22,6 +22,7 @@ import groovy.transform.CompileStatic import groovy.transform.PackageScope import nextflow.container.ContainerConfig import nextflow.executor.BashWrapperBuilder +import nextflow.executor.TaskArrayExecutor import nextflow.util.MemoryUnit /** @@ -103,6 +104,10 @@ class TaskBean implements Serializable, Cloneable { Map resourceLabels + String arrayIndexName + + Integer arrayIndexStart + @PackageScope TaskBean() { shell = BashWrapperBuilder.BASH @@ -156,6 +161,13 @@ class TaskBean implements Serializable, Cloneable { this.stageOutMode = task.config.getStageOutMode() this.resourceLabels = task.config.getResourceLabels() + + // job array + if( task instanceof TaskArrayRun ) { + final executor = (TaskArrayExecutor)task.getProcessor().getExecutor() + this.arrayIndexName = executor.getArrayIndexName() + this.arrayIndexStart = executor.getArrayIndexStart() + } } @Override diff --git a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt index c63e91f702..77aede046a 100644 --- a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt +++ b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt @@ -14,6 +14,7 @@ ## limitations under the License. #!/bin/bash {{header_script}} +{{process_directives}} # NEXTFLOW TASK: {{task_name}} set -e set -u diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy index f60f555cdb..337e2594a8 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy @@ -51,6 +51,8 @@ class BashWrapperBuilderTest extends Specification { bean.script = 'echo Hello world!' if( !bean.containsKey('inputFiles') ) bean.inputFiles = [:] + if( !bean.containsKey('outputFiles') ) + bean.outputFiles = [] new BashWrapperBuilder(bean as TaskBean) { @Override protected String getSecretsEnv() { @@ -335,6 +337,36 @@ class BashWrapperBuilderTest extends Specification { } + def 'should add process directives' () { + when: + def bash = newBashWrapperBuilder() + then: + bash.makeBinding().containsKey('process_directives') + bash.makeBinding().process_directives == '' + + when: + bash = newBashWrapperBuilder( + arrayIndexName: 'SLURM_ARRAY_TASK_ID', + arrayIndexStart: 0, + containerConfig: [enabled: true], + containerImage: 'quay.io/nextflow:bash', + outputFiles: ['foo.txt', '*.bar', '**/baz'] + ) + then: + bash.makeBinding().process_directives == '''\ + ### --- + ### array: + ### index-name: SLURM_ARRAY_TASK_ID + ### index-start: 0 + ### container: 'quay.io/nextflow:bash' + ### outputs: + ### - 'foo.txt' + ### - '*.bar' + ### - '**/baz' + ### ... + '''.stripIndent().rightTrim() + } + def 'should copy control files' () { when: @@ -1083,6 +1115,7 @@ class BashWrapperBuilderTest extends Specification { given: def bean = Mock(TaskBean) { inputFiles >> [:] + outputFiles >> [] } def copy = Mock(ScriptFileCopyStrategy) bean.workDir >> Paths.get('/work/dir') diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionHelperTest.groovy b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionHelperTest.groovy similarity index 96% rename from modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionHelperTest.groovy rename to modules/nextflow/src/test/groovy/nextflow/fusion/FusionHelperTest.groovy index 697cdd5238..e8dc875b10 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionHelperTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionHelperTest.groovy @@ -15,14 +15,12 @@ * */ -package nextflow.executor.fusion +package nextflow.fusion import java.nio.file.Path import nextflow.container.ContainerConfig import nextflow.file.http.XPath -import nextflow.fusion.FusionHelper -import nextflow.fusion.FusionScriptLauncher import spock.lang.Specification /** diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionScriptLauncherTest.groovy b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionScriptLauncherTest.groovy similarity index 97% rename from modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionScriptLauncherTest.groovy rename to modules/nextflow/src/test/groovy/nextflow/fusion/FusionScriptLauncherTest.groovy index 60ff2be1e4..c0be12e632 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/fusion/FusionScriptLauncherTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionScriptLauncherTest.groovy @@ -15,14 +15,13 @@ * */ -package nextflow.executor.fusion +package nextflow.fusion import java.nio.file.Path import nextflow.Global import nextflow.Session import nextflow.file.http.XPath -import nextflow.fusion.FusionScriptLauncher import nextflow.processor.TaskBean import spock.lang.Specification /** diff --git a/modules/nextflow/src/test/groovy/nextflow/k8s/K8sWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/k8s/K8sWrapperBuilderTest.groovy index 9951ae2bce..f3e188cdf0 100644 --- a/modules/nextflow/src/test/groovy/nextflow/k8s/K8sWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/k8s/K8sWrapperBuilderTest.groovy @@ -45,6 +45,7 @@ class K8sWrapperBuilderTest extends Specification { getProcessor() >> proc getWorkDir() >> folder getInputFilesMap() >> [:] + getOutputFilesNames() >> [] } and: From 49f7fbd144910f04e82d04909fb6a98e01fac542 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Fri, 26 Apr 2024 10:36:26 +0200 Subject: [PATCH 60/97] Prefer try/finally idiom to avoid closure [ci fast] Signed-off-by: Paolo Di Tommaso --- .../nextflow/processor/TaskArrayCollector.groovy | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 15c1c58546..883eb3a689 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -87,7 +87,8 @@ class TaskArrayCollector { * @param task */ void collect(TaskRun task) { - sync.withLock { + sync.lock() + 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 ) { @@ -104,13 +105,17 @@ class TaskArrayCollector { array = new ArrayList<>(arraySize) } } + finally { + sync.unlock() + } } /** * Close the collector, submitting any remaining tasks as a partial job array. */ void close() { - sync.withLock { + sync.lock() + try { if( array.size() == 1 ) executor.submit(array.first()) else if( array.size() > 0 ) @@ -118,6 +123,9 @@ class TaskArrayCollector { closed = true } + finally { + sync.unlock() + } } protected void submit0(List tasks) { From 99a1ec3bb92a2c6e37e18b507b79059dd1bd9aed Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 26 Apr 2024 11:11:22 -0500 Subject: [PATCH 61/97] Apply suggestions from review Signed-off-by: Ben Sherman --- .../nextflow/executor/BashWrapperBuilder.groovy | 7 +++++-- .../nextflow/processor/TaskArrayCollector.groovy | 4 ++-- .../main/groovy/nextflow/processor/TaskBean.groovy | 3 +++ .../resources/nextflow/executor/command-run.txt | 2 +- .../nextflow/executor/BashWrapperBuilderTest.groovy | 13 +++++++++---- .../processor/TaskArrayCollectorTest.groovy | 4 ++-- 6 files changed, 22 insertions(+), 11 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index 58094650bf..acbb1420c2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -307,7 +307,7 @@ class BashWrapperBuilder { final binding = new HashMap(20) binding.header_script = headerScript - binding.process_directives = getProcessDirectives() + binding.task_metadata = getTaskMetadata() binding.task_name = name binding.helpers_script = getHelpersScript() @@ -462,7 +462,7 @@ class BashWrapperBuilder { } } - protected String getProcessDirectives() { + protected String getTaskMetadata() { final lines = [] lines << '---' @@ -470,6 +470,9 @@ class BashWrapperBuilder { lines << 'array:' lines << " index-name: ${arrayIndexName}" lines << " index-start: ${arrayIndexStart}" + lines << " work-dirs:" + for( String workDir : arrayWorkDirs ) + lines << " - ${Escape.path(workDir)}" } if( containerConfig?.isEnabled() ) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 883eb3a689..6e4bc8fbc3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -187,7 +187,7 @@ class TaskArrayCollector { // get work directory and launch command for each task final workDirs = array.collect( h -> h.getWorkDir() ) final args = array.first().getLaunchCommand().toArray() as String[] - final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${task_dir}') + final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${NXF_CHDIR}') // create wrapper script final indexName = executor.getArrayIndexName() @@ -198,7 +198,7 @@ class TaskArrayCollector { """ array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export task_dir=\${array[${arrayIndex}]} + export NXF_CHDIR=\${array[${arrayIndex}]} ${cmd} """.stripIndent().leftTrim() } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy index eff207c5a5..f8108da631 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy @@ -108,6 +108,8 @@ class TaskBean implements Serializable, Cloneable { Integer arrayIndexStart + List arrayWorkDirs + @PackageScope TaskBean() { shell = BashWrapperBuilder.BASH @@ -167,6 +169,7 @@ class TaskBean implements Serializable, Cloneable { final executor = (TaskArrayExecutor)task.getProcessor().getExecutor() this.arrayIndexName = executor.getArrayIndexName() this.arrayIndexStart = executor.getArrayIndexStart() + this.arrayWorkDirs = task.children.collect( h -> h.getWorkDir() ) } } diff --git a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt index 77aede046a..c60c83e85f 100644 --- a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt +++ b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt @@ -14,7 +14,7 @@ ## limitations under the License. #!/bin/bash {{header_script}} -{{process_directives}} +{{task_metadata}} # NEXTFLOW TASK: {{task_name}} set -e set -u diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy index 337e2594a8..e5b9297bd6 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy @@ -337,27 +337,32 @@ class BashWrapperBuilderTest extends Specification { } - def 'should add process directives' () { + def 'should add task metadata' () { when: def bash = newBashWrapperBuilder() then: - bash.makeBinding().containsKey('process_directives') - bash.makeBinding().process_directives == '' + bash.makeBinding().containsKey('task_metadata') + bash.makeBinding().task_metadata == '' when: bash = newBashWrapperBuilder( arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, + arrayWorkDirs: [ '/work/01', '/work/02', '/work/03' ], containerConfig: [enabled: true], containerImage: 'quay.io/nextflow:bash', outputFiles: ['foo.txt', '*.bar', '**/baz'] ) then: - bash.makeBinding().process_directives == '''\ + bash.makeBinding().task_metadata == '''\ ### --- ### array: ### index-name: SLURM_ARRAY_TASK_ID ### index-start: 0 + ### work-dirs: + ### - /work/01 + ### - /work/02 + ### - /work/03 ### container: 'quay.io/nextflow:bash' ### outputs: ### - 'foo.txt' diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index da4ceb33de..6ef01d1452 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -147,8 +147,8 @@ class TaskArrayCollectorTest extends Specification { taskArray.processor == proc taskArray.script == ''' array=( /work/foo /work/foo /work/foo ) - export task_dir=${array[ARRAY_JOB_INDEX]} - bash -o pipefail -c 'bash ${task_dir}/.command.run 2>&1 | tee ${task_dir}/.command.log' + export NXF_CHDIR=${array[ARRAY_JOB_INDEX]} + bash -o pipefail -c 'bash ${NXF_CHDIR}/.command.run 2>&1 | tee ${NXF_CHDIR}/.command.log' '''.stripIndent().leftTrim() and: taskArray.getArraySize() == 3 From bf04e3f6cdfbd688600c5c80d6389ad9b2bdb423 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 29 Apr 2024 16:36:41 -0500 Subject: [PATCH 62/97] Fix bug with job array context Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 6e4bc8fbc3..d4b9f4ec80 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -174,6 +174,9 @@ class TaskArrayCollector { script: script, children: handlers ) + taskArray.config.context = taskArray.context + taskArray.config.process = taskArray.processor.name + taskArray.config.executor = taskArray.processor.executor.name return taskArray } From 16dfb257156d18e752f3d91bc7ed94b8a627e622 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 1 May 2024 12:53:56 +0200 Subject: [PATCH 63/97] Add integration tests Signed-off-by: Paolo Di Tommaso --- validation/awsbatch.sh | 8 ++++++++ validation/google.sh | 7 +++++++ 2 files changed, 15 insertions(+) diff --git a/validation/awsbatch.sh b/validation/awsbatch.sh index 6b0298756b..9f65c8beeb 100644 --- a/validation/awsbatch.sh +++ b/validation/awsbatch.sh @@ -60,3 +60,11 @@ $NXF_CMD run nextflow-io/rnaseq-nf \ -profile batch \ -plugins nf-cloudcache,nf-wave \ -c awsfargate.config + +## Test use of job array +NXF_CLOUDCACHE_PATH=s3://nextflow-ci/cache \ +$NXF_CMD run nextflow-io/rnaseq-nf \ + -profile batch \ + -process.array 10 \ + -plugins nf-cloudcache + diff --git a/validation/google.sh b/validation/google.sh index d97fedd4c1..bc9fea34d2 100644 --- a/validation/google.sh +++ b/validation/google.sh @@ -72,3 +72,10 @@ $NXF_CMD -C ./google.config \ -resume [[ `grep -c 'Using Nextflow cache factory: nextflow.cache.CloudCacheFactory' .nextflow.log` == 1 ]] || false [[ `grep -c 'Cached process > ' .nextflow.log` == 4 ]] || false + +## Test job array +NXF_CLOUDCACHE_PATH=gs://rnaseq-nf/cache \ +$NXF_CMD -C ./google.config \ + run nextflow-io/rnaseq-nf \ + -process.array 10 \ + -plugins nf-cloudcache From 0e88bf7090854b89792d8c6dd50385a4f651d802 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 1 May 2024 14:38:35 +0200 Subject: [PATCH 64/97] Restore private method access (#4961) [ci fast] Signed-off-by: Paolo Di Tommaso --- .../groovy/nextflow/executor/BashWrapperBuilder.groovy | 2 +- .../src/main/groovy/nextflow/processor/TaskRun.groovy | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index acbb1420c2..745c279f71 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -92,7 +92,7 @@ class BashWrapperBuilder { ScriptFileCopyStrategy copyStrategy @Delegate - protected TaskBean bean + private TaskBean bean private boolean runWithContainer diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index 459e1ebbb2..69e217291e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -454,7 +454,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('outputFileNames', (it)-> getOutputFilesNames0()) } - protected List getOutputFilesNames0() { + private List getOutputFilesNames0() { def result = [] for( FileOutParam param : getOutputsByType(FileOutParam).keySet() ) { @@ -617,7 +617,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('condaEnv', (it)-> getCondaEnv0()) } - protected Path getCondaEnv0() { + private Path getCondaEnv0() { if( !config.conda || !processor.session.getCondaConfig().isEnabled() ) return null @@ -629,7 +629,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('spackEnv', (it)-> getSpackEnv0()) } - protected Path getSpackEnv0() { + private Path getSpackEnv0() { if( !config.spack || !processor.session.getSpackConfig().isEnabled() ) return null @@ -643,7 +643,7 @@ class TaskRun implements Cloneable { cache0.computeIfAbsent('containerInfo', (it)-> containerInfo0()) } - protected ContainerInfo containerInfo0() { + private ContainerInfo containerInfo0() { // fetch the container image from the config def configImage = config.getContainer() // the boolean `false` literal can be provided From 11aeaaea7e3b300c070a0e6e2b136951a05832f5 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 1 May 2024 14:39:13 +0200 Subject: [PATCH 65/97] Minor changes Signed-off-by: Paolo Di Tommaso --- .../processor/TaskArrayCollector.groovy | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index d4b9f4ec80..176326e850 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -97,11 +97,11 @@ class TaskArrayCollector { } // add task to the array - array << task + array.add(task) // submit job array when it is ready if( array.size() == arraySize ) { - submit0(array) + executor.submit(createTaskArray(array)) array = new ArrayList<>(arraySize) } } @@ -116,11 +116,13 @@ class TaskArrayCollector { void close() { sync.lock() try { - if( array.size() == 1 ) + if( array.size() == 1 ) { executor.submit(array.first()) - else if( array.size() > 0 ) - submit0(array) - + } + else if( array.size() > 0 ) { + executor.submit(createTaskArray(array)) + array = null + } closed = true } finally { @@ -128,10 +130,6 @@ class TaskArrayCollector { } } - protected void submit0(List tasks) { - executor.submit(createTaskArray(tasks)) - } - /** * Create the task run for a job array. * @@ -140,13 +138,13 @@ class TaskArrayCollector { protected TaskRun createTaskArray(List tasks) { // prepare child job launcher scripts final handlers = tasks.collect( t -> executor.createTaskHandler(t) ) - for( TaskHandler handler : handlers ) + for( TaskHandler handler : handlers ) { handler.prepareLauncher() + } // create work directory final hash = CacheHelper.hasher( tasks.collect( t -> t.getHash().asLong() ) ).hash() final workDir = FileHelper.getWorkFolder(executor.getWorkDir(), hash) - Files.createDirectories(workDir) // create wrapper script From 519c93a4c6105df64253158a32776dc10a76520b Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 1 May 2024 14:52:59 +0200 Subject: [PATCH 66/97] Add aws batch integration test Signed-off-by: Paolo Di Tommaso --- validation/awsbatch.sh | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/validation/awsbatch.sh b/validation/awsbatch.sh index 9f65c8beeb..cdb958c315 100644 --- a/validation/awsbatch.sh +++ b/validation/awsbatch.sh @@ -63,8 +63,7 @@ $NXF_CMD run nextflow-io/rnaseq-nf \ ## Test use of job array NXF_CLOUDCACHE_PATH=s3://nextflow-ci/cache \ -$NXF_CMD run nextflow-io/rnaseq-nf \ - -profile batch \ +$NXF_CMD run nextflow-io/hello \ -process.array 10 \ - -plugins nf-cloudcache - + -plugins nf-cloudcache \ + -c awsbatch.config From ade72e2194b58cdb023619dd8ca977268d3ad0eb Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 1 May 2024 09:25:50 -0500 Subject: [PATCH 67/97] Revert re-use of NXF_CHDIR in job array script Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 4 ++-- .../groovy/nextflow/processor/TaskArrayCollectorTest.groovy | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 176326e850..a544c69dbb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -188,7 +188,7 @@ class TaskArrayCollector { // get work directory and launch command for each task final workDirs = array.collect( h -> h.getWorkDir() ) final args = array.first().getLaunchCommand().toArray() as String[] - final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${NXF_CHDIR}') + final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${array_task_dir}') // create wrapper script final indexName = executor.getArrayIndexName() @@ -199,7 +199,7 @@ class TaskArrayCollector { """ array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export NXF_CHDIR=\${array[${arrayIndex}]} + export array_task_dir=\${array[${arrayIndex}]} ${cmd} """.stripIndent().leftTrim() } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index 6ef01d1452..c92eb50e84 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -147,8 +147,8 @@ class TaskArrayCollectorTest extends Specification { taskArray.processor == proc taskArray.script == ''' array=( /work/foo /work/foo /work/foo ) - export NXF_CHDIR=${array[ARRAY_JOB_INDEX]} - bash -o pipefail -c 'bash ${NXF_CHDIR}/.command.run 2>&1 | tee ${NXF_CHDIR}/.command.log' + export array_task_dir=${array[ARRAY_JOB_INDEX]} + bash -o pipefail -c 'bash ${array_task_dir}/.command.run 2>&1 | tee ${array_task_dir}/.command.log' '''.stripIndent().leftTrim() and: taskArray.getArraySize() == 3 From 8c13ec35ef75f305e5a1bc6a2255aff3037e8713 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 1 May 2024 10:57:57 -0500 Subject: [PATCH 68/97] Add unit tests for grid executors Signed-off-by: Ben Sherman --- .../executor/GridTaskHandlerTest.groovy | 18 +++ .../nextflow/executor/LsfExecutorTest.groovy | 29 +++-- .../nextflow/executor/PbsExecutorTest.groovy | 23 +++- .../executor/PbsProExecutorTest.groovy | 27 +++- .../nextflow/executor/SgeExecutorTest.groovy | 115 +++++++++--------- .../executor/SlurmExecutorTest.groovy | 26 +++- 6 files changed, 154 insertions(+), 84 deletions(-) diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index cace7b24fc..46035154cd 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -143,4 +143,22 @@ class GridTaskHandlerTest extends Specification { SUBMIT_COMMAND_EOF '''.stripIndent() } + + def 'should get launch command' () { + given: + def task = Mock(TaskRun) { + workDir >> Path.of('/work/dir') + } + def exec = Mock(AbstractGridExecutor) + def handler = Spy(new GridTaskHandler(task, exec)) { + fusionEnabled() >> false + } + + expect: + handler.getLaunchCommand() == [ + 'bash', + '-o', 'pipefail', + '-c', 'bash /work/dir/.command.run 2>&1 | tee /work/dir/.command.log' + ] + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy index d7fae2fa77..c92c52ccb4 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy @@ -21,6 +21,7 @@ import java.nio.file.Path import java.nio.file.Paths import nextflow.Session +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -129,21 +130,17 @@ class LsfExecutorTest extends Specification { } - def testHeaders() { + def 'test job script headers' () { setup: - // LSF executor def executor = Spy(LsfExecutor) executor.@memUnit = 'MB' executor.@usageUnit = 'MB' executor.session = new Session() - // mock process def proc = Mock(TaskProcessor) - // process name proc.getName() >> 'task' - // task object def task = new TaskRun() task.processor = proc task.workDir = Paths.get('/scratch') @@ -151,13 +148,11 @@ class LsfExecutorTest extends Specification { when: task.config = new TaskConfig() - // config task.config.queue = 'bsc_ls' task.config.clusterOptions = "-x 1 -R \"span[ptile=2]\"" task.config.cpus = '2' task.config.time = '1h 30min' task.config.memory = '8GB' - then: executor.getHeaders(task) == ''' #BSUB -o /scratch/.command.log @@ -173,7 +168,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'alpha' @@ -186,7 +180,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'alpha' @@ -204,7 +197,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'gamma' @@ -222,7 +214,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'gamma' @@ -259,7 +250,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'gamma' @@ -279,7 +269,6 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: task.config = new TaskConfig() task.config.queue = 'delta' @@ -296,6 +285,20 @@ class LsfExecutorTest extends Specification { ''' .stripIndent().leftTrim() + when: 'with job array' + def taskArray = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> task.name + workDir >> task.workDir + getArraySize() >> 5 + } + then: + executor.getHeaders(taskArray) == ''' + #BSUB -o /scratch/.command.log + #BSUB -J "nf-mapping_hola[1-5]" + ''' + .stripIndent().leftTrim() + } def testDiskResources() { diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy index 28f0904dc8..9307ea919f 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy @@ -18,6 +18,7 @@ package nextflow.executor import java.nio.file.Paths +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -41,15 +42,11 @@ class PbsExecutorTest extends Specification { } - def testHeaders() { + def 'test job script headers'() { setup: def executor = [:] as PbsExecutor - - // mock process def proc = Mock(TaskProcessor) - - // task object def task = new TaskRun() task.processor = proc task.workDir = Paths.get('/work/dir') @@ -161,6 +158,22 @@ class PbsExecutorTest extends Specification { #PBS -l nodes=1:x86:ppn=4 ''' .stripIndent().leftTrim() + + when: 'with job array' + def taskArray = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> task.name + workDir >> task.workDir + getArraySize() >> 5 + } + then: + executor.getHeaders(taskArray) == ''' + #PBS -J 0-4 + #PBS -N nf-task_name + #PBS -o /work/dir/.command.log + #PBS -j oe + ''' + .stripIndent().leftTrim() } def WorkDirWithBlanks() { diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy index dfd5a39a6c..72b946ab91 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy @@ -16,13 +16,13 @@ package nextflow.executor -import nextflow.processor.TaskProcessor -import spock.lang.Specification - import java.nio.file.Paths +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig +import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun +import spock.lang.Specification /** * * @author Lorenz Gerber @@ -174,12 +174,31 @@ class PbsProExecutorTest extends Specification { ] } + def 'should get directives with job array' () { + given: + def executor = Spy(PbsProExecutor) + def task = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> 'foo' + workDir >> Paths.get('/foo/bar') + getArraySize() >> 5 + } + + expect: + executor.getDirectives(task, []) == [ + '-J', '0-4', + '-N', 'nf-foo', + '-o', '/foo/bar/.command.log', + '-j', 'oe', + ] + } + def 'should return qstat command line' () { given: def executor = [:] as PbsProExecutor expect: - executor.queueStatusCommand(null) == ['bash','-c', "set -o pipefail; qstat -f \$( qstat -B | grep -E -v '(^Server|^---)' | awk -v ORS=' ' '{print \"@\"\$1}' ) | { grep -E '(Job Id:|job_state =)' || true; }"] + executor.queueStatusCommand(null) == ['bash','-c', "set -o pipefail; qstat -f \$( qstat -B | grep -E -v '(^Server|^---)' | awk -v ORS=' ' '{print \"@\"\$1}' ) | { grep -E '(Job Id:|job_state =)' || true; }"] executor.queueStatusCommand('xxx') == ['bash','-c', "set -o pipefail; qstat -f xxx | { grep -E '(Job Id:|job_state =)' || true; }"] executor.queueStatusCommand('xxx').each { assert it instanceof String } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy index c12897e21e..4b236ee72f 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy @@ -17,6 +17,7 @@ package nextflow.executor import java.nio.file.Paths +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -51,10 +52,7 @@ class SgeExecutorTest extends Specification { def 'test qsub headers' () { given: - def config - // mock process def proc = Mock(TaskProcessor) - def executor = [:] as SgeExecutor def task = new TaskRun() task.processor = proc @@ -62,11 +60,9 @@ class SgeExecutorTest extends Specification { task.name = 'the task name' when: - - // config - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' then: executor.getHeaders(task) == ''' @@ -80,9 +76,9 @@ class SgeExecutorTest extends Specification { .stripIndent().leftTrim() when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -94,13 +90,12 @@ class SgeExecutorTest extends Specification { ''' .stripIndent().leftTrim() - when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' - config.time = '10s ' - config.clusterOptions = '-hard -alpha -beta' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' + task.config.time = '10s ' + task.config.clusterOptions = '-hard -alpha -beta' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -114,15 +109,12 @@ class SgeExecutorTest extends Specification { ''' .stripIndent().leftTrim() - - when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' - config.time = '10m' - config.memory = '1M' - config.remove('clusterOptions') + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' + task.config.time = '10m' + task.config.memory = '1M' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -136,16 +128,14 @@ class SgeExecutorTest extends Specification { ''' .stripIndent().leftTrim() - - when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' - config.cpus = 1 - config.penv = 'smp' - config.time = '2 m' - config.memory = '2 M' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' + task.config.cpus = 1 + task.config.penv = 'smp' + task.config.time = '2 m' + task.config.memory = '2 M' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -161,13 +151,13 @@ class SgeExecutorTest extends Specification { .stripIndent().leftTrim() when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' - config.cpus = 2 - config.penv = 'mpi' - config.time = '3 d' - config.memory = '3 g' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' + task.config.cpus = 2 + task.config.penv = 'mpi' + task.config.time = '3 d' + task.config.memory = '3 g' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -183,13 +173,13 @@ class SgeExecutorTest extends Specification { .stripIndent().leftTrim() when: - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' - config.cpus = 4 - config.penv = 'orte' - config.time = '1d3h' - config.memory = '4 GB ' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' + task.config.cpus = 4 + task.config.penv = 'orte' + task.config.time = '1d3h' + task.config.memory = '4 GB ' then: executor.getHeaders(task) == ''' #$ -N nf-the_task_name @@ -204,15 +194,30 @@ class SgeExecutorTest extends Specification { ''' .stripIndent().leftTrim() + when: 'with job array' + def taskArray = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> task.name + workDir >> task.workDir + getArraySize() >> 5 + } + then: + executor.getHeaders(taskArray) == ''' + #$ -t 1-5 + #$ -N nf-the_task_name + #$ -o /abc/.command.log + #$ -j y + #$ -terse + #$ -notify + ''' + .stripIndent().leftTrim() + } def testWorkDirWithBlanks() { given: - def config - // mock process def proc = Mock(TaskProcessor) - def executor = [:] as SgeExecutor def task = new TaskRun() task.processor = proc @@ -220,11 +225,9 @@ class SgeExecutorTest extends Specification { task.name = 'the task name' when: - - // config - config = task.config = new TaskConfig() - config.queue = 'my-queue' - config.name = 'task' + task.config = new TaskConfig() + task.config.queue = 'my-queue' + task.config.name = 'task' then: executor.getHeaders(task) == ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy index c1642686f7..b7a9ca99fa 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy @@ -18,6 +18,7 @@ package nextflow.executor import java.nio.file.Paths import nextflow.Session +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -77,7 +78,7 @@ class SlurmExecutorTest extends Specification { '/some/path/job.sh' | true | ['sbatch'] } - def testGetHeaders() { + def 'test job script headers' () { setup: // SLURM executor @@ -135,7 +136,6 @@ class SlurmExecutorTest extends Specification { task.config.time = '1h' task.config.memory = '50 M' task.config.clusterOptions = '-a 1 --signal=KILL' - then: executor.getHeaders(task) == ''' #SBATCH -J nf-the_task_name @@ -153,7 +153,6 @@ class SlurmExecutorTest extends Specification { task.config.time = '2h' task.config.memory = '200 M' task.config.clusterOptions = '-b 2' - then: executor.getHeaders(task) == ''' #SBATCH -J nf-the_task_name @@ -173,7 +172,6 @@ class SlurmExecutorTest extends Specification { task.config.time = '2d 3h' task.config.memory = '3 G' task.config.clusterOptions = '-x 3' - then: executor.getHeaders(task) == ''' #SBATCH -J nf-the_task_name @@ -187,8 +185,7 @@ class SlurmExecutorTest extends Specification { ''' .stripIndent().leftTrim() - // test perCpuMemAllocation - when: + when: 'with perCpuMemAllocation' executor.@perCpuMemAllocation = true task.config = new TaskConfig() task.config.cpus = 8 @@ -203,6 +200,23 @@ class SlurmExecutorTest extends Specification { #SBATCH --mem-per-cpu 3072M ''' .stripIndent().leftTrim() + + when: 'with job array' + def taskArray = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> task.name + workDir >> task.workDir + getArraySize() >> 5 + } + then: + executor.getHeaders(taskArray) == ''' + #SBATCH --array 0-4 + #SBATCH -J nf-the_task_name + #SBATCH -o /work/path/.command.log + #SBATCH --no-requeue + #SBATCH --signal B:USR2@30 + ''' + .stripIndent().leftTrim() } def testWorkDirWithBlanks() { From b7e829dbac618cf0bbc9debae5e3096b17a77259 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 1 May 2024 11:09:33 -0500 Subject: [PATCH 69/97] Add unit test for crg executor Signed-off-by: Ben Sherman --- .../nextflow/executor/CrgExecutorTest.groovy | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy index a4f012d5a8..211201a1db 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy @@ -20,6 +20,7 @@ import java.nio.file.Paths import nextflow.Session import nextflow.container.ContainerConfig +import nextflow.processor.TaskArrayRun import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -337,6 +338,26 @@ class CrgExecutorTest extends Specification { .stripIndent().leftTrim() } + def 'should get headers with job array' () { + when: + def executor = Spy(new CrgExecutor()) + def task = Mock(TaskArrayRun) { + config >> new TaskConfig() + name >> 'mapping tag' + workDir >> Paths.get('/abc') + getArraySize() >> 5 + } + then: + executor.getHeaders(task) == ''' + #$ -t 1-5 + #$ -N nf-mapping_tag + #$ -o /abc/.command.log + #$ -j y + #$ -terse + #$ -notify + ''' + .stripIndent().leftTrim() + } def testParseJobId() { From 6f6e5a8b0d64e6ca704f45c924ddc0d54c39a02b Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 1 May 2024 22:46:44 +0200 Subject: [PATCH 70/97] Fix TaskArrayRun access to private methods Signed-off-by: Paolo Di Tommaso --- .../groovy/nextflow/processor/TaskRun.groovy | 38 ++++++++++++-- .../processor/TaskArrayRunTest.groovy | 49 +++++++++++++++++++ .../nextflow/processor/TaskRunTest.groovy | 15 ++++++ 3 files changed, 97 insertions(+), 5 deletions(-) create mode 100644 modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayRunTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index 69e217291e..90899f3f70 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -20,6 +20,7 @@ import java.nio.file.FileSystems import java.nio.file.NoSuchFileException import java.nio.file.Path import java.util.concurrent.ConcurrentHashMap +import java.util.function.Function import com.google.common.hash.HashCode import groovy.transform.PackageScope @@ -448,10 +449,16 @@ class TaskRun implements Cloneable { /** * Look at the {@code nextflow.script.FileOutParam} which name is the expected * output name - * */ List getOutputFilesNames() { - cache0.computeIfAbsent('outputFileNames', (it)-> getOutputFilesNames0()) + // note: use an explict function instead of a closure or lambda syntax, otherwise + // when calling this method from a subclass it will result into a MissingMethodExeception + // see https://issues.apache.org/jira/browse/GROOVY-2433 + cache0.computeIfAbsent('outputFileNames', new Function>() { + @Override + List apply(String s) { + return getOutputFilesNames0() + }}) } private List getOutputFilesNames0() { @@ -614,7 +621,14 @@ class TaskRun implements Cloneable { } Path getCondaEnv() { - cache0.computeIfAbsent('condaEnv', (it)-> getCondaEnv0()) + // note: use an explict function instead of a closure or lambda syntax, otherwise + // when calling this method from a subclass it will result into a MissingMethodExeception + // see https://issues.apache.org/jira/browse/GROOVY-2433 + cache0.computeIfAbsent('condaEnv', new Function() { + @Override + Path apply(String it) { + return getCondaEnv0() + }}) } private Path getCondaEnv0() { @@ -626,7 +640,14 @@ class TaskRun implements Cloneable { } Path getSpackEnv() { - cache0.computeIfAbsent('spackEnv', (it)-> getSpackEnv0()) + // note: use an explict function instead of a closure or lambda syntax, otherwise + // when calling this method from a subclass it will result into a MissingMethodExeception + // see https://issues.apache.org/jira/browse/GROOVY-2433 + cache0.computeIfAbsent('spackEnv', new Function() { + @Override + Path apply(String it) { + return getSpackEnv0() + }}) } private Path getSpackEnv0() { @@ -640,7 +661,14 @@ class TaskRun implements Cloneable { } protected ContainerInfo containerInfo() { - cache0.computeIfAbsent('containerInfo', (it)-> containerInfo0()) + // note: use an explict function instead of a closure or lambda syntax, otherwise + // when calling this method from a subclass it will result into a MissingMethodExeception + // see https://issues.apache.org/jira/browse/GROOVY-2433 + cache0.computeIfAbsent('containerInfo', new Function() { + @Override + ContainerInfo apply(String s) { + return containerInfo0() + }}) } private ContainerInfo containerInfo0() { diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayRunTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayRunTest.groovy new file mode 100644 index 0000000000..e64af32035 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayRunTest.groovy @@ -0,0 +1,49 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.processor + +import nextflow.Session +import nextflow.container.resolver.ContainerInfo +import nextflow.executor.Executor +import nextflow.executor.TaskArrayExecutor +import spock.lang.Specification +/** + * + * @author Paolo Di Tommaso + */ +class TaskArrayRunTest extends Specification { + + static abstract class TestExecutor extends Executor implements TaskArrayExecutor { + + } + + def 'should get container info' () { + given: + def session = Mock(Session) + def executor = Mock(TestExecutor) { getSession()>>session } + def processor = Mock(TaskProcessor) { getExecutor()>>executor; getSession()>>session } + and: + def config = new TaskConfig([container:'ubuntu']) + def task = new TaskArrayRun(config: config, processor: processor) + when: + def info = task.containerInfo() + then: + info == new ContainerInfo('ubuntu','ubuntu','ubuntu') + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskRunTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskRunTest.groovy index 80bcbf3458..3fcaac0206 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskRunTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskRunTest.groovy @@ -870,4 +870,19 @@ class TaskRunTest extends Specification { // the engine is enabled by default config == new ContainerConfig(engine:'foo', enabled: true) // <-- 'foo' engine is enabled } + + def 'should get container info' () { + given: + def session = Mock(Session) + def executor = Mock(Executor) { getSession()>>session } + def processor = Mock(TaskProcessor) { getExecutor()>>executor; getSession()>>session } + and: + def config = new TaskConfig([container:'ubuntu']) + def task = new TaskRun(config: config, processor: processor) + when: + def info = task.containerInfo() + then: + info == new ContainerInfo('ubuntu','ubuntu','ubuntu') + } + } From 4bb0e983977741642635465278c0823140c6448f Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 10:34:55 +0200 Subject: [PATCH 71/97] Fix typ Signed-off-by: Paolo Di Tommaso --- .../src/main/groovy/nextflow/processor/TaskRun.groovy | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index 90899f3f70..2189ef39d1 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -451,7 +451,7 @@ class TaskRun implements Cloneable { * output name */ List getOutputFilesNames() { - // note: use an explict function instead of a closure or lambda syntax, otherwise + // note: use an explicit function instead of a closure or lambda syntax, otherwise // when calling this method from a subclass it will result into a MissingMethodExeception // see https://issues.apache.org/jira/browse/GROOVY-2433 cache0.computeIfAbsent('outputFileNames', new Function>() { @@ -621,7 +621,7 @@ class TaskRun implements Cloneable { } Path getCondaEnv() { - // note: use an explict function instead of a closure or lambda syntax, otherwise + // note: use an explicit function instead of a closure or lambda syntax, otherwise // when calling this method from a subclass it will result into a MissingMethodExeception // see https://issues.apache.org/jira/browse/GROOVY-2433 cache0.computeIfAbsent('condaEnv', new Function() { @@ -640,7 +640,7 @@ class TaskRun implements Cloneable { } Path getSpackEnv() { - // note: use an explict function instead of a closure or lambda syntax, otherwise + // note: use an explicit function instead of a closure or lambda syntax, otherwise // when calling this method from a subclass it will result into a MissingMethodExeception // see https://issues.apache.org/jira/browse/GROOVY-2433 cache0.computeIfAbsent('spackEnv', new Function() { @@ -661,7 +661,7 @@ class TaskRun implements Cloneable { } protected ContainerInfo containerInfo() { - // note: use an explict function instead of a closure or lambda syntax, otherwise + // note: use an explicit function instead of a closure or lambda syntax, otherwise // when calling this method from a subclass it will result into a MissingMethodExeception // see https://issues.apache.org/jira/browse/GROOVY-2433 cache0.computeIfAbsent('containerInfo', new Function() { From beddbbd6f12842ac99c5dcf54bef0d2e9d91ee19 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 10:35:15 +0200 Subject: [PATCH 72/97] Add integration tests Signed-off-by: Paolo Di Tommaso --- validation/awsbatch.sh | 7 +++++++ validation/google.sh | 11 ++++++++--- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/validation/awsbatch.sh b/validation/awsbatch.sh index cdb958c315..d58727e7e8 100644 --- a/validation/awsbatch.sh +++ b/validation/awsbatch.sh @@ -67,3 +67,10 @@ $NXF_CMD run nextflow-io/hello \ -process.array 10 \ -plugins nf-cloudcache \ -c awsbatch.config + +## Test use of job array using Fusion +$NXF_CMD run nextflow-io/hello \ + -process.array 10 \ + -with-wave \ + -with-fusion \ + -c awsbatch.config \ No newline at end of file diff --git a/validation/google.sh b/validation/google.sh index bc9fea34d2..ef6ade267d 100644 --- a/validation/google.sh +++ b/validation/google.sh @@ -74,8 +74,13 @@ $NXF_CMD -C ./google.config \ [[ `grep -c 'Cached process > ' .nextflow.log` == 4 ]] || false ## Test job array -NXF_CLOUDCACHE_PATH=gs://rnaseq-nf/cache \ $NXF_CMD -C ./google.config \ - run nextflow-io/rnaseq-nf \ + run nextflow-io/hello \ + -process.array 10 + +## Test job array with Fusion +$NXF_CMD -C ./google.config \ + run nextflow-io/hello \ -process.array 10 \ - -plugins nf-cloudcache + -with-wave \ + -with-fusion From 1713167d618cfde5ce370809a01e3f45045bbdf7 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 10:37:11 +0200 Subject: [PATCH 73/97] Use idiomatic name for task array dir var Signed-off-by: Paolo Di Tommaso --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 4 ++-- .../groovy/nextflow/processor/TaskArrayCollectorTest.groovy | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index a544c69dbb..b1800909f2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -188,7 +188,7 @@ class TaskArrayCollector { // get work directory and launch command for each task final workDirs = array.collect( h -> h.getWorkDir() ) final args = array.first().getLaunchCommand().toArray() as String[] - final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${array_task_dir}') + final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${nxf_array_task_dir}') // create wrapper script final indexName = executor.getArrayIndexName() @@ -199,7 +199,7 @@ class TaskArrayCollector { """ array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export array_task_dir=\${array[${arrayIndex}]} + export nxf_array_task_dir=\${array[${arrayIndex}]} ${cmd} """.stripIndent().leftTrim() } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index c92eb50e84..e9119c3044 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -147,8 +147,8 @@ class TaskArrayCollectorTest extends Specification { taskArray.processor == proc taskArray.script == ''' array=( /work/foo /work/foo /work/foo ) - export array_task_dir=${array[ARRAY_JOB_INDEX]} - bash -o pipefail -c 'bash ${array_task_dir}/.command.run 2>&1 | tee ${array_task_dir}/.command.log' + export nxf_array_task_dir=${array[ARRAY_JOB_INDEX]} + bash -o pipefail -c 'bash ${nxf_array_task_dir}/.command.run 2>&1 | tee ${nxf_array_task_dir}/.command.log' '''.stripIndent().leftTrim() and: taskArray.getArraySize() == 3 From bf859bdda164e24e7139a46b7f6096211bf19d4b Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 10:50:49 +0200 Subject: [PATCH 74/97] Use TaskArrayRun as return type Signed-off-by: Paolo Di Tommaso --- .../main/groovy/nextflow/processor/TaskArrayCollector.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index b1800909f2..7177f99071 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -135,7 +135,7 @@ class TaskArrayCollector { * * @param tasks */ - protected TaskRun createTaskArray(List tasks) { + protected TaskArrayRun createTaskArray(List tasks) { // prepare child job launcher scripts final handlers = tasks.collect( t -> executor.createTaskHandler(t) ) for( TaskHandler handler : handlers ) { From a4518a30bbfe62a8b3ecdbee5457fbeae68105c2 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 11:03:25 +0200 Subject: [PATCH 75/97] Promote status update methods private Signed-off-by: Paolo Di Tommaso --- .../groovy/nextflow/executor/GridTaskHandler.groovy | 10 +++++----- .../cloud/aws/batch/AwsBatchTaskHandler.groovy | 11 ++++++----- .../cloud/google/batch/GoogleBatchTaskHandler.groovy | 11 ++++++----- 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 5bc3635951..43d47bca29 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -285,7 +285,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { final result = safeExecute( () -> processStart(builder, stdinScript) ) // -- save the job id final jobId = (String)executor.parseJobId(result) - this.onSubmit(jobId) + updateStatus(jobId) log.debug "[${executor.name.toUpperCase()}] submitted process ${task.name} > jobId: $jobId; workDir: ${task.workDir}" } @@ -302,14 +302,14 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { status = COMPLETED throw new ProcessFailedException("Error submitting process '${task.name}' for execution", e ) } - } - void onSubmit(String jobId) { + private void updateStatus(String jobId) { if( task instanceof TaskArrayRun ) { - task.children.eachWithIndex { handler, i -> + for( int i=0; i job=$jobId; work-dir=${task.getWorkDirStr()}" } - void onSubmit(String jobId, String queueName) { + private void updateStatus(String jobId, String queueName) { if( task instanceof TaskArrayRun ) { - task.children.eachWithIndex { handler, i -> + // update status for children tasks + for( int i=0; i $req" final resp = client.submitJob(jobId, req) final uid = resp.getUid() - this.onSubmit(jobId, '0', uid) + updateStatus(jobId, '0', uid) log.debug "[GOOGLE BATCH] Process `${task.lazyName()}` submitted > job=$jobId; uid=$uid; work-dir=${task.getWorkDirStr()}" } - void onSubmit(String jobId, String taskId, String uid) { + private void updateStatus(String jobId, String taskId, String uid) { if( task instanceof TaskArrayRun ) { - task.children.eachWithIndex { handler, i -> + // update status for children + for( int i=0; i Date: Thu, 2 May 2024 20:46:11 +0200 Subject: [PATCH 76/97] Minor change Signed-off-by: Paolo Di Tommaso --- .../nextflow/processor/ParallelPollingMonitor.groovy | 2 +- .../groovy/nextflow/processor/TaskPollingMonitor.groovy | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy index 3d7b4d5aa1..07af8f22c2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/ParallelPollingMonitor.groovy @@ -86,7 +86,7 @@ class ParallelPollingMonitor extends TaskPollingMonitor { if( !session.success ) return // ignore error when the session has been interrupted handleException(handler, e) - notifyTaskComplete0(handler) + notifyTaskComplete(handler) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy index 00a98857f8..2caf82d1ce 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskPollingMonitor.groovy @@ -247,7 +247,7 @@ class TaskPollingMonitor implements TaskMonitor { try{ pendingQueue << handler taskAvail.signal() // signal that a new task is available for execution - notifyTaskPending0(handler) + notifyTaskPending(handler) log.trace "Scheduled task > $handler" } finally { @@ -587,7 +587,7 @@ class TaskPollingMonitor implements TaskMonitor { } catch ( Throwable e ) { handleException(handler, e) - notifyTaskComplete0(handler) + notifyTaskComplete(handler) } // remove processed handler either on successful submit or failed one (managed by catch section) // when `canSubmit` return false the handler should be retained to be tried in a following iteration @@ -718,7 +718,7 @@ class TaskPollingMonitor implements TaskMonitor { return pendingQueue } - protected void notifyTaskPending0(TaskHandler handler) { + protected void notifyTaskPending(TaskHandler handler) { if( handler.task instanceof TaskArrayRun ) { final task = handler.task as TaskArrayRun for( TaskHandler it : task.children ) @@ -729,7 +729,7 @@ class TaskPollingMonitor implements TaskMonitor { } } - protected void notifyTaskComplete0(TaskHandler handler) { + protected void notifyTaskComplete(TaskHandler handler) { if( handler.task instanceof TaskArrayRun ) { final task = handler.task as TaskArrayRun for( TaskHandler it : task.children ) From 8afc6a9682fdf46feeb0bcbbd12863be2051c3cd Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 2 May 2024 21:35:42 +0200 Subject: [PATCH 77/97] Refactor task name in task meta comment Signed-off-by: Paolo Di Tommaso --- .../executor/BashWrapperBuilder.groovy | 28 ++++----- .../nextflow/executor/command-run.txt | 1 - .../executor/BashTemplateEngineTest.groovy | 2 - .../executor/BashWrapperBuilderTest.groovy | 60 ++++++++++++++++++- .../executor/test-bash-wrapper-with-trace.txt | 4 +- .../nextflow/executor/test-bash-wrapper.txt | 4 +- .../google/lifesciences/bash-wrapper-gcp.txt | 4 +- 7 files changed, 78 insertions(+), 25 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index 745c279f71..857911ef7c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -463,32 +463,28 @@ class BashWrapperBuilder { } protected String getTaskMetadata() { - final lines = [] - lines << '---' - + final lines = new StringBuilder() + lines << '### ---\n' + lines << "### name: '${name}'\n" if( arrayIndexName ) { - lines << 'array:' - lines << " index-name: ${arrayIndexName}" - lines << " index-start: ${arrayIndexStart}" - lines << " work-dirs:" + lines << '### array:\n' + lines << "### index-name: ${arrayIndexName}\n" + lines << "### index-start: ${arrayIndexStart}\n" + lines << "### work-dirs:\n" for( String workDir : arrayWorkDirs ) - lines << " - ${Escape.path(workDir)}" + lines << "### - ${Escape.path(workDir)}\n" } if( containerConfig?.isEnabled() ) - lines << "container: '${containerImage}'" + lines << "### container: '${containerImage}'\n" if( outputFiles.size() > 0 ) { - lines << 'outputs:' + lines << '### outputs:\n' for( final output : outputFiles ) - lines << "- '${output}'" + lines << "### - '${output}'\n" } - if( lines.size() == 1 ) - return '' - - lines << '...' - return lines.collect( line -> '### ' + line ).join('\n') + lines << '### ...\n' } protected String getHelpersScript() { diff --git a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt index c60c83e85f..2bf34b617a 100644 --- a/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt +++ b/modules/nextflow/src/main/resources/nextflow/executor/command-run.txt @@ -15,7 +15,6 @@ #!/bin/bash {{header_script}} {{task_metadata}} -# NEXTFLOW TASK: {{task_name}} set -e set -u NXF_DEBUG=${NXF_DEBUG:=0}; [[ $NXF_DEBUG > 1 ]] && set -x diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashTemplateEngineTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashTemplateEngineTest.groovy index 90df503a53..30616c682a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashTemplateEngineTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashTemplateEngineTest.groovy @@ -31,7 +31,6 @@ class BashTemplateEngineTest extends Specification { def template = '''\ ## comment #!/bin/bash - # NEXTFLOW TASK: foo line 1 ## comment ## @@ -42,7 +41,6 @@ class BashTemplateEngineTest extends Specification { expect: engine.render(template, [:]) == '''\ #!/bin/bash - # NEXTFLOW TASK: foo line 1 line 2 line 3 diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy index e5b9297bd6..cad2969b9e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy @@ -27,6 +27,7 @@ import nextflow.container.DockerBuilder import nextflow.container.SingularityBuilder import nextflow.processor.TaskBean import nextflow.util.MustacheTemplateEngine +import org.yaml.snakeyaml.Yaml import spock.lang.Specification import spock.lang.Unroll /** @@ -337,15 +338,67 @@ class BashWrapperBuilderTest extends Specification { } + def 'should create task metadata string' () { + given: + def builder = newBashWrapperBuilder( + name: 'foo', + arrayIndexName: 'SLURM_ARRAY_TASK_ID', + arrayIndexStart: 0, + arrayWorkDirs: [ '/work/01', '/work/02', '/work/03' ], + containerConfig: [enabled: true], + containerImage: 'quay.io/nextflow:bash', + outputFiles: ['foo.txt', '*.bar', '**/baz'] + ) + + when: + def meta = builder.getTaskMetadata() + then: + meta == '''\ + ### --- + ### name: 'foo' + ### array: + ### index-name: SLURM_ARRAY_TASK_ID + ### index-start: 0 + ### work-dirs: + ### - /work/01 + ### - /work/02 + ### - /work/03 + ### container: 'quay.io/nextflow:bash' + ### outputs: + ### - 'foo.txt' + ### - '*.bar' + ### - '**/baz' + ### ... + '''.stripIndent() + + when: + def yaml = meta.readLines().collect(it-> it.substring(4)).join('\n') + def obj = new Yaml().load(yaml) as Map + then: + obj.name == 'foo' + obj.array == [ + 'index-name':'SLURM_ARRAY_TASK_ID', + 'index-start':0, + 'work-dirs':['/work/01', '/work/02', '/work/03'] + ] + obj.container == 'quay.io/nextflow:bash' + obj.outputs == ['foo.txt', '*.bar', '**/baz'] + } + def 'should add task metadata' () { when: - def bash = newBashWrapperBuilder() + def bash = newBashWrapperBuilder([name:'task1']) then: bash.makeBinding().containsKey('task_metadata') - bash.makeBinding().task_metadata == '' + bash.makeBinding().task_metadata == '''\ + ### --- + ### name: 'task1' + ### ... + '''.stripIndent() when: bash = newBashWrapperBuilder( + name: 'task2', arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, arrayWorkDirs: [ '/work/01', '/work/02', '/work/03' ], @@ -356,6 +409,7 @@ class BashWrapperBuilderTest extends Specification { then: bash.makeBinding().task_metadata == '''\ ### --- + ### name: 'task2' ### array: ### index-name: SLURM_ARRAY_TASK_ID ### index-start: 0 @@ -369,7 +423,7 @@ class BashWrapperBuilderTest extends Specification { ### - '*.bar' ### - '**/baz' ### ... - '''.stripIndent().rightTrim() + '''.stripIndent() } def 'should copy control files' () { diff --git a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt index b72d5880ce..ef1380e7cf 100644 --- a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt +++ b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt @@ -1,5 +1,7 @@ #!/bin/bash -# NEXTFLOW TASK: Hello 2 +### --- +### name: 'Hello 2' +### ... set -e set -u NXF_DEBUG=${NXF_DEBUG:=0}; [[ $NXF_DEBUG > 1 ]] && set -x diff --git a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt index 5abcf4a511..f465b5b9b4 100644 --- a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt +++ b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt @@ -1,7 +1,9 @@ #!/bin/bash #BSUB -x 1 #BSUB -y 2 -# NEXTFLOW TASK: Hello 1 +### --- +### name: 'Hello 1' +### ... set -e set -u NXF_DEBUG=${NXF_DEBUG:=0}; [[ $NXF_DEBUG > 1 ]] && set -x diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt index ab322a6ef6..70a68452aa 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt +++ b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt @@ -1,5 +1,7 @@ #!/bin/bash -# NEXTFLOW TASK: Hello 1 +### --- +### name: 'Hello 1' +### ... set -e set -u NXF_DEBUG=${NXF_DEBUG:=0}; [[ $NXF_DEBUG > 1 ]] && set -x From b42ccbb82628bb9383b8cfd628ad7fcaa2c13869 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Fri, 3 May 2024 15:40:59 +0200 Subject: [PATCH 78/97] Force new test Signed-off-by: Paolo Di Tommaso --- .github/workflows/build.yml | 1 + .../main/io/seqera/tower/plugin/LogsCheckpoint.groovy | 2 +- validation/google.sh | 11 ++++++----- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 6bcde07363..74f060ad13 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -111,6 +111,7 @@ jobs: if: ${{ !contains(github.event.head_commit.message, '[ci fast]') && needs.build.outputs.any_changed == 'true' }} needs: build runs-on: ubuntu-latest + timeout-minutes: 90 strategy: fail-fast: false matrix: diff --git a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy index ee353d66b4..9b57a7a166 100644 --- a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy +++ b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy @@ -36,7 +36,7 @@ class LogsCheckpoint implements TraceObserver { private Session session private Map config - private Thread thread + private volatile Thread thread private Duration interval private LogsHandler handler private volatile boolean terminated diff --git a/validation/google.sh b/validation/google.sh index ef6ade267d..8dcda8ab99 100644 --- a/validation/google.sh +++ b/validation/google.sh @@ -73,14 +73,15 @@ $NXF_CMD -C ./google.config \ [[ `grep -c 'Using Nextflow cache factory: nextflow.cache.CloudCacheFactory' .nextflow.log` == 1 ]] || false [[ `grep -c 'Cached process > ' .nextflow.log` == 4 ]] || false -## Test job array -$NXF_CMD -C ./google.config \ - run nextflow-io/hello \ - -process.array 10 - ## Test job array with Fusion $NXF_CMD -C ./google.config \ run nextflow-io/hello \ -process.array 10 \ -with-wave \ -with-fusion + +## Test job array +$NXF_CMD -C ./google.config \ + run nextflow-io/hello \ + -process.array 10 + From 653064f543a31a18a72b96cc0c781a51e09a5c80 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 2 May 2024 14:55:07 -0500 Subject: [PATCH 79/97] Fix collectFile saving to GCS with sort: false (#4965) Signed-off-by: Ben Sherman --- .../groovy/nextflow/file/SimpleFileCollector.groovy | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/file/SimpleFileCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/file/SimpleFileCollector.groovy index c6636b69a8..296c346775 100644 --- a/modules/nextflow/src/main/groovy/nextflow/file/SimpleFileCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/file/SimpleFileCollector.groovy @@ -15,14 +15,15 @@ */ package nextflow.file + import java.nio.file.Files import java.nio.file.Path -import java.nio.file.StandardCopyOption import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentMap import groovy.transform.CompileStatic import groovy.util.logging.Slf4j +import nextflow.extension.FilesEx /** * Helper class used to aggregate values having the same key * to files @@ -94,16 +95,13 @@ class SimpleFileCollector extends FileCollector { @Override void saveFile( Closure closure ) { - def result = [] Iterator itr = cache.values().iterator() while( itr.hasNext() ) { - def item = itr.next() - def target = closure.call(item.getName()) - result << Files.move(item, target, StandardCopyOption.REPLACE_EXISTING) + final item = itr.next() + final target = closure.call(item.getName()) + FilesEx.moveTo(item, target) itr.remove() } - } - } From ea1cc2c1031ec62f5cd4e88ee1257df6b0979b43 Mon Sep 17 00:00:00 2001 From: Mahesh Binzer-Panchal Date: Thu, 2 May 2024 22:23:47 +0200 Subject: [PATCH 80/97] Fix script error text alignment (#4681) Signed-off-by: Mahesh Binzer-Panchal Signed-off-by: Ben Sherman Co-authored-by: Ben Sherman --- .../nextflow/ast/NextflowDSLImpl.groovy | 40 +++++++++++++------ .../nextflow/processor/TaskProcessor.groovy | 5 ++- 2 files changed, 31 insertions(+), 14 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/ast/NextflowDSLImpl.groovy b/modules/nextflow/src/main/groovy/nextflow/ast/NextflowDSLImpl.groovy index 7396a5ab16..739c554f2d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/ast/NextflowDSLImpl.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/ast/NextflowDSLImpl.groovy @@ -473,7 +473,7 @@ class NextflowDSLImpl implements ASTTransformation { } } // read the closure source - readSource(closure, source, unit, true) + readSource(closure, source, unit) final bodyClosure = closureX(null, block(scope, body)) final invokeBody = makeScriptWrapper(bodyClosure, source.toString(), 'workflow', unit) @@ -750,7 +750,29 @@ class NextflowDSLImpl implements ASTTransformation { * @param buffer * @param unit */ - private void readSource( ASTNode node, StringBuilder buffer, SourceUnit unit, stripBrackets=false ) { + private void readSource( Statement node, StringBuilder buffer, SourceUnit unit ) { + final colx = node.getColumnNumber() + final colz = node.getLastColumnNumber() + final first = node.getLineNumber() + final last = node.getLastLineNumber() + for( int i = first; i <= last; i++ ) { + final line = unit.source.getLine(i, null) + + // prepend first-line indent + if( i == first ) { + int k = 0 + while( k < line.size() && line[k] == ' ' ) + k++ + buffer.append( line.substring(0, k) ) + } + + final begin = (i == first) ? colx - 1 : 0 + final end = (i == last) ? colz - 1 : line.size() + buffer.append( line.substring(begin, end) ).append('\n') + } + } + + private void readSource( ClosureExpression node, StringBuilder buffer, SourceUnit unit ) { final colx = node.getColumnNumber() final colz = node.getLastColumnNumber() final first = node.getLineNumber() @@ -758,18 +780,12 @@ class NextflowDSLImpl implements ASTTransformation { for( int i=first; i<=last; i++ ) { def line = unit.source.getLine(i, null) if( i==last ) { - line = line.substring(0,colz-1) - if( stripBrackets ) { - line = line.replaceFirst(/}.*$/,'') - if( !line.trim() ) continue - } + line = line.substring(0,colz-1).replaceFirst(/}.*$/,'') + if( !line.trim() ) continue } if( i==first ) { - line = line.substring(colx-1) - if( stripBrackets ) { - line = line.replaceFirst(/^.*\{/,'').trim() - if( !line.trim() ) continue - } + line = line.substring(colx-1).replaceFirst(/^.*\{/,'').trim() + if( !line ) continue } buffer.append(line) .append('\n') } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 83a7707500..edc8ab6642 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -1335,9 +1335,10 @@ class TaskProcessor { else message = err0(error.cause) + message.eachLine { line -> + result << ' ' << line << '\n' + } result - .append(' ') - .append(message) .append('\n') .toString() } From 73676870fee7e81efa64aff1c5af440b52096d82 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Fri, 3 May 2024 10:15:02 +0200 Subject: [PATCH 81/97] Use for instead eachLine in error formatting [ci fast] Signed-off-by: Paolo Di Tommaso --- .../src/main/groovy/nextflow/processor/TaskProcessor.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index edc8ab6642..302955617a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -1335,9 +1335,10 @@ class TaskProcessor { else message = err0(error.cause) - message.eachLine { line -> + for( String line : message.readLines() ) { result << ' ' << line << '\n' } + result .append('\n') .toString() From be536f8e875b6ad80e82f832ecdc8cfd27df95ec Mon Sep 17 00:00:00 2001 From: Herman Singh Date: Fri, 3 May 2024 06:37:10 -0400 Subject: [PATCH 82/97] Update aws.md to include Cluster access (#4951) [ci skip] Signed-off-by: Herman Singh --- docs/aws.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/aws.md b/docs/aws.md index 5fccc7b4fa..46b117041e 100644 --- a/docs/aws.md +++ b/docs/aws.md @@ -85,6 +85,15 @@ Minimal permissions policies to be attached to the AWS account used by Nextflow "ecr:DescribeImageScanFindings" ``` +Note: If you are running Fargate or Fargate Spot, you may need the following policies in addition to the listed above: + ```json + "ecs:CreateCluster" + "ecs:DeleteCluster" + "ecs:DescribeClusters" + "ecs:ListClusters" + "ec2:DescribeSubnets" + ``` + ### S3 policies Nextflow also requires policies to access [S3 buckets](https://aws.amazon.com/s3/) in order to use the work directory, pull input data, and publish results. From d899ebcedb8d6d854bf47e0bf19a603a95e83bed Mon Sep 17 00:00:00 2001 From: Dr Marco Claudio De La Pierre Date: Fri, 3 May 2024 22:09:40 +0800 Subject: [PATCH 83/97] Add Wave and Fusion info to workflow metadata (#4945) Signed-off-by: Dr Marco Claudio De La Pierre Signed-off-by: Paolo Di Tommaso Co-authored-by: Paolo Di Tommaso Co-authored-by: Ben Sherman --- docs/metadata.md | 9 ++++ .../nextflow/fusion/FusionConfig.groovy | 26 +++++++++- .../nextflow/script/FusionMetadata.groovy | 49 +++++++++++++++++++ .../nextflow/script/WaveMetadata.groovy | 49 +++++++++++++++++++ .../nextflow/script/WorkflowMetadata.groovy | 15 ++++++ .../resources/nextflow/mail/notification.html | 10 ++++ .../resources/nextflow/mail/notification.txt | 2 + .../nextflow/trace/ReportTemplate.html | 8 +++ .../nextflow/fusion/FusionConfigTest.groovy | 22 +++++++++ .../nextflow/script/FusionMetaTest.groovy | 47 ++++++++++++++++++ .../nextflow/script/WaveMetadataTest.groovy | 45 +++++++++++++++++ .../script/WorkflowMetadataTest.groovy | 6 ++- .../script/WorkflowNotifierTest.groovy | 11 +++++ .../nextflow/trace/ReportObserverTest.groovy | 4 ++ 14 files changed, 301 insertions(+), 2 deletions(-) create mode 100644 modules/nextflow/src/main/groovy/nextflow/script/FusionMetadata.groovy create mode 100644 modules/nextflow/src/main/groovy/nextflow/script/WaveMetadata.groovy create mode 100644 modules/nextflow/src/test/groovy/nextflow/script/FusionMetaTest.groovy create mode 100644 modules/nextflow/src/test/groovy/nextflow/script/WaveMetadataTest.groovy diff --git a/docs/metadata.md b/docs/metadata.md index 529060bb98..193396be09 100644 --- a/docs/metadata.md +++ b/docs/metadata.md @@ -58,6 +58,12 @@ The following table lists the properties that can be accessed on the `workflow` : *Available only in the `workflow.onComplete` and `workflow.onError` handlers* : Exit status of the task that caused the workflow execution to fail. +`workflow.fusion.enabled` +: Whether Fusion is enabled. + +`workflow.fusion.version` +: Fusion version in use. + `workflow.homeDir` : User system home directory. @@ -111,6 +117,9 @@ The following table lists the properties that can be accessed on the `workflow` `workflow.userName` : User system account name. +`workflow.wave.enabled` +: Whether Wave is enabled. + `workflow.workDir` : Workflow working directory. diff --git a/modules/nextflow/src/main/groovy/nextflow/fusion/FusionConfig.groovy b/modules/nextflow/src/main/groovy/nextflow/fusion/FusionConfig.groovy index 76a509d10e..fd12177be7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/fusion/FusionConfig.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/fusion/FusionConfig.groovy @@ -17,12 +17,15 @@ package nextflow.fusion + +import java.util.regex.Pattern + import groovy.transform.CompileStatic import groovy.transform.Memoized import nextflow.Global +import nextflow.Session import nextflow.SysEnv import nextflow.util.MemoryUnit - /** * Model Fusion config options * @@ -37,6 +40,8 @@ class FusionConfig { final static public String FUSION_PATH = '/usr/bin/fusion' + final static private Pattern VERSION_JSON = ~/https:\/\/.*\/releases\/v(\d+(?:\.\w+)*)-(\w*)\.json$/ + final private Boolean enabled final private String containerConfigUrl @Deprecated final private Boolean exportAwsAccessKeys @@ -99,8 +104,27 @@ class FusionConfig { return createConfig0(Global.config?.fusion as Map ?: Collections.emptyMap(), SysEnv.get()) } + static FusionConfig getConfig(Session session) { + return createConfig0(session.config?.fusion as Map ?: Collections.emptyMap(), SysEnv.get()) + } + @Memoized static private FusionConfig createConfig0(Map config, Map env) { new FusionConfig(config, env) } + + protected String retrieveFusionVersion(String url) { + if( !url ) + return null + final matcher_json = VERSION_JSON.matcher(url) + if( matcher_json.matches() ) + return matcher_json.group(1) + return null + } + + String version() { + return enabled + ? retrieveFusionVersion(this.containerConfigUrl ?: DEFAULT_FUSION_AMD64_URL) + : null + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/script/FusionMetadata.groovy b/modules/nextflow/src/main/groovy/nextflow/script/FusionMetadata.groovy new file mode 100644 index 0000000000..6786eb6323 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/script/FusionMetadata.groovy @@ -0,0 +1,49 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.script + +import groovy.transform.CompileStatic +import groovy.transform.EqualsAndHashCode +import groovy.transform.ToString +import groovy.util.logging.Slf4j +import nextflow.fusion.FusionConfig +import nextflow.Session +/** + * Models Fusion metadata for Nextflow execution + * + * @author Marco De La Pierre + */ +@Slf4j +@CompileStatic +@ToString(includeNames = true, includePackage = false) +@EqualsAndHashCode +class FusionMetadata { + boolean enabled + String version + + FusionMetadata(Session session) { + final FusionConfig fusionConfig = FusionConfig.getConfig(session) + this.enabled = fusionConfig.enabled() + this.version = fusionConfig.version() + } + + FusionMetadata(Boolean enabled, String version) { + this.enabled = enabled + this.version = version + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/script/WaveMetadata.groovy b/modules/nextflow/src/main/groovy/nextflow/script/WaveMetadata.groovy new file mode 100644 index 0000000000..39692eb196 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/script/WaveMetadata.groovy @@ -0,0 +1,49 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.script + +import groovy.transform.CompileStatic +import groovy.transform.EqualsAndHashCode +import groovy.transform.ToString +import groovy.util.logging.Slf4j +import nextflow.Session +/** + * Models Wave metadata for Nextflow execution + * + * @author Marco De La Pierre + */ +@Slf4j +@CompileStatic +@ToString(includeNames = true, includePackage = false) +@EqualsAndHashCode +class WaveMetadata { + + final boolean enabled + + WaveMetadata(Session session) { + this( session.config.wave as Map ?: Map.of() ) + } + + WaveMetadata(Map opts) { + this.enabled = opts.enabled as boolean + } + + WaveMetadata(Boolean enabled) { + this.enabled = enabled + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/script/WorkflowMetadata.groovy b/modules/nextflow/src/main/groovy/nextflow/script/WorkflowMetadata.groovy index 07394e5e55..a053212216 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/WorkflowMetadata.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/WorkflowMetadata.groovy @@ -194,6 +194,19 @@ class WorkflowMetadata { */ String containerEngine + /** + * Metadata specific to Wave, including: + *
  • enabled: whether Wave is enabled + */ + WaveMetadata wave + + /** + * Metadata specific to Fusion, including: + *
  • enabled: whether Fusion is enabled + *
  • version: the version of Fusion in use + */ + FusionMetadata fusion + /** * The list of files that concurred to create the config object */ @@ -247,6 +260,8 @@ class WorkflowMetadata { this.userName = System.getProperty('user.name') this.homeDir = Paths.get(System.getProperty('user.home')) this.manifest = session.getManifest() + this.wave = new WaveMetadata(session) + this.fusion = new FusionMetadata(session) // check if there's a onComplete action in the config file registerConfigAction(session.config.workflow as Map) diff --git a/modules/nextflow/src/main/resources/nextflow/mail/notification.html b/modules/nextflow/src/main/resources/nextflow/mail/notification.html index 8d376dce7e..58df6a41e3 100644 --- a/modules/nextflow/src/main/resources/nextflow/mail/notification.html +++ b/modules/nextflow/src/main/resources/nextflow/mail/notification.html @@ -132,6 +132,16 @@

    Execution summary

    ${workflow.containerEngine?:'-'} + + Wave enabled + ${workflow.wave.enabled} + + + + Fusion enabled + ${workflow.fusion.enabled}, version ${workflow.fusion.enabled ? workflow.fusion.version : '-'} + + Nextflow version ${workflow.nextflow.version}, build ${workflow.nextflow.build} (${workflow.nextflow.timestamp}) diff --git a/modules/nextflow/src/main/resources/nextflow/mail/notification.txt b/modules/nextflow/src/main/resources/nextflow/mail/notification.txt index f0a5612675..0c15df42e1 100644 --- a/modules/nextflow/src/main/resources/nextflow/mail/notification.txt +++ b/modules/nextflow/src/main/resources/nextflow/mail/notification.txt @@ -37,6 +37,8 @@ The command used to launch the workflow was as follows: Workflow profile : ${workflow.profile ?: '-'} Workflow container: ${workflow.container ?: '-'} Container engine : ${workflow.containerEngine?:'-'} + Wave enabled : ${workflow.wave.enabled} + Fusion enabled : ${workflow.fusion.enabled}, version ${workflow.fusion.enabled ? workflow.fusion.version : '-'} Nextflow version : ${workflow.nextflow.version}, build ${workflow.nextflow.build} (${workflow.nextflow.timestamp}) -- diff --git a/modules/nextflow/src/main/resources/nextflow/trace/ReportTemplate.html b/modules/nextflow/src/main/resources/nextflow/trace/ReportTemplate.html index 54437e16d9..0921415427 100644 --- a/modules/nextflow/src/main/resources/nextflow/trace/ReportTemplate.html +++ b/modules/nextflow/src/main/resources/nextflow/trace/ReportTemplate.html @@ -196,6 +196,14 @@

    Workflow execution completed unsuccessfully!

    ${workflow.containerEngine?:'-'}
    <% } %> + <% if (workflow.wave.enabled) { %> +
    Wave enabled
    +
    ${workflow.wave.enabled}
    + +
    Fusion enabled
    +
    ${workflow.fusion.enabled}, version ${workflow.fusion.version ?: '-'}
    + <% } %> +
    Nextflow version
    version ${workflow.nextflow.version}, build ${workflow.nextflow.build} (${workflow.nextflow.timestamp})
    diff --git a/modules/nextflow/src/test/groovy/nextflow/fusion/FusionConfigTest.groovy b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionConfigTest.groovy index 6b8dd709a1..a4a5285304 100644 --- a/modules/nextflow/src/test/groovy/nextflow/fusion/FusionConfigTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/fusion/FusionConfigTest.groovy @@ -129,4 +129,26 @@ class FusionConfigTest extends Specification { [privileged:true] | true [privileged:false] | false } + + @Unroll + def 'should parse fusion version' () { + expect: + new FusionConfig([:]).retrieveFusionVersion(FUSION_URL) == EXPECTED + where: + FUSION_URL | EXPECTED + FusionConfig.DEFAULT_FUSION_AMD64_URL | '2.2' + FusionConfig.DEFAULT_FUSION_ARM64_URL | '2.2' + 'https://foo.com/releases/v3.0-amd.json'| '3.0' + } + + def 'should get version version from config' () { + expect: + new FusionConfig([containerConfigUrl:FUSION_URL, enabled:ENABLED]).version() == EXPECTED + where: + FUSION_URL | ENABLED | EXPECTED + null | false | null + null | true | '2.2' + 'https://foo.com/releases/v4.0-amd64.json' | true | '4.0' + 'https://foo.com/releases/v4.0.1-amd64.json' | true | '4.0.1' + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/script/FusionMetaTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/FusionMetaTest.groovy new file mode 100644 index 0000000000..eef33380d5 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/script/FusionMetaTest.groovy @@ -0,0 +1,47 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.script + +import nextflow.Session +import spock.lang.Specification +import spock.lang.Unroll + +/** + * + * @author Paolo Di Tommaso + */ +class FusionMetaTest extends Specification { + + @Unroll + def 'should get fusion meta' () { + given: + def session = Mock(Session) { getConfig()>>OPTS } + + expect: + new FusionMetadata(session).enabled == EXPECTED_ENABLED + new FusionMetadata(session).version == EXPECTED_VERSION + + where: + OPTS | EXPECTED_ENABLED | EXPECTED_VERSION + [:] | false | null + [fusion:[enabled:false]] | false | null + [fusion:[enabled:true]] | true | '2.2' + [fusion:[enabled:true, containerConfigUrl: 'https://foo.io/releases/v3.0-amd64.json']] | true | '3.0' + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/script/WaveMetadataTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/WaveMetadataTest.groovy new file mode 100644 index 0000000000..e51668692b --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/script/WaveMetadataTest.groovy @@ -0,0 +1,45 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.script + +import nextflow.Session +import spock.lang.Specification +import spock.lang.Unroll + +/** + * + * @author Paolo Di Tommaso + */ +class WaveMetadataTest extends Specification { + + @Unroll + def 'should create meta' () { + given: + def session = Mock(Session) { getConfig()>>OPTS } + + expect: + new WaveMetadata(session).enabled == EXPECTED + where: + OPTS | EXPECTED + [:] | false + [wave:[enabled:false]] | false + [wave:[enabled:true]] | true + + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/script/WorkflowMetadataTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/WorkflowMetadataTest.groovy index d9ce1ada42..edf6d8459c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/WorkflowMetadataTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/WorkflowMetadataTest.groovy @@ -58,7 +58,8 @@ class WorkflowMetadataTest extends Specification { def handlerInvoked def config = [workflow: [onComplete: { -> handlerInvoked=workflow.commandLine } ], docker:[enabled:true], - manifest: [version: '1.0.0', nextflowVersion: '>=0.31.1']] + manifest: [version: '1.0.0', nextflowVersion: '>=0.31.1'], + wave:[enabled:true], fusion:[enabled:true, containerConfigUrl: 'https://fusionfs.seqera.io/releases/v1.2.3-amd64.json']] Session session = Spy(Session, constructorArgs: [config]) session.configFiles >> [Paths.get('foo'), Paths.get('bar')] session.getStatsObserver() >> Mock(WorkflowStatsObserver) { getStats() >> new WorkflowStats() } @@ -89,6 +90,9 @@ class WorkflowMetadataTest extends Specification { metadata.sessionId == session.uniqueId metadata.runName == session.runName metadata.containerEngine == 'docker' + metadata.wave.enabled == true + metadata.fusion.enabled == true + metadata.fusion.version == '1.2.3' metadata.configFiles == [Paths.get('foo').toAbsolutePath(), Paths.get('bar').toAbsolutePath()] metadata.resume == false metadata.stubRun == false diff --git a/modules/nextflow/src/test/groovy/nextflow/script/WorkflowNotifierTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/WorkflowNotifierTest.groovy index ce39fc361e..667d1ea15b 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/WorkflowNotifierTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/WorkflowNotifierTest.groovy @@ -26,6 +26,8 @@ import nextflow.NextflowMeta import nextflow.mail.Attachment import nextflow.mail.Mail import nextflow.mail.Mailer +import nextflow.script.FusionMetadata +import nextflow.script.WaveMetadata import nextflow.trace.WorkflowStats import nextflow.util.Duration import spock.lang.Specification @@ -64,6 +66,8 @@ class WorkflowNotifierTest extends Specification { profile: 'my-cluster', container: 'image/foo:tag', containerEngine: 'docker', + wave: new WaveMetadata(true), + fusion: new FusionMetadata(true, '1.2.3'), nextflow: new NextflowMeta('0.27.0', 333, '2017-12-12'), stats: new WorkflowStats(succeedMillis: 4_000_000, succeededCount: 10, failedCount: 20, cachedCount: 30, ignoredCount: 0) ) @@ -103,6 +107,8 @@ class WorkflowNotifierTest extends Specification { Workflow profile : my-cluster Workflow container: image/foo:tag Container engine : docker + Wave enabled : true + Fusion enabled : true, version 1.2.3 Nextflow version : 0.27.0, build 333 (2017-12-12) -- @@ -152,6 +158,8 @@ class WorkflowNotifierTest extends Specification { Workflow profile : my-cluster Workflow container: image/foo:tag Container engine : docker + Wave enabled : true + Fusion enabled : true, version 1.2.3 Nextflow version : 0.27.0, build 333 (2017-12-12) -- @@ -191,6 +199,9 @@ class WorkflowNotifierTest extends Specification { profile: 'my-cluster', container: 'image/foo:tag', containerEngine: 'docker', + wave: new WaveMetadata(true), + fusion: new FusionMetadata(true, '1.2.3'), + nextflow: new NextflowMeta('0.27.0', 333, '2017-12-12'), stats: new WorkflowStats(succeedMillis: 4000) ) diff --git a/modules/nextflow/src/test/groovy/nextflow/trace/ReportObserverTest.groovy b/modules/nextflow/src/test/groovy/nextflow/trace/ReportObserverTest.groovy index cbe0a19b6b..08e4001e5c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/trace/ReportObserverTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/trace/ReportObserverTest.groovy @@ -24,6 +24,8 @@ import groovy.json.JsonSlurper import nextflow.NextflowMeta import nextflow.processor.TaskHandler import nextflow.processor.TaskId +import nextflow.script.FusionMetadata +import nextflow.script.WaveMetadata import nextflow.script.WorkflowMetadata import spock.lang.Specification import test.TestHelper @@ -120,6 +122,8 @@ class ReportObserverTest extends Specification { complete: OffsetDateTime.now(), workDir: workDir, stats: new WorkflowStats(), + wave: new WaveMetadata(true), + fusion: new FusionMetadata(true, '1.2.3'), nextflow: new NextflowMeta('0.27.9', 3232, '2017-12-12') ) From 11d4a877c3e90f3ad366131b4ae6e0565108634e Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Sat, 4 May 2024 19:58:43 +0200 Subject: [PATCH 84/97] Strip auth secret from logs Signed-off-by: Paolo Di Tommaso --- modules/nf-commons/src/main/nextflow/util/StringUtils.groovy | 3 ++- .../nf-commons/src/test/nextflow/util/StringUtilsTest.groovy | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/nf-commons/src/main/nextflow/util/StringUtils.groovy b/modules/nf-commons/src/main/nextflow/util/StringUtils.groovy index 64ad3b2672..432e7b6826 100644 --- a/modules/nf-commons/src/main/nextflow/util/StringUtils.groovy +++ b/modules/nf-commons/src/main/nextflow/util/StringUtils.groovy @@ -81,7 +81,8 @@ class StringUtils { return str.contains('password') \ || str.contains('token') \ || str.contains('secret') \ - || str.contains('license') + || str.contains('license') \ + || str.contains('auth') } diff --git a/modules/nf-commons/src/test/nextflow/util/StringUtilsTest.groovy b/modules/nf-commons/src/test/nextflow/util/StringUtilsTest.groovy index 08f7b5987f..8a257bcdd5 100644 --- a/modules/nf-commons/src/test/nextflow/util/StringUtilsTest.groovy +++ b/modules/nf-commons/src/test/nextflow/util/StringUtilsTest.groovy @@ -92,6 +92,7 @@ class StringUtilsTest extends Specification { [url: 'redis://host:port'] | [url: 'redis://host:port'] [url: 'redis://secret@host:port'] | [url: 'redis://****@host:port'] [url: 'ftp://secretlong@host:port/x/y'] | [url: 'ftp://sec****@host:port/x/y'] + [providers:[github:[auth:'12345']]] | [providers:[github:[auth:'****']]] } @Unroll From a9d5d86db86c6c096be4c6786367b293bc495b38 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Sun, 5 May 2024 21:19:15 +0200 Subject: [PATCH 85/97] [ci skip] empty Signed-off-by: Paolo Di Tommaso From 174bdfb52a024009725906e1286b36b10188501f Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Mon, 6 May 2024 20:57:45 +0200 Subject: [PATCH 86/97] Job array refactor (#4973) --------- Signed-off-by: Paolo Di Tommaso --- .../executor/BashWrapperBuilder.groovy | 17 +++-- .../nextflow/executor/GridTaskHandler.groovy | 16 ---- .../nextflow/executor/LsfExecutor.groovy | 13 +++- .../nextflow/executor/PbsExecutor.groovy | 11 ++- .../nextflow/executor/SgeExecutor.groovy | 12 ++- .../nextflow/executor/SlurmExecutor.groovy | 13 +++- .../executor/TaskArrayExecutor.groovy | 32 +++++++- .../processor/TaskArrayCollector.groovy | 31 ++++---- .../groovy/nextflow/processor/TaskBean.groovy | 4 +- .../nextflow/processor/TaskHandler.groovy | 10 --- .../executor/BashWrapperBuilderTest.groovy | 4 +- .../executor/GridTaskHandlerTest.groovy | 17 ----- .../nextflow/executor/LsfExecutorTest.groovy | 21 ++++++ .../nextflow/executor/PbsExecutorTest.groovy | 20 +++++ .../executor/PbsProExecutorTest.groovy | 22 ++++++ .../nextflow/executor/SgeExecutorTest.groovy | 22 ++++++ .../executor/SlurmExecutorTest.groovy | 21 ++++++ .../processor/TaskArrayCollectorTest.groovy | 53 +++++++++++-- .../cloud/aws/batch/AwsBatchExecutor.groovy | 45 ++++++++++- .../aws/batch/AwsBatchTaskHandler.groovy | 32 +------- .../aws/batch/AwsBatchTaskHandlerTest.groovy | 59 ++++++++------- .../executor/AwsBatchExecutorTest.groovy | 75 +++++++++++++++++++ .../executor/TaskArrayExecutorTest.groovy | 73 ++++++++++++++++++ .../google/batch/GoogleBatchExecutor.groovy | 35 ++++++++- .../batch/GoogleBatchScriptLauncher.groovy | 17 ++++- .../batch/GoogleBatchTaskHandler.groovy | 19 ----- .../batch/GoogleBatchExecutorTest.groovy | 70 ++++++++++++++++- 27 files changed, 586 insertions(+), 178 deletions(-) create mode 100644 plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index 857911ef7c..9cb514a811 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -31,6 +31,7 @@ import nextflow.container.ContainerBuilder import nextflow.container.DockerBuilder import nextflow.container.SingularityBuilder import nextflow.exception.ProcessException +import nextflow.extension.FilesEx import nextflow.file.FileHelper import nextflow.processor.TaskBean import nextflow.processor.TaskProcessor @@ -465,22 +466,22 @@ class BashWrapperBuilder { protected String getTaskMetadata() { final lines = new StringBuilder() lines << '### ---\n' - lines << "### name: '${name}'\n" - if( arrayIndexName ) { + lines << "### name: '${bean.name}'\n" + if( bean.arrayIndexName ) { lines << '### array:\n' - lines << "### index-name: ${arrayIndexName}\n" - lines << "### index-start: ${arrayIndexStart}\n" + lines << "### index-name: ${bean.arrayIndexName}\n" + lines << "### index-start: ${bean.arrayIndexStart}\n" lines << "### work-dirs:\n" - for( String workDir : arrayWorkDirs ) - lines << "### - ${Escape.path(workDir)}\n" + for( Path it : bean.arrayWorkDirs ) + lines << "### - ${Escape.path(FilesEx.toUriString(it))}\n" } if( containerConfig?.isEnabled() ) - lines << "### container: '${containerImage}'\n" + lines << "### container: '${bean.containerImage}'\n" if( outputFiles.size() > 0 ) { lines << '### outputs:\n' - for( final output : outputFiles ) + for( final output : bean.outputFiles ) lines << "### - '${output}'\n" } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 43d47bca29..78cf4be967 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -44,7 +44,6 @@ import nextflow.processor.TaskRun import nextflow.trace.TraceRecord import nextflow.util.CmdLineHelper import nextflow.util.Duration -import nextflow.util.Escape import nextflow.util.Throttle /** * Handles a job execution in the underlying grid platform @@ -108,21 +107,6 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { createTaskWrapper(task).build() } - @Override - String getWorkDir() { - fusionEnabled() - ? FusionHelper.toContainerMount(task.workDir).toString() - : task.workDir.toString() - } - - @Override - List getLaunchCommand() { - final workDir = Escape.path(getWorkDir()) - final cmd = "bash ${workDir}/${TaskRun.CMD_RUN} 2>&1 | tee ${workDir}/${TaskRun.CMD_LOG}" - - List.of('bash', '-o', 'pipefail', '-c', cmd.toString()) - } - protected ProcessBuilder createProcessBuilder() { // -- log the qsub command diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index b25f59d7b8..32f31c53e4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -320,12 +320,19 @@ class LsfExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } @Override - String getArrayIndexName() { 'LSB_JOBINDEX' } + String getArrayIndexName() { + return 'LSB_JOBINDEX' + } @Override - int getArrayIndexStart() { 1 } + int getArrayIndexStart() { + return 1 + } @Override - String getArrayTaskId(String jobId, int index) { "${jobId}[${index + 1}]" } + String getArrayTaskId(String jobId, int index) { + assert jobId, "Missing 'jobId' argument" + return "${jobId}[${index + 1}]" + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 8589d8b65a..0688104776 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -182,14 +182,19 @@ class PbsExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } @Override - String getArrayIndexName() { 'PBS_ARRAY_INDEX' } + String getArrayIndexName() { + return 'PBS_ARRAY_INDEX' + } @Override - int getArrayIndexStart() { 0 } + int getArrayIndexStart() { + return 0 + } @Override String getArrayTaskId(String jobId, int index) { - jobId.replace('[]', "[$index]") + assert jobId, "Missing 'jobId' argument" + return jobId.replace('[]', "[$index]") } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index e456cd32e6..353a477a80 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -199,12 +199,18 @@ class SgeExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } @Override - String getArrayIndexName() { 'SGE_TASK_ID' } + String getArrayIndexName() { + return 'SGE_TASK_ID' + } @Override - int getArrayIndexStart() { 1 } + int getArrayIndexStart() { + return 1 + } @Override - String getArrayTaskId(String jobId, int index) { "${jobId}.${index}" } + String getArrayTaskId(String jobId, int index) { + return "${jobId}.${index}" + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index beb7777009..519141ec4a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -220,12 +220,19 @@ class SlurmExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } @Override - String getArrayIndexName() { 'SLURM_ARRAY_TASK_ID' } + String getArrayIndexName() { + return 'SLURM_ARRAY_TASK_ID' + } @Override - int getArrayIndexStart() { 0 } + int getArrayIndexStart() { + return 0 + } @Override - String getArrayTaskId(String jobId, int index) { "${jobId}_${index}" } + String getArrayTaskId(String jobId, int index) { + assert jobId, "Missing 'jobId' argument" + return "${jobId}_${index}" + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy index cbd06fa71f..b85990d652 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/TaskArrayExecutor.groovy @@ -16,16 +16,19 @@ package nextflow.executor +import java.nio.file.FileSystems import java.nio.file.Path +import groovy.transform.CompileStatic +import nextflow.fusion.FusionHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskRun - /** * Interface for executors that support job arrays. * * @author Ben Sherman */ +@CompileStatic interface TaskArrayExecutor { String getName() @@ -54,4 +57,31 @@ interface TaskArrayExecutor { */ String getArrayTaskId(String jobId, int index) + default boolean isWorkDirDefaultFS() { + getWorkDir().fileSystem== FileSystems.default + } + + /** + * Get a {@link TaskHandler} work directory for the task array resolution + * + * @param handler + * @return + */ + default String getArrayWorkDir(TaskHandler handler) { + return isFusionEnabled() + ? FusionHelper.toContainerMount(handler.task.workDir).toString() + : handler.task.workDir.toUriString() + } + + default String getArrayLaunchCommand(String taskDir) { + if( isFusionEnabled() ) { + return "bash ${taskDir}/${TaskRun.CMD_RUN}" + } + else if( isWorkDirDefaultFS() ) { + return "bash ${taskDir}/${TaskRun.CMD_RUN} 2>&1 > ${taskDir}/${TaskRun.CMD_LOG}" + } + else { + throw new IllegalStateException("Executor ${getName()} does not support array jobs") + } + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 7177f99071..8576419564 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -27,7 +27,6 @@ import nextflow.executor.TaskArrayExecutor import nextflow.file.FileHelper import nextflow.util.CacheHelper import nextflow.util.Escape - /** * Task monitor that batches tasks and submits them as job arrays * to an underlying task monitor. @@ -148,8 +147,9 @@ class TaskArrayCollector { Files.createDirectories(workDir) // create wrapper script - final script = createTaskArrayScript(handlers) - + final script = createArrayTaskScript(handlers) + log.debug "Creating task array run >> $workDir\n$script" + // create config for job array final rawConfig = new HashMap(ARRAY_DIRECTIVES.size()) for( final key : ARRAY_DIRECTIVES ) { @@ -184,24 +184,21 @@ class TaskArrayCollector { * * @param array */ - protected String createTaskArrayScript(List array) { + protected String createArrayTaskScript(List array) { // get work directory and launch command for each task - final workDirs = array.collect( h -> h.getWorkDir() ) - final args = array.first().getLaunchCommand().toArray() as String[] - final cmd = Escape.cli(args).replaceAll(workDirs.first(), '\\${nxf_array_task_dir}') - - // create wrapper script - final indexName = executor.getArrayIndexName() - final indexStart = executor.getArrayIndexStart() - final arrayIndex = indexStart > 0 - ? "${indexName} - ${indexStart}" - : "${indexName}" - + final workDirs = array.collect( h -> executor.getArrayWorkDir(h) ) """ array=( ${workDirs.collect( p -> Escape.path(p) ).join(' ')} ) - export nxf_array_task_dir=\${array[${arrayIndex}]} - ${cmd} + export nxf_array_task_dir=${getArrayIndexRef()} + ${executor.getArrayLaunchCommand('$nxf_array_task_dir')} """.stripIndent().leftTrim() } + protected String getArrayIndexRef() { + final name = executor.getArrayIndexName() + final start = executor.getArrayIndexStart() + final index = start > 0 ? "${name} - ${start}" : name + return '${array[' + index + "]}" + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy index f8108da631..d1be0cf236 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy @@ -108,7 +108,7 @@ class TaskBean implements Serializable, Cloneable { Integer arrayIndexStart - List arrayWorkDirs + List arrayWorkDirs @PackageScope TaskBean() { @@ -169,7 +169,7 @@ class TaskBean implements Serializable, Cloneable { final executor = (TaskArrayExecutor)task.getProcessor().getExecutor() this.arrayIndexName = executor.getArrayIndexName() this.arrayIndexStart = executor.getArrayIndexStart() - this.arrayWorkDirs = task.children.collect( h -> h.getWorkDir() ) + this.arrayWorkDirs = task.children.collect( h -> h.task.workDir ) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 2791071ce8..f7ae9c652d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -96,16 +96,6 @@ abstract class TaskHandler { */ void prepareLauncher() {} - /** - * Get the work directory as it will be seen from the launcher script. - */ - String getWorkDir() { null } - - /** - * Get the command to execute the launcher script. - */ - List getLaunchCommand() { null } - /** * Task status attribute setter. * diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy index cad2969b9e..c2757468e1 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy @@ -344,7 +344,7 @@ class BashWrapperBuilderTest extends Specification { name: 'foo', arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, - arrayWorkDirs: [ '/work/01', '/work/02', '/work/03' ], + arrayWorkDirs: [ Path.of('/work/01'), Path.of('/work/02'), Path.of('/work/03') ], containerConfig: [enabled: true], containerImage: 'quay.io/nextflow:bash', outputFiles: ['foo.txt', '*.bar', '**/baz'] @@ -401,7 +401,7 @@ class BashWrapperBuilderTest extends Specification { name: 'task2', arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, - arrayWorkDirs: [ '/work/01', '/work/02', '/work/03' ], + arrayWorkDirs: [ Path.of('/work/01'), Path.of('/work/02'), Path.of('/work/03') ], containerConfig: [enabled: true], containerImage: 'quay.io/nextflow:bash', outputFiles: ['foo.txt', '*.bar', '**/baz'] diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 46035154cd..564a03c776 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -144,21 +144,4 @@ class GridTaskHandlerTest extends Specification { '''.stripIndent() } - def 'should get launch command' () { - given: - def task = Mock(TaskRun) { - workDir >> Path.of('/work/dir') - } - def exec = Mock(AbstractGridExecutor) - def handler = Spy(new GridTaskHandler(task, exec)) { - fusionEnabled() >> false - } - - expect: - handler.getLaunchCommand() == [ - 'bash', - '-o', 'pipefail', - '-c', 'bash /work/dir/.command.run 2>&1 | tee /work/dir/.command.log' - ] - } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy index c92c52ccb4..4ea2b7863f 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy @@ -724,5 +724,26 @@ class LsfExecutorTest extends Specification { 'a'.repeat(509) | 'nf-'.concat("a".repeat(508)) } + def 'should get array index name and start' () { + given: + def executor = Spy(LsfExecutor) + expect: + executor.getArrayIndexName() == 'LSB_JOBINDEX' + executor.getArrayIndexStart() == 1 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(LsfExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo' | 1 | 'foo[2]' + 'bar' | 2 | 'bar[3]' + } + } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy index 9307ea919f..6b0b176dea 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy @@ -312,4 +312,24 @@ class PbsExecutorTest extends Specification { !PbsExecutor.matchOptions('-x-l foo') } + def 'should get array index name and start' () { + given: + def executor = Spy(PbsExecutor) + expect: + executor.getArrayIndexName() == 'PBS_ARRAY_INDEX' + executor.getArrayIndexStart() == 0 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(PbsExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo[]' | 1 | 'foo[1]' + 'bar[]' | 2 | 'bar[2]' + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy index 72b946ab91..03a48b881c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy @@ -23,6 +23,8 @@ import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun import spock.lang.Specification +import spock.lang.Unroll + /** * * @author Lorenz Gerber @@ -264,4 +266,24 @@ class PbsProExecutorTest extends Specification { .stripIndent().leftTrim() } + def 'should get array index name and start' () { + given: + def executor = Spy(PbsProExecutor) + expect: + executor.getArrayIndexName() == 'PBS_ARRAY_INDEX' + executor.getArrayIndexStart() == 0 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(PbsProExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo[]' | 1 | 'foo[1]' + 'bar[]' | 2 | 'bar[2]' + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy index 4b236ee72f..77aab90205 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy @@ -22,6 +22,8 @@ import nextflow.processor.TaskConfig import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun import spock.lang.Specification +import spock.lang.Unroll + /** * * @author Paolo Di Tommaso @@ -385,4 +387,24 @@ class SgeExecutorTest extends Specification { } + def 'should get array index name and start' () { + given: + def executor = Spy(SgeExecutor) + expect: + executor.getArrayIndexName() == 'SGE_TASK_ID' + executor.getArrayIndexStart() == 1 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(SgeExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo' | 1 | 'foo.1' + 'bar' | 2 | 'bar.2' + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy index b7a9ca99fa..a9d270772e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy @@ -287,6 +287,27 @@ class SlurmExecutorTest extends Specification { usr exec.queueStatusCommand(null) == ['squeue','--noheader','-o','%i %t','-t','all','-u', usr] exec.queueStatusCommand('xxx') == ['squeue','--noheader','-o','%i %t','-t','all','-p','xxx','-u', usr] + } + def 'should get array index name and start' () { + given: + def executor = Spy(SlurmExecutor) + expect: + executor.getArrayIndexName() == 'SLURM_ARRAY_TASK_ID' + executor.getArrayIndexStart() == 0 } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(SlurmExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo' | 1 | 'foo_1' + 'bar' | 2 | 'bar_2' + } + } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy index e9119c3044..2b4c7fb7e9 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayCollectorTest.groovy @@ -128,11 +128,10 @@ class TaskArrayCollectorTest extends Specification { def task = Mock(TaskRun) { index >> 1 getHash() >> HashCode.fromString('0123456789abcdef') + getWorkDir() >> Paths.get('/work/foo') } def handler = Mock(TaskHandler) { getTask() >> task - getWorkDir() >> Paths.get('/work/foo') - getLaunchCommand() >> ['bash', '-o', 'pipefail', '-c', 'bash /work/foo/.command.run 2>&1 | tee /work/foo/.command.log'] } when: @@ -140,20 +139,60 @@ class TaskArrayCollectorTest extends Specification { then: 3 * exec.createTaskHandler(task) >> handler 3 * handler.prepareLauncher() + 1 * collector.createArrayTaskScript([handler, handler, handler]) >> 'the-task-array-script' and: taskArray.name == 'PROC (1)' taskArray.config.cpus == 4 taskArray.config.tag == null taskArray.processor == proc - taskArray.script == ''' - array=( /work/foo /work/foo /work/foo ) - export nxf_array_task_dir=${array[ARRAY_JOB_INDEX]} - bash -o pipefail -c 'bash ${nxf_array_task_dir}/.command.run 2>&1 | tee ${nxf_array_task_dir}/.command.log' - '''.stripIndent().leftTrim() + taskArray.script == 'the-task-array-script' and: taskArray.getArraySize() == 3 taskArray.getContainerConfig().getEnvWhitelist() == [ 'ARRAY_JOB_INDEX' ] taskArray.isContainerEnabled() == false } + def 'should get array ref' () { + given: + def processor = Mock(TaskProcessor) + def executor = Mock(DummyExecutor) { + getArrayIndexName() >> NAME + getArrayIndexStart() >> START + } + def collector = Spy(new TaskArrayCollector(processor, executor, 10)) + expect: + collector.getArrayIndexRef() == EXPECTED + + where: + NAME | START | EXPECTED + 'INDEX' | 0 | '${array[INDEX]}' + 'INDEX' | 1 | '${array[INDEX - 1]}' + 'OTHER' | 99 | '${array[OTHER - 99]}' + } + + def 'should get array launch script' () { + given: + def processor = Mock(TaskProcessor) + def executor = Spy(DummyExecutor) { isWorkDirDefaultFS()>>true } + def collector = Spy(new TaskArrayCollector(processor, executor, 10)) + and: + def h1 = Mock(TaskHandler) + def h2 = Mock(TaskHandler) + def h3 = Mock(TaskHandler) + + when: + def result = collector.createArrayTaskScript([h1,h2,h3]) + then: + executor.getArrayWorkDir(h1) >> '/work/dir/1' + executor.getArrayWorkDir(h2) >> '/work/dir/2' + executor.getArrayWorkDir(h3) >> '/work/dir/3' + and: + collector.getArrayIndexRef() >> '$array[INDEX]' + then: + result == '''\ + array=( /work/dir/1 /work/dir/2 /work/dir/3 ) + export nxf_array_task_dir=$array[INDEX] + bash $nxf_array_task_dir/.command.run 2>&1 > $nxf_array_task_dir/.command.log + '''.stripIndent(true) + } } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 13631b7b9e..a3ea252f25 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -16,6 +16,7 @@ package nextflow.cloud.aws.batch + import java.nio.file.Path import java.util.concurrent.TimeUnit @@ -24,24 +25,25 @@ import com.amazonaws.services.batch.model.AWSBatchException import com.amazonaws.services.batch.model.TerminateJobRequest import com.amazonaws.services.ecs.model.AccessDeniedException import com.amazonaws.services.logs.model.ResourceNotFoundException -import nextflow.cloud.aws.nio.S3Path import groovy.transform.CompileDynamic import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.util.logging.Slf4j import nextflow.cloud.aws.AwsClientFactory import nextflow.cloud.aws.config.AwsConfig +import nextflow.cloud.aws.nio.S3Path import nextflow.cloud.types.CloudMachineInfo import nextflow.exception.AbortOperationException import nextflow.executor.Executor import nextflow.executor.TaskArrayExecutor -import nextflow.fusion.FusionHelper import nextflow.extension.FilesEx +import nextflow.fusion.FusionHelper import nextflow.processor.ParallelPollingMonitor import nextflow.processor.TaskHandler import nextflow.processor.TaskMonitor import nextflow.processor.TaskRun import nextflow.util.Duration +import nextflow.util.Escape import nextflow.util.RateUnit import nextflow.util.ServiceName import nextflow.util.ThreadPoolHelper @@ -341,6 +343,43 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExec int getArrayIndexStart() { 0 } @Override - String getArrayTaskId(String jobId, int index) { "${jobId}:${index}" } + String getArrayTaskId(String jobId, int index) { + return "${jobId}:${index}" + } + + @Override + String getArrayLaunchCommand(String taskDir) { + if( isFusionEnabled() || isWorkDirDefaultFS() ) + return TaskArrayExecutor.super.getArrayLaunchCommand(taskDir) + else + return Escape.cli(getLaunchCommand(taskDir) as String[]) + } + + List getLaunchCommand(String s3WorkDir) { + // the cmd list to launch it + final opts = getAwsOptions() + final cmd = opts.s5cmdPath + ? s5Cmd(s3WorkDir, opts) + : s3Cmd(s3WorkDir, opts) + return ['bash','-o','pipefail','-c', cmd.toString()] + } + + static String s3Cmd(String workDir, AwsOptions opts) { + final cli = opts.getAwsCli() + final debug = opts.debug ? ' --debug' : '' + final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' + final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' + final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" + final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} ${workDir}/${TaskRun.CMD_LOG}||true; exit \$ret; }\" EXIT; $aws ${workDir}/${TaskRun.CMD_RUN} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" + return cmd + } + + static String s5Cmd(String workDir, AwsOptions opts) { + final cli = opts.getS5cmdPath() + final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' + final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' + final cmd = "trap \"{ ret=\$?; $cli cp${sse}${kms} ${TaskRun.CMD_LOG} ${workDir}/${TaskRun.CMD_LOG}||true; exit \$ret; }\" EXIT; $cli cat ${workDir}/${TaskRun.CMD_RUN} | bash 2>&1 | tee ${TaskRun.CMD_LOG}" + return cmd + } } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 2daad26833..05e8183b15 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -65,7 +65,6 @@ import nextflow.exception.ProcessSubmitException import nextflow.exception.ProcessUnrecoverableException import nextflow.executor.BashWrapperBuilder import nextflow.fusion.FusionAwareTask -import nextflow.fusion.FusionHelper import nextflow.processor.BatchContext import nextflow.processor.BatchHandler import nextflow.processor.TaskArrayRun @@ -658,31 +657,9 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler classicSubmitCli() { - // the cmd list to launch it - final opts = getAwsOptions() - final cmd = opts.s5cmdPath ? s5Cmd(opts) : s3Cmd(opts) - return ['bash','-o','pipefail','-c', cmd.toString()] - } - - protected String s3Cmd(AwsOptions opts) { - final cli = opts.getAwsCli() - final debug = opts.debug ? ' --debug' : '' - final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' - final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' - final aws = "$cli s3 cp --only-show-errors${sse}${kms}${debug}" - final cmd = "trap \"{ ret=\$?; $aws ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $aws s3:/${getWrapperFile()} - | bash 2>&1 | tee ${TaskRun.CMD_LOG}" - return cmd - } - - protected String s5Cmd(AwsOptions opts) { - final cli = opts.getS5cmdPath() - final sse = opts.storageEncryption ? " --sse $opts.storageEncryption" : '' - final kms = opts.storageKmsKeyId ? " --sse-kms-key-id $opts.storageKmsKeyId" : '' - final cmd = "trap \"{ ret=\$?; $cli cp${sse}${kms} ${TaskRun.CMD_LOG} s3:/${getLogFile()}||true; exit \$ret; }\" EXIT; $cli cat s3:/${getWrapperFile()} | bash 2>&1 | tee ${TaskRun.CMD_LOG}" - return cmd + return executor.getLaunchCommand(task.getWorkDirStr()) } - @Override List getLaunchCommand() { // final launcher command return fusionEnabled() @@ -690,13 +667,6 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler> 'batch-task' task.getConfig() >> new TaskConfig(memory: '2GB', cpus: 4, accelerator: 2) - - def handler = Spy(AwsBatchTaskHandler) + task.getWorkDirStr() >> 's3://my-bucket/work/dir' + and: + def executor = Spy(AwsBatchExecutor) { getAwsOptions()>> new AwsOptions() } + and: + def handler = Spy(new AwsBatchTaskHandler(executor: executor)) when: def req = handler.newSubmitRequest(task) then: handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'],region: 'eu-west-1')) } and: + _ * handler.getTask() >> task _ * handler.fusionEnabled() >> false 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' @@ -210,15 +213,21 @@ class AwsBatchTaskHandlerTest extends Specification { given: def task = Mock(TaskRun) - def handler = Spy(AwsBatchTaskHandler) + task.getWorkDirStr() >> 's3://my-bucket/work/dir' + and: + def executor = Spy(AwsBatchExecutor) { + getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } + } + and: + def handler = Spy(new AwsBatchTaskHandler(executor: executor)) when: def req = handler.newSubmitRequest(task) then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig() - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: + _ * handler.getTask() >> task _ * handler.fusionEnabled() >> false 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue1' @@ -234,8 +243,8 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '5 sec') - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: + _ * handler.getTask() >> task _ * handler.fusionEnabled() >> false 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue2' @@ -253,8 +262,8 @@ class AwsBatchTaskHandlerTest extends Specification { then: task.getName() >> 'batch-task' task.getConfig() >> new TaskConfig(time: '1 hour') - handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } and: + _ * handler.getTask() >> task _ * handler.fusionEnabled() >> false 1 * handler.maxSpotAttempts() >> 0 1 * handler.getJobQueue(task) >> 'queue3' @@ -911,62 +920,60 @@ class AwsBatchTaskHandlerTest extends Specification { def 'should render submit command' () { given: - def handler = Spy(AwsBatchTaskHandler) { + def executor = Spy(AwsBatchExecutor) + and: + def handler = Spy(new AwsBatchTaskHandler(executor: executor)) { fusionEnabled() >> false + getTask() >> Mock(TaskRun) { getWorkDirStr()>> 's3://work'} } when: def result = handler.getLaunchCommand() then: - handler.getAwsOptions() >> Mock(AwsOptions) { getAwsCli() >> 'aws' } - handler.getLogFile() >> Paths.get('/work/log') - handler.getWrapperFile() >> Paths.get('/work/run') + executor.getAwsOptions()>> Mock(AwsOptions) { getAwsCli() >> 'aws' } then: - result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors .command.log s3://work/log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors s3://work/run - | bash 2>&1 | tee .command.log' + result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors .command.log s3://work/.command.log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors s3://work/.command.run - | bash 2>&1 | tee .command.log' when: result = handler.getLaunchCommand() then: - handler.getAwsOptions() >> Mock(AwsOptions) { + executor.getAwsOptions() >> Mock(AwsOptions) { getAwsCli() >> 'aws'; getDebug() >> true getStorageEncryption() >> 'aws:kms' getStorageKmsKeyId() >> 'kms-key-123' } - handler.getLogFile() >> Paths.get('/work/log') - handler.getWrapperFile() >> Paths.get('/work/run') then: - result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors --sse aws:kms --sse-kms-key-id kms-key-123 --debug .command.log s3://work/log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors --sse aws:kms --sse-kms-key-id kms-key-123 --debug s3://work/run - | bash 2>&1 | tee .command.log' + result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors --sse aws:kms --sse-kms-key-id kms-key-123 --debug .command.log s3://work/.command.log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors --sse aws:kms --sse-kms-key-id kms-key-123 --debug s3://work/.command.run - | bash 2>&1 | tee .command.log' } def 'should render submit command with s5cmd' () { given: - def handler = Spy(AwsBatchTaskHandler) { + def executor = Spy(AwsBatchExecutor) + and: + def handler = Spy(new AwsBatchTaskHandler(executor: executor)) { fusionEnabled() >> false + getTask() >> Mock(TaskRun) { getWorkDirStr()>> 's3://work'} } when: def result = handler.getLaunchCommand() then: - handler.getAwsOptions() >> Mock(AwsOptions) { getS5cmdPath() >> 's5cmd' } - handler.getLogFile() >> Paths.get('/work/log') - handler.getWrapperFile() >> Paths.get('/work/run') + executor.getAwsOptions() >> Mock(AwsOptions) { getS5cmdPath() >> 's5cmd' } then: - result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; s5cmd cp .command.log s3://work/log||true; exit $ret; }" EXIT; s5cmd cat s3://work/run | bash 2>&1 | tee .command.log' + result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; s5cmd cp .command.log s3://work/.command.log||true; exit $ret; }" EXIT; s5cmd cat s3://work/.command.run | bash 2>&1 | tee .command.log' when: result = handler.getLaunchCommand() then: - handler.getAwsOptions() >> Mock(AwsOptions) { + executor.getAwsOptions() >> Mock(AwsOptions) { getS5cmdPath() >> 's5cmd --debug' getStorageEncryption() >> 'aws:kms' getStorageKmsKeyId() >> 'kms-key-123' } - handler.getLogFile() >> Paths.get('/work/log') - handler.getWrapperFile() >> Paths.get('/work/run') then: - result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; s5cmd --debug cp --sse aws:kms --sse-kms-key-id kms-key-123 .command.log s3://work/log||true; exit $ret; }" EXIT; s5cmd --debug cat s3://work/run | bash 2>&1 | tee .command.log' + result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; s5cmd --debug cp --sse aws:kms --sse-kms-key-id kms-key-123 .command.log s3://work/.command.log||true; exit $ret; }" EXIT; s5cmd --debug cat s3://work/.command.run | bash 2>&1 | tee .command.log' } diff --git a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy index 19800e3879..c9208825a7 100644 --- a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy @@ -7,11 +7,18 @@ package nextflow.executor +import java.nio.file.Path + import nextflow.Session import nextflow.SysEnv import nextflow.cloud.aws.batch.AwsBatchExecutor +import nextflow.cloud.aws.batch.AwsOptions +import nextflow.cloud.aws.util.S3PathFactory +import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun import nextflow.util.ThrottlingExecutor import spock.lang.Specification +import spock.lang.Unroll /** * @@ -65,4 +72,72 @@ class AwsBatchExecutorTest extends Specification { 1 * executor.killTask0('array-job-id') >> null } + def 'should get array index variable and start' () { + given: + def executor = Spy(AwsBatchExecutor) + expect: + executor.getArrayIndexName() == 'AWS_BATCH_JOB_ARRAY_INDEX' + executor.getArrayIndexStart() == 0 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(AwsBatchExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo' | 1 | 'foo:1' + 'bar' | 2 | 'bar:2' + } + + protected Path s3(String path) { + S3PathFactory.parse('s3:/' + path) + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(AwsBatchExecutor) { + isFusionEnabled()>>FUSION + isWorkDirDefaultFS()>>DEFAULT_FS + } + and: + def handler = Mock(TaskHandler) { + getTask() >> Mock(TaskRun) { getWorkDir() >> WORK_DIR } + } + expect: + executor.getArrayWorkDir(handler) == EXPECTED + + where: + FUSION | DEFAULT_FS | WORK_DIR | EXPECTED + false | false | s3('/foo/work/dir') | 's3://foo/work/dir' + true | false | s3('/foo/work/dir') | '/fusion/s3/foo/work/dir' + false | true | Path.of('/nfs/work') | '/nfs/work' + } + + def 'should get array launch command' (){ + given: + def executor = Spy(AwsBatchExecutor) { + isFusionEnabled()>>FUSION + isWorkDirDefaultFS()>>DEFAULT_FS + getAwsOptions() >> Mock(AwsOptions) { + getS5cmdPath() >> { S5CMD ? 's5cmd' : null } + getAwsCli() >> { 'aws' } + } + } + expect: + executor.getArrayLaunchCommand(TASK_DIR) == EXPECTED + + where: + FUSION | DEFAULT_FS | S5CMD | TASK_DIR | EXPECTED + false | false | false | 's3://foo/work/dir' | 'bash -o pipefail -c \'trap "{ ret=$?; aws s3 cp --only-show-errors .command.log s3://foo/work/dir/.command.log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors s3://foo/work/dir/.command.run - | bash 2>&1 | tee .command.log\'' + false | false | true | 's3://foo/work/dir' | 'bash -o pipefail -c \'trap "{ ret=$?; s5cmd cp .command.log s3://foo/work/dir/.command.log||true; exit $ret; }" EXIT; s5cmd cat s3://foo/work/dir/.command.run | bash 2>&1 | tee .command.log\'' + and: + true | false | false | '/fusion/work/dir' | 'bash /fusion/work/dir/.command.run' + false | true | false | '/nfs/work/dir' | 'bash /nfs/work/dir/.command.run 2>&1 > /nfs/work/dir/.command.log' + } + } diff --git a/plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy b/plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy new file mode 100644 index 0000000000..0697fb807b --- /dev/null +++ b/plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy @@ -0,0 +1,73 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.executor + +import java.nio.file.Path + +import nextflow.file.http.XPath +import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun +import spock.lang.Specification +import spock.lang.Unroll +/** + * + * @author Paolo Di Tommaso + */ +class TaskArrayExecutorTest extends Specification { + + static abstract class TestArrayExecutor implements TaskArrayExecutor { + + } + + @Unroll + def 'should get array work dir' () { + given: + def task = Mock(TaskRun) { getWorkDir() >> WORK_DIR } + def handler = Mock(TaskHandler) { getTask()>>task } + and: + def executor = Spy(TestArrayExecutor) { isFusionEnabled() >> FUSION } + when: + def result = executor.getArrayWorkDir(handler) + then: + result == EXPECTED + + where: + FUSION | WORK_DIR | EXPECTED + false | Path.of('/work/dir') | '/work/dir' + false | XPath.get('http://foo.com/work') | 'http://foo.com/work' + true | XPath.get('http://foo.com/work') | '/fusion/http/foo.com/work' + } + + @Unroll + def 'should get array launch command' () { + given: + def executor = Spy(TestArrayExecutor) { + isFusionEnabled() >> FUSION + isWorkDirDefaultFS() >> !FUSION + } + + expect: + executor.getArrayLaunchCommand(WORK_DIR) == EXPECTED + + where: + FUSION | WORK_DIR | EXPECTED + false | '/work/dir' | 'bash /work/dir/.command.run 2>&1 > /work/dir/.command.log' + true | '/fusion/work/dir' | 'bash /fusion/work/dir/.command.run' + } + +} diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy index 11a5cd9586..9655da65ab 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy @@ -17,13 +17,16 @@ package nextflow.cloud.google.batch +import static nextflow.cloud.google.batch.GoogleBatchScriptLauncher.* + import java.nio.file.Path +import com.google.cloud.storage.contrib.nio.CloudStoragePath import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.SysEnv -import nextflow.cloud.google.batch.client.BatchConfig import nextflow.cloud.google.batch.client.BatchClient +import nextflow.cloud.google.batch.client.BatchConfig import nextflow.cloud.google.batch.logging.BatchLogging import nextflow.exception.AbortOperationException import nextflow.executor.Executor @@ -35,6 +38,7 @@ import nextflow.processor.TaskMonitor import nextflow.processor.TaskPollingMonitor import nextflow.processor.TaskRun import nextflow.util.Duration +import nextflow.util.Escape import nextflow.util.ServiceName import org.pf4j.ExtensionPoint /** @@ -145,11 +149,34 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayE } @Override - String getArrayIndexName() { 'BATCH_TASK_INDEX' } + String getArrayIndexName() { + return 'BATCH_TASK_INDEX' + } + + @Override + int getArrayIndexStart() { return 0 } + + @Override + String getArrayTaskId(String jobId, int index) { + return index.toString() + } @Override - int getArrayIndexStart() { 0 } + String getArrayWorkDir(TaskHandler handler) { + return isFusionEnabled() || isWorkDirDefaultFS() + ? TaskArrayExecutor.super.getArrayWorkDir(handler) + : containerMountPath(handler.task.workDir as CloudStoragePath) + } @Override - String getArrayTaskId(String jobId, int index) { index.toString() } + String getArrayLaunchCommand(String taskDir) { + if( isFusionEnabled() || isWorkDirDefaultFS() ) { + return TaskArrayExecutor.super.getArrayLaunchCommand(taskDir) + } + else { + final cmd = List.of('/bin/bash','-o','pipefail','-c', launchCommand(taskDir)) + return Escape.cli(cmd as String[]) + } + } + } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy index 87c75c157f..c54b92f615 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchScriptLauncher.groovy @@ -64,6 +64,12 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc bean.workDir = toContainerMount(bean.workDir) bean.targetDir = toContainerMount(bean.targetDir) + // add all children work dir + if( bean.arrayWorkDirs ) { + for( Path it : bean.arrayWorkDirs ) + toContainerMount(it) + } + // remap input files to container mounted paths for( Map.Entry entry : new HashMap<>(bean.inputFiles).entrySet() ) { bean.inputFiles.put( entry.key, toContainerMount(entry.value, true) ) @@ -102,7 +108,7 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc if( path instanceof CloudStoragePath ) { buckets.add(path.bucket()) pathTrie.add( (parent ? "/${path.bucket()}${path.parent}" : "/${path.bucket()}${path}").toString() ) - final containerMount = "$MOUNT_ROOT/${path.bucket()}${path}" + final containerMount = containerMountPath(path) log.trace "Path ${FilesEx.toUriString(path)} to container mount: $containerMount" return Paths.get(containerMount) } @@ -113,7 +119,7 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc @Override String runCommand() { - "trap \"{ cp ${TaskRun.CMD_LOG} ${workDirMount}/${TaskRun.CMD_LOG}; }\" ERR; /bin/bash ${workDirMount}/${TaskRun.CMD_RUN} 2>&1 | tee ${TaskRun.CMD_LOG}" + launchCommand(workDirMount) } @Override @@ -171,4 +177,11 @@ class GoogleBatchScriptLauncher extends BashWrapperBuilder implements GoogleBatc return this } + static String launchCommand( String workDir ) { + "trap \"{ cp ${TaskRun.CMD_LOG} ${workDir}/${TaskRun.CMD_LOG}; }\" ERR; /bin/bash ${workDir}/${TaskRun.CMD_RUN} 2>&1 | tee ${TaskRun.CMD_LOG}" + } + + static String containerMountPath(CloudStoragePath path) { + return "$MOUNT_ROOT/${path.bucket()}${path}" + } } diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index c82cd6f456..61a4cdafca 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -156,20 +156,6 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { launcher.build() } - @Override - String getWorkDir() { - launcher.workDir.toString() - } - - @Override - List getLaunchCommand() { - spec0(launcher).launchCommand() - } - - List getContainerMounts() { - spec0(launcher).getContainerMounts() - } - @Override void submit() { /* @@ -254,11 +240,6 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { if( containerOptions ) container.setOptions( containerOptions ) - // add child container mounts if task is an array - if( task instanceof TaskArrayRun ) - for( TaskHandler handler : task.children ) - container.addAllVolumes( ((GoogleBatchTaskHandler)handler).getContainerMounts() ) - // task spec final env = Environment .newBuilder() diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy index adb8c748c9..e4e6167f44 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy @@ -7,12 +7,16 @@ package nextflow.cloud.google.batch +import java.nio.file.Path + +import com.google.cloud.storage.contrib.nio.CloudStorageFileSystem import nextflow.Session import nextflow.SysEnv import nextflow.cloud.google.batch.client.BatchClient +import nextflow.processor.TaskHandler +import nextflow.processor.TaskRun import spock.lang.Specification import spock.lang.Unroll - /** * * @author Paolo Di Tommaso @@ -75,4 +79,68 @@ class GoogleBatchExecutorTest extends Specification { then: 1 * client.deleteJob('job-id') } + + def 'should get array index variable and start' () { + given: + def executor = Spy(GoogleBatchExecutor) + expect: + executor.getArrayIndexName() == 'BATCH_TASK_INDEX' + executor.getArrayIndexStart() == 0 + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(GoogleBatchExecutor) + expect: + executor.getArrayTaskId(JOB_ID, TASK_INDEX) == EXPECTED + + where: + JOB_ID | TASK_INDEX | EXPECTED + 'foo' | 1 | '1' + 'bar' | 2 | '2' + } + + protected Path gs(String str) { + def b = str.tokenize('/')[0] + def p = str.tokenize('/')[1..-1].join('/') + CloudStorageFileSystem.forBucket(b).getPath('/'+p) + } + + @Unroll + def 'should get array task id' () { + given: + def executor = Spy(GoogleBatchExecutor) { + isFusionEnabled()>>FUSION + isWorkDirDefaultFS()>>DEFAULT_FS + } + and: + def handler = Mock(TaskHandler) { + getTask() >> Mock(TaskRun) { getWorkDir() >> WORK_DIR } + } + expect: + executor.getArrayWorkDir(handler) == EXPECTED + + where: + FUSION | DEFAULT_FS | WORK_DIR | EXPECTED + false | false | gs('/foo/work/dir') | '/mnt/disks/foo/work/dir' + true | false | gs('/foo/work/dir') | '/fusion/gs/foo/work/dir' + false | true | Path.of('/nfs/work') | '/nfs/work' + } + + def 'should get array launch command' (){ + given: + def executor = Spy(GoogleBatchExecutor) { + isFusionEnabled()>>FUSION + isWorkDirDefaultFS()>>DEFAULT_FS + } + expect: + executor.getArrayLaunchCommand(TASK_DIR) == EXPECTED + + where: + FUSION | DEFAULT_FS | TASK_DIR | EXPECTED + false | false | 'gs://foo/work/dir' | '/bin/bash -o pipefail -c \'trap "{ cp .command.log gs://foo/work/dir/.command.log; }" ERR; /bin/bash gs://foo/work/dir/.command.run 2>&1 | tee .command.log\'' + true | false | '/fusion/work/dir' | 'bash /fusion/work/dir/.command.run' + false | true | '/nfs/work/dir' | 'bash /nfs/work/dir/.command.run 2>&1 > /nfs/work/dir/.command.log' + } } From a77cfd577497225a800511968d96adba17f70dbc Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 3 May 2024 11:24:08 -0500 Subject: [PATCH 87/97] Move misplaced test Signed-off-by: Ben Sherman --- .../test/groovy/nextflow/processor}/TaskArrayExecutorTest.groovy | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename {plugins/nf-amazon/src/test/nextflow/executor => modules/nextflow/src/test/groovy/nextflow/processor}/TaskArrayExecutorTest.groovy (100%) diff --git a/plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayExecutorTest.groovy similarity index 100% rename from plugins/nf-amazon/src/test/nextflow/executor/TaskArrayExecutorTest.groovy rename to modules/nextflow/src/test/groovy/nextflow/processor/TaskArrayExecutorTest.groovy From 7e411fd60056a1799d9e9d9f045f8d9efb58440f Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 3 May 2024 11:24:43 -0500 Subject: [PATCH 88/97] Disable job log when submitting job arrays to grid schedulers Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/executor/CrgExecutor.groovy | 8 ++++++-- .../src/main/groovy/nextflow/executor/LsfExecutor.groovy | 4 +++- .../src/main/groovy/nextflow/executor/PbsExecutor.groovy | 7 +++++-- .../main/groovy/nextflow/executor/PbsProExecutor.groovy | 7 +++++-- .../src/main/groovy/nextflow/executor/SgeExecutor.groovy | 8 ++++++-- .../main/groovy/nextflow/executor/SlurmExecutor.groovy | 7 ++++++- .../test/groovy/nextflow/executor/CrgExecutorTest.groovy | 2 -- .../test/groovy/nextflow/executor/LsfExecutorTest.groovy | 1 - .../test/groovy/nextflow/executor/PbsExecutorTest.groovy | 2 -- .../groovy/nextflow/executor/PbsProExecutorTest.groovy | 2 -- .../test/groovy/nextflow/executor/SgeExecutorTest.groovy | 2 -- .../groovy/nextflow/executor/SlurmExecutorTest.groovy | 1 - 12 files changed, 31 insertions(+), 20 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy index 8a395b297c..2ed880e15b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/CrgExecutor.groovy @@ -48,8 +48,12 @@ class CrgExecutor extends SgeExecutor { } result << '-N' << getJobNameFor(task) - result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) - result << '-j' << 'y' + + if( task !instanceof TaskArrayRun ) { + result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) + result << '-j' << 'y' + } + result << '-terse' << '' // note: directive need to be returned as pairs /* diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy index 32f31c53e4..409ef31449 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/LsfExecutor.groovy @@ -69,7 +69,9 @@ class LsfExecutor extends AbstractGridExecutor implements TaskArrayExecutor { */ protected List getDirectives(TaskRun task, List result) { - result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() + if( task !instanceof TaskArrayRun ) { + result << '-o' << task.workDir.resolve(TaskRun.CMD_LOG).toString() + } // add other parameters (if any) if( task.config.queue ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy index 0688104776..dfab80c8d7 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsExecutor.groovy @@ -50,8 +50,11 @@ class PbsExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } result << '-N' << getJobNameFor(task) - result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) - result << '-j' << 'oe' + + if( task !instanceof TaskArrayRun ) { + result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) + result << '-j' << 'oe' + } // the requested queue name if( task.config.queue ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy index 5d044ceaea..2562b05cd4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/PbsProExecutor.groovy @@ -59,8 +59,11 @@ class PbsProExecutor extends PbsExecutor { } result << '-N' << getJobNameFor(task) - result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) - result << '-j' << 'oe' + + if( task !instanceof TaskArrayRun ) { + result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) + result << '-j' << 'oe' + } // the requested queue name if( task.config.queue ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy index 353a477a80..bf9b99e724 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SgeExecutor.groovy @@ -44,8 +44,12 @@ class SgeExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } result << '-N' << getJobNameFor(task) - result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) - result << '-j' << 'y' + + if( task !instanceof TaskArrayRun ) { + result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) + result << '-j' << 'y' + } + result << '-terse' << '' // note: directive need to be returned as pairs /* diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy index 519141ec4a..7993537827 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/SlurmExecutor.groovy @@ -62,7 +62,12 @@ class SlurmExecutor extends AbstractGridExecutor implements TaskArrayExecutor { } result << '-J' << getJobNameFor(task) - result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) // -o OUTFILE and no -e option => stdout and stderr merged to stdout/OUTFILE + + if( task !instanceof TaskArrayRun ) { + // -o OUTFILE and no -e option => stdout and stderr merged to stdout/OUTFILE + result << '-o' << quote(task.workDir.resolve(TaskRun.CMD_LOG)) + } + result << '--no-requeue' << '' // note: directive need to be returned as pairs if( !hasSignalOpt(task.config) ) { diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy index 211201a1db..6036be4980 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/CrgExecutorTest.groovy @@ -351,8 +351,6 @@ class CrgExecutorTest extends Specification { executor.getHeaders(task) == ''' #$ -t 1-5 #$ -N nf-mapping_tag - #$ -o /abc/.command.log - #$ -j y #$ -terse #$ -notify ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy index 4ea2b7863f..1ebcb4a8ae 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/LsfExecutorTest.groovy @@ -294,7 +294,6 @@ class LsfExecutorTest extends Specification { } then: executor.getHeaders(taskArray) == ''' - #BSUB -o /scratch/.command.log #BSUB -J "nf-mapping_hola[1-5]" ''' .stripIndent().leftTrim() diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy index 6b0b176dea..dcd0947234 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsExecutorTest.groovy @@ -170,8 +170,6 @@ class PbsExecutorTest extends Specification { executor.getHeaders(taskArray) == ''' #PBS -J 0-4 #PBS -N nf-task_name - #PBS -o /work/dir/.command.log - #PBS -j oe ''' .stripIndent().leftTrim() } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy index 03a48b881c..142187c098 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/PbsProExecutorTest.groovy @@ -190,8 +190,6 @@ class PbsProExecutorTest extends Specification { executor.getDirectives(task, []) == [ '-J', '0-4', '-N', 'nf-foo', - '-o', '/foo/bar/.command.log', - '-j', 'oe', ] } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy index 77aab90205..7466c83885 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SgeExecutorTest.groovy @@ -207,8 +207,6 @@ class SgeExecutorTest extends Specification { executor.getHeaders(taskArray) == ''' #$ -t 1-5 #$ -N nf-the_task_name - #$ -o /abc/.command.log - #$ -j y #$ -terse #$ -notify ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy index a9d270772e..021fc907de 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/SlurmExecutorTest.groovy @@ -212,7 +212,6 @@ class SlurmExecutorTest extends Specification { executor.getHeaders(taskArray) == ''' #SBATCH --array 0-4 #SBATCH -J nf-the_task_name - #SBATCH -o /work/path/.command.log #SBATCH --no-requeue #SBATCH --signal B:USR2@30 ''' From 4e414365fbf966cb7c85dd4ea2de1a86ddae8b4e Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Tue, 7 May 2024 12:10:59 +0200 Subject: [PATCH 89/97] Strengthen aws batch deletion logi Signed-off-by: Paolo Di Tommaso --- .../cloud/aws/batch/AwsBatchExecutor.groovy | 35 ++++++----------- .../aws/batch/AwsBatchTaskHandler.groovy | 22 ++++++++++- .../aws/batch/AwsBatchTaskHandlerTest.groovy | 38 +++++++++++++++++-- .../executor/AwsBatchExecutorTest.groovy | 32 +++++++--------- 4 files changed, 79 insertions(+), 48 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index a3ea252f25..9def1389a8 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -16,13 +16,11 @@ package nextflow.cloud.aws.batch - import java.nio.file.Path import java.util.concurrent.TimeUnit import com.amazonaws.services.batch.AWSBatch import com.amazonaws.services.batch.model.AWSBatchException -import com.amazonaws.services.batch.model.TerminateJobRequest import com.amazonaws.services.ecs.model.AccessDeniedException import com.amazonaws.services.logs.model.ResourceNotFoundException import groovy.transform.CompileDynamic @@ -86,7 +84,7 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExec private AwsOptions awsOptions - private Set deletedJobs = [] as Set + private final Set deletedJobs = new HashSet<>() AwsOptions getAwsOptions() { awsOptions } @@ -271,27 +269,16 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExec @PackageScope ThrottlingExecutor getReaper() { reaper } - void killTask(String jobId) { - // extract job array id - if( jobId.contains(':') ) - jobId = jobId.split(':')[0] - - // prevent duplicate delete requests on the same job - if( jobId in deletedJobs ) - return - else - deletedJobs.add(jobId) - - // submit terminate request - reaper.submit({ killTask0(jobId) }) - } - - protected void killTask0(String jobId) { - final req = new TerminateJobRequest() - .withJobId(jobId) - .withReason('Job killed by NF') - final resp = client.terminateJob(req) - log.debug "[AWS BATCH] killing job=$jobId; response=$resp" + boolean shouldDeleteJob(String jobId) { + if( jobId in deletedJobs ) { + // if the job is already in the list if has been already delete + return false + } + synchronized (deletedJobs) { + // add the job id to the set of delete jobs, if it's a new id, the `add` method + // returns true therefore the job should be deleted + return deletedJobs.add(jobId) + } } CloudMachineInfo getMachineInfoByQueueAndTaskArn(String queue, String taskArn) { diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index 05e8183b15..c7d2ef5910 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -53,6 +53,7 @@ import com.amazonaws.services.batch.model.RetryStrategy import com.amazonaws.services.batch.model.RuntimePlatform import com.amazonaws.services.batch.model.SubmitJobRequest import com.amazonaws.services.batch.model.SubmitJobResult +import com.amazonaws.services.batch.model.TerminateJobRequest import com.amazonaws.services.batch.model.Volume import groovy.transform.Canonical import groovy.transform.CompileStatic @@ -295,8 +296,27 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler> true + and: + 1 * handler.terminateJob('job1') >> null + when: + handler.@jobId = 'job1:task2' + handler.kill() + then: + 1 * executor.shouldDeleteJob('job1') >> true + and: + 1 * handler.terminateJob('job1') >> null + + when: + handler.@jobId = 'job1:task2' + handler.kill() + then: + 1 * executor.shouldDeleteJob('job1') >> false + and: + 0 * handler.terminateJob('job1') >> null } def 'should create the trace record' () { @@ -1070,4 +1085,19 @@ class AwsBatchTaskHandlerTest extends Specification { 16 | 100000 | 106496 16 | 200000 | 122880 } + + @Unroll + def 'should normalise job id' () { + given: + def handler = Spy(AwsBatchTaskHandler) + + expect: + handler.normaliseJobId(JOB_ID) == EXPECTED + + where: + JOB_ID | EXPECTED + null | null + 'job1' | 'job1' + 'job1:task2' | 'job1' + } } diff --git a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy index c9208825a7..5dbef4439e 100644 --- a/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/executor/AwsBatchExecutorTest.groovy @@ -16,10 +16,8 @@ import nextflow.cloud.aws.batch.AwsOptions import nextflow.cloud.aws.util.S3PathFactory import nextflow.processor.TaskHandler import nextflow.processor.TaskRun -import nextflow.util.ThrottlingExecutor import spock.lang.Specification import spock.lang.Unroll - /** * * @author Paolo Di Tommaso @@ -51,25 +49,21 @@ class AwsBatchExecutorTest extends Specification { } - def 'should kill tasks' () { + def 'should validate shouldDeleteJob method' () { given: - def reaper = Mock(ThrottlingExecutor) { - submit(_) >> { Closure cl -> cl() } - } def executor = Spy(AwsBatchExecutor) - executor.@reaper = reaper - - when: - executor.killTask('job-id') - executor.killTask('job-id') - then: - 1 * executor.killTask0('job-id') >> null - - when: - executor.killTask('array-job-id:0') - executor.killTask('array-job-id:1') - then: - 1 * executor.killTask0('array-job-id') >> null + + expect: + executor.shouldDeleteJob('job-1') + executor.shouldDeleteJob('job-2') + executor.shouldDeleteJob('job-3') + and: + !executor.shouldDeleteJob('job-1') + !executor.shouldDeleteJob('job-1') + !executor.shouldDeleteJob('job-2') + !executor.shouldDeleteJob('job-2') + !executor.shouldDeleteJob('job-3') + !executor.shouldDeleteJob('job-3') } def 'should get array index variable and start' () { From e8b701da5ae0dded9760775158520807dbb0dcc1 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Tue, 7 May 2024 12:19:08 +0200 Subject: [PATCH 90/97] Strengthen google batch deletion logic Signed-off-by: Paolo Di Tommaso --- .../google/batch/GoogleBatchExecutor.groovy | 21 +++++----- .../batch/GoogleBatchTaskHandler.groovy | 3 +- .../batch/GoogleBatchExecutorTest.groovy | 30 ++++++++------- .../batch/GoogleBatchTaskHandlerTest.groovy | 38 +++++++++++++++++++ 4 files changed, 68 insertions(+), 24 deletions(-) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy index 9655da65ab..c78e60e7a3 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchExecutor.groovy @@ -56,7 +56,7 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayE private Path remoteBinDir private BatchLogging logging - private Set deletedJobs = [] as Set + private final Set deletedJobs = new HashSet<>() BatchClient getClient() { return client } BatchConfig getConfig() { return config } @@ -137,15 +137,16 @@ class GoogleBatchExecutor extends Executor implements ExtensionPoint, TaskArrayE return Boolean.parseBoolean(SysEnv.get('NXF_CLOUDINFO_ENABLED', 'true') ) } - void killTask(String jobId) { - // prevent duplicate delete requests on the same job - if( jobId in deletedJobs ) - return - else - deletedJobs.add(jobId) - - // delete job - client.deleteJob(jobId) + boolean shouldDeleteJob(String jobId) { + if( jobId in deletedJobs ) { + // if the job is already in the list if has been already delete + return false + } + synchronized (deletedJobs) { + // add the job id to the set of delete jobs, if it's a new id, the `add` method + // returns true therefore the job should be deleted + return deletedJobs.add(jobId) + } } @Override diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 61a4cdafca..7006b007fe 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -531,7 +531,8 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { void kill() { if( isActive() ) { log.trace "[GOOGLE BATCH] Process `${task.lazyName()}` - deleting job name=$jobId" - executor.killTask(jobId) + if( executor.shouldDeleteJob(jobId) ) + client.deleteJob(jobId) } else { log.debug "[GOOGLE BATCH] Process `${task.lazyName()}` - invalid delete action" diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy index e4e6167f44..ed05765ff8 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchExecutorTest.groovy @@ -12,7 +12,6 @@ import java.nio.file.Path import com.google.cloud.storage.contrib.nio.CloudStorageFileSystem import nextflow.Session import nextflow.SysEnv -import nextflow.cloud.google.batch.client.BatchClient import nextflow.processor.TaskHandler import nextflow.processor.TaskRun import spock.lang.Specification @@ -68,18 +67,6 @@ class GoogleBatchExecutorTest extends Specification { [NXF_CLOUDINFO_ENABLED:'false'] | false } - def 'should kill tasks' () { - given: - def client = Mock(BatchClient) - def executor = new GoogleBatchExecutor(client: client) - - when: - executor.killTask('job-id') - executor.killTask('job-id') - then: - 1 * client.deleteJob('job-id') - } - def 'should get array index variable and start' () { given: def executor = Spy(GoogleBatchExecutor) @@ -143,4 +130,21 @@ class GoogleBatchExecutorTest extends Specification { true | false | '/fusion/work/dir' | 'bash /fusion/work/dir/.command.run' false | true | '/nfs/work/dir' | 'bash /nfs/work/dir/.command.run 2>&1 > /nfs/work/dir/.command.log' } + + def 'should validate shouldDeleteJob method' () { + given: + def executor = Spy(GoogleBatchExecutor) + + expect: + executor.shouldDeleteJob('job-1') + executor.shouldDeleteJob('job-2') + executor.shouldDeleteJob('job-3') + and: + !executor.shouldDeleteJob('job-1') + !executor.shouldDeleteJob('job-1') + !executor.shouldDeleteJob('job-2') + !executor.shouldDeleteJob('job-2') + !executor.shouldDeleteJob('job-3') + !executor.shouldDeleteJob('job-3') + } } diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy index 97fafec5f1..005a91d4ca 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/GoogleBatchTaskHandlerTest.groovy @@ -539,4 +539,42 @@ class GoogleBatchTaskHandlerTest extends Specification { cleanup: SysEnv.pop() } + + def 'should kill a job' () { + given: + def client = Mock(BatchClient) + def executor = Mock(GoogleBatchExecutor) + def task = Mock(TaskRun) + def handler = Spy(GoogleBatchTaskHandler) + handler.@executor = executor + handler.@client = client + handler.task = task + + when: + handler.@jobId = 'job1' + handler.kill() + then: + handler.isActive() >> false + 0 * executor.shouldDeleteJob('job1') >> true + and: + 0 * client.deleteJob('job1') >> null + + when: + handler.@jobId = 'job1' + handler.kill() + then: + handler.isActive() >> true + 1 * executor.shouldDeleteJob('job1') >> true + and: + 1 * client.deleteJob('job1') >> null + + when: + handler.@jobId = 'job1' + handler.kill() + then: + handler.isActive() >> true + 1 * executor.shouldDeleteJob('job1') >> false + and: + 0 * client.deleteJob('job1') >> null + } } From b7ce0a11197052d31cb07cdcf061e929b7b8aa81 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 7 May 2024 15:47:51 -0500 Subject: [PATCH 91/97] minor edits Signed-off-by: Ben Sherman --- .../groovy/nextflow/processor/TaskArrayCollector.groovy | 2 +- .../main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy | 4 ++-- .../nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy | 2 +- .../cloud/google/batch/GoogleBatchExecutor.groovy | 8 +++++--- .../nextflow/cloud/google/batch/client/BatchClient.groovy | 1 - 5 files changed, 9 insertions(+), 8 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy index 8576419564..b5a40db98d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskArrayCollector.groovy @@ -198,7 +198,7 @@ class TaskArrayCollector { final name = executor.getArrayIndexName() final start = executor.getArrayIndexStart() final index = start > 0 ? "${name} - ${start}" : name - return '${array[' + index + "]}" + return '${array[' + index + ']}' } } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy index 9def1389a8..5ebca92283 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchExecutor.groovy @@ -271,11 +271,11 @@ class AwsBatchExecutor extends Executor implements ExtensionPoint, TaskArrayExec boolean shouldDeleteJob(String jobId) { if( jobId in deletedJobs ) { - // if the job is already in the list if has been already delete + // if the job is already in the list if has been already deleted return false } synchronized (deletedJobs) { - // add the job id to the set of delete jobs, if it's a new id, the `add` method + // add the job id to the set of deleted jobs, if it's a new id, the `add` method // returns true therefore the job should be deleted return deletedJobs.add(jobId) } diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index c7d2ef5910..bbfe26f5ab 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -306,7 +306,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler Date: Tue, 7 May 2024 15:48:57 -0500 Subject: [PATCH 92/97] Update google batch logging to select task logs Signed-off-by: Ben Sherman --- .../batch/GoogleBatchTaskHandler.groovy | 4 ++-- .../google/batch/logging/BatchLogging.groovy | 19 +++++++++---------- .../batch/logging/BatchLoggingTest.groovy | 4 ++-- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy index 7006b007fe..fdaee0a74f 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/GoogleBatchTaskHandler.groovy @@ -484,8 +484,8 @@ class GoogleBatchTaskHandler extends TaskHandler implements FusionAwareTask { if( task.exitStatus == null ) task.exitStatus = readExitFile() if( state == 'FAILED' ) { - task.stdout = executor.logging.stdout(uid) ?: outputFile - task.stderr = executor.logging.stderr(uid) ?: errorFile + task.stdout = executor.logging.stdout(uid, taskId) ?: outputFile + task.stderr = executor.logging.stderr(uid, taskId) ?: errorFile } else { task.stdout = outputFile diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/batch/logging/BatchLogging.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/batch/logging/BatchLogging.groovy index da394dd39d..04091f8b51 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/batch/logging/BatchLogging.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/batch/logging/BatchLogging.groovy @@ -47,30 +47,29 @@ class BatchLogging implements Closeable { this.opts = LoggingOptions .newBuilder() .setCredentials(creds) .setProjectId(this.projectId) .build() } - String stdout(String jobId) { - return safeLogs(jobId,0) + String stdout(String uid, String taskId) { + return safeLogs(uid, taskId, 0) } - String stderr(String jobId) { - return safeLogs(jobId,1) + String stderr(String uid, String taskId) { + return safeLogs(uid, taskId, 1) } - protected String safeLogs(String jobId, int index) { + protected String safeLogs(String uid, String taskId, int index) { try { - return fetchLogs(jobId)[index] + return fetchLogs(uid, taskId)[index] } catch (Exception e) { - log.warn("Cannot read logs for Batch job '$jobId' - cause: ${e.message}", e) + log.warn("Cannot read logs for Batch job '$uid/$taskId' - cause: ${e.message}", e) return null } } @Memoized(maxCacheSize = 1000) - @PackageScope List fetchLogs(String uid) { + @PackageScope List fetchLogs(String uid, String taskId) { final stdout = new StringBuilder() final stderr = new StringBuilder() - // use logging here - final filter = "resource.type=generic_task OR resource.type=\"batch.googleapis.com/Job\" AND logName=\"projects/${projectId}/logs/batch_task_logs\" AND labels.job_uid=$uid" + final filter = "resource.type=generic_task OR resource.type=\"batch.googleapis.com/Job\" AND logName=\"projects/${projectId}/logs/batch_task_logs\" AND labels.job_uid=$uid AND labels.task_id=$uid-group0-$taskId" final entries = loggingService().listLogEntries( Logging.EntryListOption.filter(filter), Logging.EntryListOption.pageSize(1000) ) diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy index a4a4478172..625a5336a0 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/batch/logging/BatchLoggingTest.groovy @@ -141,8 +141,8 @@ class BatchLoggingTest extends Specification { state in ['SUCCEEDED', 'FAILED'] when: - def stdout = logClient.stdout(uid) - def stderr = logClient.stderr(uid) + def stdout = logClient.stdout(uid, '0') + def stderr = logClient.stderr(uid, '0') log.debug "STDOUT: $stdout" log.debug "STDERR: $stderr" then: From 247b72161de11ee904200a2d0de819be63a330ae Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 7 May 2024 16:34:19 -0500 Subject: [PATCH 93/97] Fix race condition in LogsCheckpoint Signed-off-by: Ben Sherman --- .../src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy index 9b57a7a166..53b73e37f2 100644 --- a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy +++ b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy @@ -36,7 +36,7 @@ class LogsCheckpoint implements TraceObserver { private Session session private Map config - private volatile Thread thread + private Thread thread private Duration interval private LogsHandler handler private volatile boolean terminated @@ -67,7 +67,7 @@ class LogsCheckpoint implements TraceObserver { protected void run() { log.debug "Starting logs checkpoint thread - interval: ${interval}" try { - while( !terminated && !thread.isInterrupted() ) { + while( !terminated ) { // just wait the declared delay await(interval) // checkpoint the logs From e08667469da46a232de813f36db108911fd1c225 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Wed, 8 May 2024 11:01:42 +0200 Subject: [PATCH 94/97] Revert "Fix race condition in LogsCheckpoint" This reverts commit 247b72161de11ee904200a2d0de819be63a330ae. --- .../src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy index 53b73e37f2..9b57a7a166 100644 --- a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy +++ b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy @@ -36,7 +36,7 @@ class LogsCheckpoint implements TraceObserver { private Session session private Map config - private Thread thread + private volatile Thread thread private Duration interval private LogsHandler handler private volatile boolean terminated @@ -67,7 +67,7 @@ class LogsCheckpoint implements TraceObserver { protected void run() { log.debug "Starting logs checkpoint thread - interval: ${interval}" try { - while( !terminated ) { + while( !terminated && !thread.isInterrupted() ) { // just wait the declared delay await(interval) // checkpoint the logs From 6b8bb494a87177f751f1fa7738efbf4fd92c3745 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 9 May 2024 10:57:39 +0200 Subject: [PATCH 95/97] Restore grid handler names [ci fast] Signed-off-by: Paolo Di Tommaso --- .../groovy/nextflow/executor/GridTaskHandler.groovy | 8 ++++---- .../nextflow/executor/GridTaskHandlerTest.groovy | 13 ++++++------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy index 78cf4be967..11d310b600 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/GridTaskHandler.groovy @@ -196,7 +196,7 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { // -- wait the the process completes final result = process.text final exitStatus = process.waitFor() - final cmd = submitCmd0(builder,pipeScript) + final cmd = launchCmd0(builder,pipeScript) if( exitStatus ) { throw new ProcessNonZeroExitStatusException("Failed to submit process to grid scheduler for execution", result, exitStatus, cmd) @@ -244,12 +244,12 @@ class GridTaskHandler extends TaskHandler implements FusionAwareTask { return result } - protected String submitCmd0(ProcessBuilder builder, String pipeScript) { + protected String launchCmd0(ProcessBuilder builder, String pipeScript) { def result = CmdLineHelper.toLine(builder.command()) if( pipeScript ) { - result = "cat << 'SUBMIT_COMMAND_EOF' | ${result}\n" + result = "cat << 'LAUNCH_COMMAND_EOF' | ${result}\n" result += pipeScript.trim() + '\n' - result += 'SUBMIT_COMMAND_EOF\n' + result += 'LAUNCH_COMMAND_EOF\n' } return result } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy index 564a03c776..b406499025 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/GridTaskHandlerTest.groovy @@ -129,19 +129,18 @@ class GridTaskHandlerTest extends Specification { '''.stripIndent(true) } - def 'should get submit command' () { + def 'should create launch command' () { given: - def handler = Spy(GridTaskHandler) + def exec = Spy(GridTaskHandler) expect: - handler.submitCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' + exec.launchCmd0(new ProcessBuilder().command(['qsub', '/some/file']), null) == 'qsub /some/file' and: - handler.submitCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == + exec.launchCmd0(new ProcessBuilder().command(['qsub']), 'docker run /some/file') == '''\ - cat << 'SUBMIT_COMMAND_EOF' | qsub + cat << 'LAUNCH_COMMAND_EOF' | qsub docker run /some/file - SUBMIT_COMMAND_EOF + LAUNCH_COMMAND_EOF '''.stripIndent() } - } From 912f90d7bddfcdd853c406527d20a5285fdebf18 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 9 May 2024 11:14:52 +0200 Subject: [PATCH 96/97] Revert submit naming Signed-off-by: Paolo Di Tommaso --- .../aws/batch/AwsBatchTaskHandler.groovy | 4 ++-- .../aws/batch/AwsBatchTaskHandlerTest.groovy | 22 +++++++++---------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy index bbfe26f5ab..ff20ca8da3 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/batch/AwsBatchTaskHandler.groovy @@ -680,7 +680,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler getLaunchCommand() { + protected List getSubmitCommand() { // final launcher command return fusionEnabled() ? fusionSubmitCli() @@ -748,7 +748,7 @@ class AwsBatchTaskHandler extends TaskHandler implements BatchHandler(5) final container = new ContainerOverrides() - container.command = getLaunchCommand() + container.command = getSubmitCommand() // set the task memory final cpus = task.config.getCpus() final mem = task.config.getMemory() diff --git a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy index 47d93162be..10cde9f3c4 100644 --- a/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy +++ b/plugins/nf-amazon/src/test/nextflow/cloud/aws/batch/AwsBatchTaskHandlerTest.groovy @@ -90,7 +90,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch:[cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -111,7 +111,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: req = handler.newSubmitRequest(task) then: - 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 0 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'], region: 'eu-west-1')) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -141,7 +141,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'],client: [storageEncryption: 'AES256'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -158,7 +158,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req2 = handler.newSubmitRequest(task) then: - 1 * handler.getLaunchCommand() >> ['bash', '-c', 'something'] + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'something'] 1 * handler.maxSpotAttempts() >> 5 _ * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws',schedulingPriority: 9999,shareIdentifier: 'priority/high'], client:[storageEncryption: 'AES256', debug: true])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -294,7 +294,7 @@ class AwsBatchTaskHandlerTest extends Specification { handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws', retryMode: 'adaptive', maxTransferAttempts: 10])) } and: _ * handler.fusionEnabled() >> false - 1 * handler.getLaunchCommand() >> ['bash','-c','foo'] + 1 * handler.getSubmitCommand() >> ['bash', '-c', 'foo'] 1 * handler.maxSpotAttempts() >> 3 1 * handler.getJobQueue(task) >> 'queue1' 1 * handler.getJobDefinition(task) >> 'job-def:1' @@ -943,14 +943,14 @@ class AwsBatchTaskHandlerTest extends Specification { } when: - def result = handler.getLaunchCommand() + def result = handler.getSubmitCommand() then: executor.getAwsOptions()>> Mock(AwsOptions) { getAwsCli() >> 'aws' } then: result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; aws s3 cp --only-show-errors .command.log s3://work/.command.log||true; exit $ret; }" EXIT; aws s3 cp --only-show-errors s3://work/.command.run - | bash 2>&1 | tee .command.log' when: - result = handler.getLaunchCommand() + result = handler.getSubmitCommand() then: executor.getAwsOptions() >> Mock(AwsOptions) { getAwsCli() >> 'aws'; @@ -973,14 +973,14 @@ class AwsBatchTaskHandlerTest extends Specification { } when: - def result = handler.getLaunchCommand() + def result = handler.getSubmitCommand() then: executor.getAwsOptions() >> Mock(AwsOptions) { getS5cmdPath() >> 's5cmd' } then: result.join(' ') == 'bash -o pipefail -c trap "{ ret=$?; s5cmd cp .command.log s3://work/.command.log||true; exit $ret; }" EXIT; s5cmd cat s3://work/.command.run | bash 2>&1 | tee .command.log' when: - result = handler.getLaunchCommand() + result = handler.getSubmitCommand() then: executor.getAwsOptions() >> Mock(AwsOptions) { getS5cmdPath() >> 's5cmd --debug' @@ -1006,7 +1006,7 @@ class AwsBatchTaskHandlerTest extends Specification { when: def req = handler.newSubmitRequest(task) then: - 1 * handler.getLaunchCommand() >> ['sh','-c','hello'] + 1 * handler.getSubmitCommand() >> ['sh', '-c', 'hello'] 1 * handler.maxSpotAttempts() >> 5 1 * handler.getAwsOptions() >> { new AwsOptions(awsConfig: new AwsConfig(batch: [cliPath: '/bin/aws'])) } 1 * handler.getJobQueue(task) >> 'queue1' @@ -1037,7 +1037,7 @@ class AwsBatchTaskHandlerTest extends Specification { } when: - def result = handler.getLaunchCommand() + def result = handler.getSubmitCommand() then: result.join(' ') == '/usr/bin/fusion bash /fusion/s3/my-bucket/work/dir/.command.run' } From 4b45aa031c9fd32625595b26c33fcad1acbe63d7 Mon Sep 17 00:00:00 2001 From: Paolo Di Tommaso Date: Thu, 9 May 2024 11:15:09 +0200 Subject: [PATCH 97/97] Revert volatile change Signed-off-by: Paolo Di Tommaso --- .../src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy index 9b57a7a166..ee353d66b4 100644 --- a/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy +++ b/plugins/nf-tower/src/main/io/seqera/tower/plugin/LogsCheckpoint.groovy @@ -36,7 +36,7 @@ class LogsCheckpoint implements TraceObserver { private Session session private Map config - private volatile Thread thread + private Thread thread private Duration interval private LogsHandler handler private volatile boolean terminated