From 47d01688e074f56e7492c06de0d303259df95797 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 27 Mar 2023 14:01:09 -0500 Subject: [PATCH 01/34] Add initial task graph and metadata json file Signed-off-by: Ben Sherman --- docs/config.rst | 1 + .../src/main/groovy/nextflow/Session.groovy | 9 +++ .../groovy/nextflow/dag/ConcreteDAG.groovy | 76 +++++++++++++++++++ .../nextflow/dag/CytoscapeHtmlRenderer.groovy | 2 +- .../nextflow/dag/CytoscapeJsRenderer.groovy | 2 +- .../src/main/groovy/nextflow/dag/DAG.groovy | 2 +- .../groovy/nextflow/dag/DagRenderer.groovy | 15 +++- .../groovy/nextflow/dag/DotRenderer.groovy | 2 +- .../groovy/nextflow/dag/GexfRenderer.groovy | 2 +- .../nextflow/dag/GraphVizRenderer.groovy | 2 +- .../nextflow/dag/MermaidRenderer.groovy | 29 +++++-- .../groovy/nextflow/dag/NodeMarker.groovy | 18 ++++- .../nextflow/processor/TaskHandler.groovy | 29 +++++++ .../nextflow/processor/TaskProcessor.groovy | 10 ++- .../groovy/nextflow/processor/TaskRun.groovy | 1 + .../trace/DefaultObserverFactory.groovy | 1 + .../nextflow/trace/GraphObserver.groovy | 50 +++++------- .../nextflow/dag/DotRendererTest.groovy | 2 +- .../nextflow/dag/GexfRendererTest.groovy | 2 +- .../nextflow/dag/MermaidRendererTest.groovy | 2 +- .../nextflow/processor/TaskHandlerTest.groovy | 26 +++++++ .../main/nextflow/extension/FilesEx.groovy | 8 ++ 22 files changed, 239 insertions(+), 52 deletions(-) create mode 100644 modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy diff --git a/docs/config.rst b/docs/config.rst index 3be4e6b518..744b814da4 100644 --- a/docs/config.rst +++ b/docs/config.rst @@ -254,6 +254,7 @@ Name Description enabled When ``true`` turns on the generation of the DAG file (default: ``false``). file Graph file name (default: ``dag-.dot``). overwrite When ``true`` overwrites any existing DAG file with the same name. +type Can be ``abstract`` to render the abstract DAG or ``concrete`` to render the concrete (task) DAG (default: ``abstract``). ================== ================ The above options can be used by prefixing them with the ``dag`` scope or surrounding them by curly diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 9fc4248502..55ef2cf6cc 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -40,6 +40,7 @@ import nextflow.conda.CondaConfig import nextflow.config.Manifest import nextflow.container.ContainerConfig import nextflow.dag.DAG +import nextflow.dag.ConcreteDAG import nextflow.exception.AbortOperationException import nextflow.exception.AbortSignalException import nextflow.exception.IllegalConfigException @@ -193,6 +194,8 @@ class Session implements ISession { private DAG dag + private ConcreteDAG concreteDag + private CacheDB cache private Barrier processesBarrier = new Barrier() @@ -345,6 +348,7 @@ class Session implements ISession { // -- DAG object this.dag = new DAG() + this.concreteDag = new ConcreteDAG() // -- init work dir this.workDir = ((config.workDir ?: 'work') as Path).complete() @@ -800,6 +804,8 @@ class Session implements ISession { DAG getDag() { this.dag } + ConcreteDAG getConcreteDAG() { this.concreteDag } + ExecutorService getExecService() { execService } /** @@ -1008,6 +1014,9 @@ class Session implements ISession { final trace = handler.safeTraceRecord() cache.putTaskAsync(handler, trace) + // save the task meta file to the task directory + handler.writeMetaFile() + // notify the event to the observers for( int i=0; i + */ +@Slf4j +class ConcreteDAG { + + Map nodes = new HashMap<>(100) + + /** + * Create a new node for a task + * + * @param task + * @param hash + */ + synchronized void addTaskNode( TaskRun task, String hash ) { + final label = "[${hash.substring(0,2)}/${hash.substring(2,8)}] ${task.name}" + final preds = task.getInputFilesMap().values() + .collect { p -> getPredecessorHash(p) } + .findAll { h -> h != null } + + nodes[hash] = new Node( + index: nodes.size(), + label: label, + predecessors: preds + ) + } + + static public String getPredecessorHash(Path path) { + final pattern = Pattern.compile('.*/([a-z0-9]{2}/[a-z0-9]{30})') + final matcher = pattern.matcher(path.toString()) + + matcher.find() ? matcher.group(1).replace('/', '') : null + } + + @MapConstructor + @ToString(includeNames = true, includes = 'label', includePackage=false) + protected class Node { + + int index + + String label + + List predecessors + + String getSlug() { "t${index}" } + + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy index 1d0bbe117f..7cd207807b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy @@ -29,7 +29,7 @@ import java.nio.file.Path class CytoscapeHtmlRenderer implements DagRenderer { @Override - void renderDocument(DAG dag, Path file) { + void renderAbstractGraph(DAG dag, Path file) { String tmplPage = readTemplate() String network = CytoscapeJsRenderer.renderNetwork(dag) file.text = tmplPage.replaceAll(~/\/\* REPLACE_WITH_NETWORK_DATA \*\//, network) diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy index d6eff29552..1486f4648b 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy @@ -29,7 +29,7 @@ import java.nio.file.Path class CytoscapeJsRenderer implements DagRenderer { @Override - void renderDocument(DAG dag, Path file) { + void renderAbstractGraph(DAG dag, Path file) { file.text = renderNetwork(dag) } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy index 808bc86660..34e703c0e3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy @@ -41,7 +41,7 @@ import nextflow.script.params.OutputsList import nextflow.script.params.TupleInParam import nextflow.script.params.TupleOutParam /** - * Model a direct acyclic graph of the pipeline execution. + * Model the abstract graph of a pipeline execution. * * @author Paolo Di Tommaso */ diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy index 4b7e487937..f526d5f765 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy @@ -24,10 +24,19 @@ import java.nio.file.Path * @author Paolo Di Tommaso * @author Mike Smoot */ -interface DagRenderer { +trait DagRenderer { /** - * Render the dag to the specified file. + * Render an abstract (process/operator) DAG. */ - void renderDocument(DAG dag, Path file); + void renderAbstractGraph(DAG dag, Path file) { + throw new UnsupportedOperationException("Abstract graph rendering is not supported for this file format") + } + + /** + * Render a concrete (task) DAG. + */ + void renderConcreteGraph(ConcreteDAG dag, Path file) { + throw new UnsupportedOperationException("Concrete graph rendering is not supported for this file format") + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy index 65184f62c2..817d47a3d2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy @@ -46,7 +46,7 @@ class DotRenderer implements DagRenderer { static String normalise(String str) { str.replaceAll(/[^0-9_A-Za-z]/,'') } @Override - void renderDocument(DAG dag, Path file) { + void renderAbstractGraph(DAG dag, Path file) { file.text = renderNetwork(dag) } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy index c2bac51110..43aaba6e9c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy @@ -41,7 +41,7 @@ class GexfRenderer implements DagRenderer { } @Override - void renderDocument(DAG dag, Path file) { + void renderAbstractGraph(DAG dag, Path file) { final Charset charset = Charset.defaultCharset() Writer bw = Files.newBufferedWriter(file, charset) final XMLOutputFactory xof = XMLOutputFactory.newFactory() diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy index 5d119adbdc..7ce79b839c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy @@ -42,7 +42,7 @@ class GraphvizRenderer implements DagRenderer { * See http://www.graphviz.org for more info. */ @Override - void renderDocument(DAG dag, Path target) { + void renderAbstractGraph(DAG dag, Path target) { def result = Files.createTempFile('nxf-',".$format") def temp = Files.createTempFile('nxf-','.dot') // save the DAG as `dot` to a temp file diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy index 4b3ec56c2a..6e55674054 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy @@ -27,11 +27,7 @@ import java.nio.file.Path class MermaidRenderer implements DagRenderer { @Override - void renderDocument(DAG dag, Path file) { - file.text = renderNetwork(dag) - } - - String renderNetwork(DAG dag) { + void renderAbstractGraph(DAG dag, Path file) { def lines = [] lines << "flowchart TD" @@ -45,7 +41,7 @@ class MermaidRenderer implements DagRenderer { lines << "" - return lines.join('\n') + file.text = lines.join('\n') } private String renderVertex(DAG.Vertex vertex) { @@ -76,4 +72,25 @@ class MermaidRenderer implements DagRenderer { return "${edge.from.name} -->${label} ${edge.to.name}" } + + @Override + void renderConcreteGraph(ConcreteDAG graph, Path file) { + def lines = [] + lines << "flowchart TD" + + graph.nodes.values().each { node -> + lines << " ${node.getSlug()}[\"${node.label}\"]" + + node.predecessors.each { key -> + final pred = graph.nodes[key] + + if( pred ) + lines << " ${pred.getSlug()} --> ${node.getSlug()}" + } + } + + lines << "" + + file.text = lines.join('\n') + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy index 4e5dc60c1c..d9f8f14990 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy @@ -22,6 +22,7 @@ import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Global import nextflow.Session import nextflow.processor.TaskProcessor +import nextflow.processor.TaskRun import nextflow.script.params.InputsList import nextflow.script.params.OutputsList /** @@ -41,7 +42,7 @@ class NodeMarker { static private Session getSession() { Global.session as Session } /** - * Creates a new vertex in the DAG representing a computing `process` + * Creates a vertex in the abstract DAG representing a computing `process` * * @param label The label associated to the process * @param inputs The list of inputs entering in the process @@ -53,7 +54,7 @@ class NodeMarker { } /** - * Creates a new DAG vertex representing a dataflow operator + * Creates a vertex in the abstract DAG representing a dataflow operator * * @param label The operator label * @param inputs The operator input(s). It can be either a single channel or a list of channels. @@ -67,7 +68,7 @@ class NodeMarker { } /** - * Creates a vertex in the DAG representing a dataflow channel source. + * Creates a vertex in the abstract DAG representing a dataflow channel source. * * @param label The node description * @param source Either a dataflow channel or a list of channel. @@ -88,4 +89,15 @@ class NodeMarker { session.dag.addDataflowBroadcastPair(readChannel, broadcastChannel) } + /** + * Creates a vertex in the concrete DAG representing a task + * + * @param task + * @param hash + */ + static void addTaskNode( TaskRun task, String hash ) { + if( session?.concreteDAG && !session.aborted ) + session.concreteDAG.addTaskNode( task, hash ) + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index e14b612566..61efe65c18 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -21,7 +21,11 @@ import static nextflow.processor.TaskStatus.* import java.nio.file.NoSuchFileException +import groovy.json.JsonBuilder import groovy.util.logging.Slf4j +import nextflow.dag.ConcreteDAG +import nextflow.extension.FilesEx +import nextflow.script.params.FileOutParam import nextflow.trace.TraceRecord /** * Actions to handle the underlying job running the user task. @@ -214,6 +218,31 @@ abstract class TaskHandler { return record } + void writeMetaFile() { + final record = [ + hash: task.hash.toString(), + inputs: task.getInputFilesMap().collect { name, path -> + [ + name: name, + path: path.toString(), + predecessor: ConcreteDAG.getPredecessorHash(path) + ] + }, + outputs: task.getOutputsByType(FileOutParam).values().flatten().collect { path -> + final file = path.toFile() + + [ + name: path.name, + path: path.toString(), + size: file.size(), + checksum: file.isFile() ? FilesEx.getChecksum(path) : null + ] + } + ] + + task.workDir.resolve(TaskRun.CMD_META).text = new JsonBuilder(record).toString() + } + /** * Determine if a process can be forked i.e. can launch * a parallel task execution. This is only enforced when diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 2b547f27c0..05e4643928 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -524,7 +524,7 @@ class TaskProcessor { def invoke = new InvokeTaskAdapter(this, opInputs.size()) session.allOperators << (operator = new DataflowOperator(group, params, invoke)) - // notify the creation of a new vertex the execution DAG + // notify the creation of a new process in the abstract DAG NodeMarker.addProcessNode(this, config.getInputs(), config.getOutputs()) // fix issue #41 @@ -756,8 +756,11 @@ class TaskProcessor { log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" def cached = shouldTryCache && exists && checkCachedOutput(task.clone(), resumeDir, hash, entry) - if( cached ) + if( cached ) { + // add cached task to the task graph + NodeMarker.addTaskNode(task, hash.toString()) break + } } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) @@ -780,6 +783,9 @@ class TaskProcessor { lock.release() } + // add submitted task to the task graph + NodeMarker.addTaskNode(task, hash.toString()) + // submit task for execution submitTask( task, hash, workDir ) break diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index af98f7f372..dd0942fd0e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -543,6 +543,7 @@ class TaskRun implements Cloneable { static final public String CMD_RUN = '.command.run' static final public String CMD_TRACE = '.command.trace' static final public String CMD_ENV = '.command.env' + static final public String CMD_META = '.command.meta.json' String toString( ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index 42325c723a..832e73ede2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -93,6 +93,7 @@ class DefaultObserverFactory implements TraceObserverFactory { if( !fileName ) fileName = GraphObserver.DEF_FILE_NAME def traceFile = (fileName as Path).complete() def observer = new GraphObserver(traceFile) + config.navigate('dag.type') { observer.dagType = it ?: 'abstract' } config.navigate('dag.overwrite') { observer.overwrite = it } result << observer } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy index 2b10d3d244..07e2c01b12 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy @@ -24,6 +24,7 @@ import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.CytoscapeHtmlRenderer import nextflow.dag.DAG +import nextflow.dag.ConcreteDAG import nextflow.dag.DagRenderer import nextflow.dag.DotRenderer import nextflow.dag.GexfRenderer @@ -46,7 +47,11 @@ class GraphObserver implements TraceObserver { private Path file - private DAG dag + private String type + + private DAG abstractDag + + private ConcreteDAG concreteDag private String name @@ -67,7 +72,8 @@ class GraphObserver implements TraceObserver { @Override void onFlowCreate(Session session) { - this.dag = session.dag + this.abstractDag = session.dag + this.concreteDag = session.concreteDag // check file existance final attrs = FileHelper.readAttributes(file) if( attrs ) { @@ -80,14 +86,22 @@ class GraphObserver implements TraceObserver { @Override void onFlowComplete() { - // -- normalise the DAG - dag.normalize() - // -- render it to a file - createRender().renderDocument(dag,file) + if( type == 'abstract' ) { + // -- normalise the DAG + abstractDag.normalize() + // -- render it to a file + createRenderer().renderAbstractGraph(abstractDag,file) + } + else if( type == 'concrete' ) { + createRenderer().renderConcreteGraph(concreteDag,file) + } + else { + log.warn("Invalid DAG type '${type}', should be 'abstract' or 'concrete'") + } } @PackageScope - DagRenderer createRender() { + DagRenderer createRenderer() { if( format == 'dot' ) new DotRenderer(name) @@ -104,28 +118,6 @@ class GraphObserver implements TraceObserver { new GraphvizRenderer(name, format) } - - @Override - void onProcessCreate(TaskProcessor process) { - - } - - - @Override - void onProcessSubmit(TaskHandler handler, TraceRecord trace) { - - } - - @Override - void onProcessStart(TaskHandler handler, TraceRecord trace) { - - } - - @Override - void onProcessComplete(TaskHandler handler, TraceRecord trace) { - - } - @Override boolean enableMetrics() { return false diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy index 499a68105a..5fbab45413 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy @@ -55,7 +55,7 @@ class DotRendererTest extends Specification { dag.normalize() when: - new DotRenderer('TheGraph').renderDocument(dag, file) + new DotRenderer('TheGraph').renderAbstractGraph(dag, file) then: file.text == ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy index f112732b2e..6d4a9e3539 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy @@ -47,7 +47,7 @@ class GexfRendererTest extends Specification { dag.normalize() when: - new GexfRenderer('TheGraph').renderDocument(dag, file.toPath()) + new GexfRenderer('TheGraph').renderAbstractGraph(dag, file.toPath()) then: def graph = new XmlSlurper().parse(file); assert graph.name() == 'gexf' diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy index 43255e4c88..0b46bd3f84 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy @@ -47,7 +47,7 @@ class MermaidRendererTest extends Specification { dag.normalize() when: - new MermaidRenderer().renderDocument(dag, file) + new MermaidRenderer().renderAbstractGraph(dag, file) then: file.text == ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy index de052de06a..559be4ef2c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy @@ -17,10 +17,13 @@ package nextflow.processor +import java.nio.file.Paths import java.util.concurrent.atomic.LongAdder +import com.google.common.hash.HashCode import nextflow.Session import nextflow.executor.Executor +import nextflow.script.params.FileOutParam import nextflow.util.Duration import nextflow.util.MemoryUnit import spock.lang.Specification @@ -136,6 +139,29 @@ class TaskHandlerTest extends Specification { } + def 'should write meta file' () { + + given: + def folder = File.createTempDir() + def outputFile = new File(folder, 'bar.txt') ; outputFile.text = 'bar' + def task = Mock(TaskRun) { + hash >> HashCode.fromString("aabbccddeeff00112233445566778899") + workDir >> folder.toPath() + getInputFilesMap() >> [ 'foo.txt': Paths.get('/tmp/00/112233445566778899aabbccddeeff/foo.txt') ] + getOutputsByType(FileOutParam) >> [ 'bar.txt': outputFile.toPath() ] + } + def handler = [:] as TaskHandler + handler.task = task + + when: + handler.writeMetaFile() + then: + task.workDir.resolve(TaskRun.CMD_META).text == """{"hash":"aabbccddeeff00112233445566778899","inputs":[{"name":"foo.txt","path":"/tmp/00/112233445566778899aabbccddeeff/foo.txt","predecessor":"00112233445566778899aabbccddeeff"}],"outputs":[{"name":"bar.txt","path":"${folder}/bar.txt","size":3,"checksum":"37b51d194a7513e45b56f6524f2d51f2"}]}""" + + cleanup: + folder.delete() + } + LongAdder _adder(Integer x) { if( x != null ) { def adder = new LongAdder() diff --git a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy index 94447e677f..9d79cd0a1b 100644 --- a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy +++ b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy @@ -35,6 +35,7 @@ import java.nio.file.attribute.FileAttribute import java.nio.file.attribute.FileTime import java.nio.file.attribute.PosixFilePermission import java.nio.file.attribute.PosixFilePermissions +import java.security.MessageDigest import groovy.transform.CompileStatic import groovy.transform.PackageScope @@ -1600,4 +1601,11 @@ class FilesEx { static String getScheme(Path path) { path.getFileSystem().provider().getScheme() } + + static String getChecksum(Path path) { + final data = Files.readAllBytes(path) + final hash = MessageDigest.getInstance("MD5").digest(data) + + new BigInteger(1, hash).toString(16) + } } From ae6702734b61615922f3956124a9f091ec7c20f5 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 28 Mar 2023 14:09:43 -0500 Subject: [PATCH 02/34] Add task inputs and outputs to conrete DAG Signed-off-by: Ben Sherman --- .../groovy/nextflow/dag/ConcreteDAG.groovy | 59 +++++++++---- .../nextflow/dag/MermaidRenderer.groovy | 38 ++++++-- .../groovy/nextflow/dag/NodeMarker.groovy | 12 --- .../nextflow/processor/TaskProcessor.groovy | 8 +- .../trace/DefaultObserverFactory.groovy | 2 +- .../nextflow/trace/GraphObserver.groovy | 16 ++++ .../nextflow/dag/ConcreteDAGTest.groovy | 86 +++++++++++++++++++ .../nextflow/dag/MermaidRendererTest.groovy | 68 ++++++++++++++- .../nextflow/processor/TaskHandlerTest.groovy | 5 +- 9 files changed, 246 insertions(+), 48 deletions(-) create mode 100644 modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy index 735a7cf76b..5768916d16 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy @@ -23,6 +23,7 @@ import groovy.transform.MapConstructor import groovy.transform.ToString import groovy.util.logging.Slf4j import nextflow.processor.TaskRun +import nextflow.script.params.FileOutParam /** * Model the conrete (task) graph of a pipeline execution. * @@ -31,46 +32,74 @@ import nextflow.processor.TaskRun @Slf4j class ConcreteDAG { - Map nodes = new HashMap<>(100) + Map nodes = new HashMap<>(100) /** - * Create a new node for a task + * Add a task to the graph * * @param task - * @param hash */ - synchronized void addTaskNode( TaskRun task, String hash ) { + synchronized void addTask( TaskRun task ) { + final hash = task.hash.toString() final label = "[${hash.substring(0,2)}/${hash.substring(2,8)}] ${task.name}" - final preds = task.getInputFilesMap().values() - .collect { p -> getPredecessorHash(p) } - .findAll { h -> h != null } + final inputs = task.getInputFilesMap() + .collect { name, path -> + new Input(name: name, path: path, predecessor: getPredecessorHash(path)) + } - nodes[hash] = new Node( + nodes[hash] = new Task( index: nodes.size(), label: label, - predecessors: preds + inputs: inputs ) } static public String getPredecessorHash(Path path) { - final pattern = Pattern.compile('.*/([a-z0-9]{2}/[a-z0-9]{30})') + final pattern = Pattern.compile('.*/([0-9a-f]{2}/[0-9a-f]{30})') final matcher = pattern.matcher(path.toString()) matcher.find() ? matcher.group(1).replace('/', '') : null } + /** + * Add a task's outputs to the graph + * + * @param task + */ + synchronized void addTaskOutputs( TaskRun task ) { + final hash = task.hash.toString() + final outputs = task.getOutputsByType(FileOutParam) + .values() + .flatten() + .collect { path -> + new Output(name: path.name, path: path) + } + + nodes[hash].outputs = outputs + } + @MapConstructor @ToString(includeNames = true, includes = 'label', includePackage=false) - protected class Node { - + static protected class Task { int index - String label - - List predecessors + List inputs + List outputs String getSlug() { "t${index}" } + } + @MapConstructor + static protected class Input { + String name + Path path + String predecessor + } + + @MapConstructor + static protected class Output { + String name + Path path } } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy index 6e55674054..706a93f5d9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy @@ -75,17 +75,41 @@ class MermaidRenderer implements DagRenderer { @Override void renderConcreteGraph(ConcreteDAG graph, Path file) { + def renderedOutputs = [] as Set + def numInputs = 0 + def numOutputs = 0 + def lines = [] lines << "flowchart TD" - graph.nodes.values().each { node -> - lines << " ${node.getSlug()}[\"${node.label}\"]" - - node.predecessors.each { key -> - final pred = graph.nodes[key] + // render tasks and task inputs + graph.nodes.values().each { task -> + // render task node + lines << " ${task.getSlug()}[\"${task.label}\"]" + + task.inputs.each { input -> + // render task input from predecessor + if( input.predecessor != null ) { + final pred = graph.nodes[input.predecessor] + lines << " ${pred.getSlug()} -->|${input.name}| ${task.getSlug()}" + renderedOutputs << input.path + } + + // render task input from source node + else { + numInputs += 1 + lines << " i${numInputs}(( )) -->|${input.name}| ${task.getSlug()}" + } + } + } - if( pred ) - lines << " ${pred.getSlug()} --> ${node.getSlug()}" + // render task outputs with sink nodes + graph.nodes.values().each { task -> + task.outputs.each { output -> + if( output.path !in renderedOutputs ) { + numOutputs += 1 + lines << " ${task.getSlug()} -->|${output.name}| o${numOutputs}(( ))" + } } } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy index d9f8f14990..0d543130e9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy @@ -22,7 +22,6 @@ import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Global import nextflow.Session import nextflow.processor.TaskProcessor -import nextflow.processor.TaskRun import nextflow.script.params.InputsList import nextflow.script.params.OutputsList /** @@ -89,15 +88,4 @@ class NodeMarker { session.dag.addDataflowBroadcastPair(readChannel, broadcastChannel) } - /** - * Creates a vertex in the concrete DAG representing a task - * - * @param task - * @param hash - */ - static void addTaskNode( TaskRun task, String hash ) { - if( session?.concreteDAG && !session.aborted ) - session.concreteDAG.addTaskNode( task, hash ) - } - } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 05e4643928..0032b8e9ae 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -756,11 +756,8 @@ class TaskProcessor { log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" def cached = shouldTryCache && exists && checkCachedOutput(task.clone(), resumeDir, hash, entry) - if( cached ) { - // add cached task to the task graph - NodeMarker.addTaskNode(task, hash.toString()) + if( cached ) break - } } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) @@ -783,9 +780,6 @@ class TaskProcessor { lock.release() } - // add submitted task to the task graph - NodeMarker.addTaskNode(task, hash.toString()) - // submit task for execution submitTask( task, hash, workDir ) break diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index 832e73ede2..72a8b31d5f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -93,7 +93,7 @@ class DefaultObserverFactory implements TraceObserverFactory { if( !fileName ) fileName = GraphObserver.DEF_FILE_NAME def traceFile = (fileName as Path).complete() def observer = new GraphObserver(traceFile) - config.navigate('dag.type') { observer.dagType = it ?: 'abstract' } + config.navigate('dag.type') { observer.type = it ?: 'abstract' } config.navigate('dag.overwrite') { observer.overwrite = it } result << observer } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy index 07e2c01b12..2be9444056 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy @@ -84,6 +84,22 @@ class GraphObserver implements TraceObserver { } } + @Override + void onProcessSubmit(TaskHandler handler, TraceRecord trace) { + concreteDag.addTask( handler.task ) + } + + @Override + void onProcessComplete(TaskHandler handler, TraceRecord trace) { + concreteDag.addTaskOutputs( handler.task ) + } + + @Override + void onProcessCached(TaskHandler handler, TraceRecord trace) { + concreteDag.addTask( handler.task ) + concreteDag.addTaskOutputs( handler.task ) + } + @Override void onFlowComplete() { if( type == 'abstract' ) { diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy new file mode 100644 index 0000000000..a666372169 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy @@ -0,0 +1,86 @@ +/* + * 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.dag + +import java.nio.file.Paths + +import com.google.common.hash.HashCode +import nextflow.processor.TaskRun +import spock.lang.Specification +/** + * + * @author Ben Sherman + */ +class ConcreteDAGTest extends Specification { + + def 'should add task nodes and outputs' () { + + given: + def task1 = Mock(TaskRun) { + getHash() >> HashCode.fromString('00112233445566778899aabbccddeeff') + getName() >> 'foo' + getInputFilesMap() >> [ + 'data.txt': Paths.get('/inputs/data.txt') + ] + getOutputsByType(_) >> [ + 'data.foo': Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') + ] + } + def task2 = Mock(TaskRun) { + getHash() >> HashCode.fromString('aabbccddeeff00112233445566778899') + getName() >> 'bar' + getInputFilesMap() >> [ + 'data.foo': Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') + ] + getOutputsByType(_) >> [ + 'data.bar': Paths.get('/work/aa/bbccddeeff00112233445566778899/data.bar') + ] + } + def dag = new ConcreteDAG() + + when: + dag.addTask( task1 ) + dag.addTask( task2 ) + def node1 = dag.nodes['00112233445566778899aabbccddeeff'] + def node2 = dag.nodes['aabbccddeeff00112233445566778899'] + then: + node1.index == 0 + node1.label == '[00/112233] foo' + node1.inputs.size() == 1 + node1.inputs[0].name == 'data.txt' + node1.inputs[0].path == Paths.get('/inputs/data.txt') + node1.inputs[0].predecessor == null + node2.index == 1 + node2.label == '[aa/bbccdd] bar' + node2.inputs.size() == 1 + node2.inputs[0].name == 'data.foo' + node2.inputs[0].path == Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') + node2.inputs[0].predecessor == '00112233445566778899aabbccddeeff' + + when: + dag.addTaskOutputs( task1 ) + dag.addTaskOutputs( task2 ) + then: + node1.outputs.size() == 1 + node1.outputs[0].name == 'data.foo' + node1.outputs[0].path == Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') + node2.outputs.size() == 1 + node2.outputs[0].name == 'data.bar' + node2.outputs[0].path == Paths.get('/work/aa/bbccddeeff00112233445566778899/data.bar') + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy index 0b46bd3f84..95d408fdc3 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy @@ -16,12 +16,13 @@ */ package nextflow.dag + import java.nio.file.Files +import java.nio.file.Paths import groovyx.gpars.dataflow.DataflowQueue -import spock.lang.Specification - import nextflow.Session +import spock.lang.Specification /** * @@ -33,7 +34,7 @@ class MermaidRendererTest extends Specification { new Session() } - def 'should render a graph using the `mmd` format' () { + def 'should render an abstract graph using the `mmd` format' () { given: def file = Files.createTempFile('test', null) def ch1 = new DataflowQueue() @@ -65,4 +66,65 @@ class MermaidRendererTest extends Specification { cleanup: file.delete() } + + def 'should render a concrete graph using the `mmd` format' () { + given: + def file = Files.createTempFile('test', null) + + def dag = Mock(ConcreteDAG) { + nodes >> [ + '012345': new ConcreteDAG.Task( + index: 1, + label: 'foo', + inputs: [ + new ConcreteDAG.Input( + name: 'data.txt', + path: Paths.get('/inputs/data.txt'), + predecessor: null + ) + ], + outputs: [ + new ConcreteDAG.Output( + name: 'data.foo', + path: Paths.get('/work/012345/data.foo'), + ) + ] + ), + 'abcdef': new ConcreteDAG.Task( + index: 2, + label: 'bar', + inputs: [ + new ConcreteDAG.Input( + name: 'data.foo', + path: Paths.get('/work/012345/data.foo'), + predecessor: '012345' + ) + ], + outputs: [ + new ConcreteDAG.Output( + name: 'data.bar', + path: Paths.get('/work/abcdef/data.bar'), + ) + ] + ) + ] + } + + when: + new MermaidRenderer().renderConcreteGraph(dag, file) + then: + file.text == + ''' + flowchart TD + t1["foo"] + i1(( )) -->|data.txt| t1 + t2["bar"] + t1 -->|data.foo| t2 + t2 -->|data.bar| o1(( )) + ''' + .stripIndent().leftTrim() + + cleanup: + file.delete() + } } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy index 559be4ef2c..cc73df6c00 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.LongAdder import com.google.common.hash.HashCode import nextflow.Session import nextflow.executor.Executor -import nextflow.script.params.FileOutParam import nextflow.util.Duration import nextflow.util.MemoryUnit import spock.lang.Specification @@ -145,10 +144,10 @@ class TaskHandlerTest extends Specification { def folder = File.createTempDir() def outputFile = new File(folder, 'bar.txt') ; outputFile.text = 'bar' def task = Mock(TaskRun) { - hash >> HashCode.fromString("aabbccddeeff00112233445566778899") + hash >> HashCode.fromString('aabbccddeeff00112233445566778899') workDir >> folder.toPath() getInputFilesMap() >> [ 'foo.txt': Paths.get('/tmp/00/112233445566778899aabbccddeeff/foo.txt') ] - getOutputsByType(FileOutParam) >> [ 'bar.txt': outputFile.toPath() ] + getOutputsByType(_) >> [ 'bar.txt': outputFile.toPath() ] } def handler = [:] as TaskHandler handler.task = task From 8f95cd6466e5d79aadc68e7ff18f17aaa530978b Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 28 Mar 2023 14:33:44 -0500 Subject: [PATCH 03/34] Fix failing tests Signed-off-by: Ben Sherman --- .../nextflow/trace/GraphObserver.groovy | 2 +- .../nextflow/trace/GraphObserverTest.groovy | 34 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy index 2be9444056..3826e0b6b5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy @@ -47,7 +47,7 @@ class GraphObserver implements TraceObserver { private Path file - private String type + private String type = 'abstract' private DAG abstractDag diff --git a/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy b/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy index bc1efe8214..baca8d5270 100644 --- a/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy @@ -36,7 +36,7 @@ import test.TestHelper */ class GraphObserverTest extends Specification { - DAG test_dag + DAG dag def setup() { new Session() @@ -46,28 +46,28 @@ class GraphObserverTest extends Specification { def ch2 = new DataflowQueue() def ch3 = new DataflowQueue() - test_dag = new DAG() + dag = new DAG() - test_dag.addVertex( + dag.addVertex( DAG.Type.ORIGIN, 'Source', null, [ new DAG.ChannelHandler(channel: src, label: 'src') ] ) - test_dag.addVertex( + dag.addVertex( DAG.Type.PROCESS, 'Process 1', [ new DAG.ChannelHandler(channel: src, label: 'Source') ], [ new DAG.ChannelHandler(channel: ch1, label: 'Channel 1') ] ) - test_dag.addVertex( + dag.addVertex( DAG.Type.OPERATOR, 'Filter', [ new DAG.ChannelHandler(channel: ch1, label: 'Channel 1') ], [ new DAG.ChannelHandler(channel: ch2, label: 'Channel 2') ] ) - test_dag.addVertex( + dag.addVertex( DAG.Type.PROCESS, 'Process 2', [ new DAG.ChannelHandler(channel: ch2, label: 'Channel 2') ], @@ -79,7 +79,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf_','.dot') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -106,7 +106,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.html') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -134,7 +134,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.svg') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -152,7 +152,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.png') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -169,7 +169,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.pdf') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -186,7 +186,7 @@ class GraphObserverTest extends Specification { def folder = Files.createTempDirectory('test') def file = folder.resolve('nope') def gr = new GraphObserver(file) - gr.dag = test_dag + gr.abstractDag = dag when: gr.onFlowComplete() @@ -217,34 +217,34 @@ class GraphObserverTest extends Specification { then: observer.name == 'hello-world' observer.format == 'dot' - observer.createRender() instanceof DotRenderer + observer.createRenderer() instanceof DotRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.html')) then: observer.name == 'TheGraph' observer.format == 'html' - observer.createRender() instanceof CytoscapeHtmlRenderer + observer.createRenderer() instanceof CytoscapeHtmlRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.mmd')) then: observer.name == 'TheGraph' observer.format == 'mmd' - observer.createRender() instanceof MermaidRenderer + observer.createRenderer() instanceof MermaidRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.SVG')) then: observer.name == 'TheGraph' observer.format == 'svg' - observer.createRender() instanceof GraphvizRenderer + observer.createRenderer() instanceof GraphvizRenderer when: observer = new GraphObserver(Paths.get('/path/to/anonymous')) then: observer.name == 'anonymous' observer.format == 'dot' - observer.createRender() instanceof DotRenderer + observer.createRenderer() instanceof DotRenderer } } From 9f11e4b193b9187a928992416115df1bbd6b4a61 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 29 Mar 2023 11:18:40 -0500 Subject: [PATCH 04/34] Use path-based APIs to get file metadata Signed-off-by: Ben Sherman --- .../nextflow/processor/TaskHandler.groovy | 7 ++--- .../main/nextflow/extension/FilesEx.groovy | 17 +++++++++-- .../main/nextflow/file/ETagAwareFile.groovy | 29 +++++++++++++++++++ .../src/main/com/upplication/s3fs/S3Path.java | 11 ++++++- .../cloud/azure/nio/AzFileAttributes.groovy | 8 +++++ .../nextflow/cloud/azure/nio/AzPath.groovy | 8 ++++- 6 files changed, 71 insertions(+), 9 deletions(-) create mode 100644 modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 61efe65c18..44212e8cd8 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.Files import java.nio.file.NoSuchFileException import groovy.json.JsonBuilder @@ -229,13 +230,11 @@ abstract class TaskHandler { ] }, outputs: task.getOutputsByType(FileOutParam).values().flatten().collect { path -> - final file = path.toFile() - [ name: path.name, path: path.toString(), - size: file.size(), - checksum: file.isFile() ? FilesEx.getChecksum(path) : null + size: Files.size(path), + checksum: FilesEx.getChecksum(path) ] } ] diff --git a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy index 9d79cd0a1b..1bac3c19db 100644 --- a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy +++ b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy @@ -35,6 +35,7 @@ import java.nio.file.attribute.FileAttribute import java.nio.file.attribute.FileTime import java.nio.file.attribute.PosixFilePermission import java.nio.file.attribute.PosixFilePermissions +import java.security.DigestInputStream import java.security.MessageDigest import groovy.transform.CompileStatic @@ -42,6 +43,7 @@ import groovy.transform.PackageScope import groovy.transform.stc.ClosureParams import groovy.transform.stc.FromString import groovy.util.logging.Slf4j +import nextflow.file.ETagAwareFile import nextflow.file.FileHelper import nextflow.file.FileSystemPathFactory import nextflow.io.ByteBufferBackedInputStream @@ -1603,9 +1605,18 @@ class FilesEx { } static String getChecksum(Path path) { - final data = Files.readAllBytes(path) - final hash = MessageDigest.getInstance("MD5").digest(data) + if( Files.isDirectory(path) ) + return null + + if( path instanceof ETagAwareFile ) + return ((ETagAwareFile)path).getETag() + + final md = MessageDigest.getInstance('MD5') + final is = Files.newInputStream(path) + final dis = new DigestInputStream(is, md) + + while( dis.read() != -1 ) {} - new BigInteger(1, hash).toString(16) + new BigInteger(1, md.digest()).toString(16) } } diff --git a/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy b/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy new file mode 100644 index 0000000000..f1c40073b3 --- /dev/null +++ b/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy @@ -0,0 +1,29 @@ +/* + * 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.file + +/** + * Defines the interface for files that have an ETag + * + * @author Ben Sherman + */ +interface ETagAwareFile { + + String getETag() + +} diff --git a/plugins/nf-amazon/src/main/com/upplication/s3fs/S3Path.java b/plugins/nf-amazon/src/main/com/upplication/s3fs/S3Path.java index dfd12cea08..9e9688945e 100644 --- a/plugins/nf-amazon/src/main/com/upplication/s3fs/S3Path.java +++ b/plugins/nf-amazon/src/main/com/upplication/s3fs/S3Path.java @@ -66,13 +66,14 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import nextflow.file.ETagAwareFile; import nextflow.file.TagAwareFile; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Iterables.transform; import static java.lang.String.format; -public class S3Path implements Path, TagAwareFile { +public class S3Path implements Path, ETagAwareFile, TagAwareFile { public static final String PATH_SEPARATOR = "/"; /** @@ -571,6 +572,14 @@ public String getContentType() { return contentType; } + @Override + public String getETag() { + return fileSystem + .getClient() + .getObjectMetadata(getBucket(), getKey()) + .getETag(); + } + // ~ helpers methods private static Function strip(final String ... strs) { diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy index a955888d70..ba07cda22f 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy @@ -46,6 +46,8 @@ class AzFileAttributes implements BasicFileAttributes { private String objectId + private String etag + static AzFileAttributes root() { new AzFileAttributes(size: 0, objectId: '/', directory: true) } @@ -60,6 +62,7 @@ class AzFileAttributes implements BasicFileAttributes { updateTime = time(props.getLastModified()) directory = client.blobName.endsWith('/') size = props.getBlobSize() + etag = props.getETag() } AzFileAttributes(String containerName, BlobItem item) { @@ -69,6 +72,7 @@ class AzFileAttributes implements BasicFileAttributes { creationTime = time(item.properties.getCreationTime()) updateTime = time(item.properties.getLastModified()) size = item.properties.getContentLength() + etag = item.properties.getETag() } } @@ -144,6 +148,10 @@ class AzFileAttributes implements BasicFileAttributes { return objectId } + String getETag() { + return etag + } + @Override boolean equals( Object obj ) { if( this.class != obj?.class ) return false diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy index 2f654b4ad8..3bdd222f6b 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy @@ -29,6 +29,7 @@ import com.azure.storage.blob.models.BlobItem import groovy.transform.CompileStatic import groovy.transform.EqualsAndHashCode import groovy.transform.PackageScope +import nextflow.file.ETagAwareFile /** * Implements Azure path object @@ -37,7 +38,7 @@ import groovy.transform.PackageScope */ @CompileStatic @EqualsAndHashCode(includes = 'fs,path,directory', includeFields = true) -class AzPath implements Path { +class AzPath implements Path, ETagAwareFile { private AzFileSystem fs @@ -305,6 +306,11 @@ class AzPath implements Path { return this.toString() <=> other.toString() } + @Override + String getETag() { + return attributes.getETag() + } + String getContainerName() { if( path.isAbsolute() ) { path.nameCount==0 ? '/' : path.getName(0) From 84568928fc483ec4c1c55ae415b20f006a52f4d0 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 30 Mar 2023 09:57:20 -0500 Subject: [PATCH 05/34] Use buffer to compute checksum Signed-off-by: Ben Sherman --- .../src/main/nextflow/extension/FilesEx.groovy | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy index df77ee2023..026d0dbc92 100644 --- a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy +++ b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy @@ -34,7 +34,6 @@ import java.nio.file.attribute.FileAttribute import java.nio.file.attribute.FileTime import java.nio.file.attribute.PosixFilePermission import java.nio.file.attribute.PosixFilePermissions -import java.security.DigestInputStream import java.security.MessageDigest import groovy.transform.CompileStatic @@ -1610,11 +1609,13 @@ class FilesEx { if( path instanceof ETagAwareFile ) return ((ETagAwareFile)path).getETag() - final md = MessageDigest.getInstance('MD5') final is = Files.newInputStream(path) - final dis = new DigestInputStream(is, md) + final md = MessageDigest.getInstance('MD5') + final buf = new byte[16 << 10] - while( dis.read() != -1 ) {} + int len + while( (len=is.read(buf)) != -1 ) + md.update(buf, 0, len) new BigInteger(1, md.digest()).toString(16) } From 77f2cdc316b105dba4d9a42c9abec5a47397d575 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 31 Mar 2023 15:57:59 -0500 Subject: [PATCH 06/34] Add support for temporary output paths Signed-off-by: Ben Sherman --- docs/process.rst | 33 ++++ .../script/params/FileOutParam.groovy | 5 + .../trace/DefaultObserverFactory.groovy | 5 + .../trace/TemporaryFileObserver.groovy | 184 ++++++++++++++++++ tests/temporary-outputs.nf | 49 +++++ 5 files changed, 276 insertions(+) create mode 100644 modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy create mode 100644 tests/temporary-outputs.nf diff --git a/docs/process.rst b/docs/process.rst index e24de605c8..082660ee18 100644 --- a/docs/process.rst +++ b/docs/process.rst @@ -1026,6 +1026,7 @@ Name Description ``type`` Type of paths returned, either ``file``, ``dir`` or ``any`` (default: ``any``, or ``file`` if the specified file name pattern contains a double star (``**``)) ``maxDepth`` Maximum number of directory levels to visit (default: no limit) ``includeInputs`` When ``true`` any input files matching an output file glob pattern are included. +``temporary`` When ``true`` the file will be "emptied" once it is no longer needed by downstream tasks. ================== ===================== The parenthesis are optional for input and output qualifiers, but when you want to set an additional option and there @@ -1150,6 +1151,38 @@ on the actual value of the ``species`` input. because it will result in simpler and more portable code. +Temporary output files +---------------------- + +.. warning:: + This feature is experimental and may change in a future release. + +When ``path`` output is declared with ``temporary: true``, any file associated with this output will be automatically "emptied" +during pipeline execution, as soon as it is no longer needed by downstream tasks. This feature is useful for deleting large +intermediate files that can be discarded once a pipeline run is finished. + +The lifetime of a temporary file is determined by the processes that are downstream of the file's originating process, +either directly or indirectly through channel operators. When all of these processes finish (i.e. all of their tasks finish), +all temporary files produced by the original process can be deleted. + +"Emptying" a file means that the file contents are deleted, but the file metadata (size, last modified timestamp) is preserved, +such that it will be cached on a resumed run. + +The following caveats apply when using temporary outputs: + +- A pipeline run with temporary outputs should not be resumed if any regular downstream outputs have also been deleted. If a + temporary output is cached but a downstream output is not cached (e.g. because it was deleted or the process script was modified), + the downstream task will fail or produce incorrect output. + +- Temporary outputs will not be cached when the ``cache`` directive is set to ``'deep'``, because this mode includes the file + contents in the cache key. + +- A temporary output should not be forwarded by a downstream process using the ``includeInputs`` option. In this case, the temporary + output will be cleaned prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. + +- Directories and remote paths (e.g. S3) are not currently supported. + + .. _process-env: Output type ``env`` diff --git a/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy b/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy index e305c46c3e..58bc52a446 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy @@ -82,6 +82,11 @@ class FileOutParam extends BaseOutParam implements OutParam, OptionalParam, Path boolean glob = true + /** + * When true the file will be "cleaned" once it is no longer needed by downstream tasks. + */ + boolean temporary = false + private GString gstring private Closure dynamicObj private String filePattern diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index 72a8b31d5f..cd7bd72d7e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -26,6 +26,7 @@ class DefaultObserverFactory implements TraceObserverFactory { createDagObserver(result) createWebLogObserver(result) createAnsiLogObserver(result) + createTemporaryFileObserver(result) return result } @@ -117,4 +118,8 @@ class DefaultObserverFactory implements TraceObserverFactory { result << observer } + protected void createTemporaryFileObserver(Collection result) { + result << new TemporaryFileObserver() + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy new file mode 100644 index 0000000000..62a1ba2415 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -0,0 +1,184 @@ +/* + * 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.trace + +import java.nio.file.Files +import java.nio.file.Path +import java.nio.file.attribute.BasicFileAttributes +import java.util.concurrent.ConcurrentHashMap + +import groovy.transform.MapConstructor +import groovy.util.logging.Slf4j +import nextflow.Session +import nextflow.dag.DAG +import nextflow.dag.ConcreteDAG +import nextflow.processor.TaskHandler +import nextflow.processor.TaskProcessor +import nextflow.script.params.FileOutParam +/** + * Watch for temporary output files and "clean" them once they + * are no longer needed. + * + * @author Ben Sherman + */ +@Slf4j +class TemporaryFileObserver implements TraceObserver { + + private DAG dag + + private Map statusMap + + @Override + void onFlowCreate(Session session) { + this.dag = session.dag + this.statusMap = new ConcurrentHashMap<>() + } + + /** + * When a task is completed, track any temporary output files + * for automatic cleanup. + * + * @param handler + * @param trace + */ + @Override + void onProcessComplete(TaskHandler handler, TraceRecord trace) { + // find all temporary output files + final task = handler.task + final temporaryOutputs = task + .getOutputsByType(FileOutParam) + .findAll { param, paths -> param.temporary } + .values() + .flatten() + + if( temporaryOutputs.isEmpty() ) + return + + // update status tracker for the task's process + final processName = task.processor.name + + if( processName !in statusMap ) { + log.trace "Process ${processName} has temporary output files, tracking for automatic cleanup" + + statusMap[processName] = new Status( + paths: [] as Set, + processBarriers: findAllConsumers(processName) + ) + } + + statusMap[processName].paths.addAll(temporaryOutputs) + } + + /** + * Find all processes which are consumers of a given process. + * + * @param processName + */ + Set findAllConsumers(String processName) { + + // find the task's process node in the abstract dag + final processNode = dag.vertices + .find { node -> node.process?.name == processName } + + // find all downstream processes in the abstract dag + def consumers = [] as Set + def queue = [ processNode ] + + while( !queue.isEmpty() ) { + // remove a node from the search queue + final sourceNode = queue.remove(0) + + // search each outgoing edge from the source node + for( def edge : dag.edges ) { + if( edge.from != sourceNode ) + continue + + def node = edge.to + + // add process nodes to the list of consumers + if( node.process != null ) + consumers.add(node.process.name) + // add operator nodes to the queue to keep searching + else + queue.add(node) + } + } + + log.trace "Process ${processName} has the following consumers: ${consumers.join(', ')}" + + return consumers + } + + /** + * When a process is completed, update the status of any processes + * that are waiting on it in order to cleanup temporary outputs. + * + * If, after this update is removed, a process has no more barriers, + * then clean all temporary files for that process. + * + * @param process + */ + @Override + void onProcessTerminate(TaskProcessor process) { + for( def entry : statusMap ) { + // remove barrier from each upstream process + final producer = entry.key + final status = entry.value + final consumers = status.processBarriers + + consumers.remove(process.name) + + // if a process has no more barriers, trigger the cleanup + if( consumers.isEmpty() ) { + log.trace "All consumers of process ${producer} are complete, time to clean up temporary files" + + cleanTemporaryFiles(status.paths) + statusMap.remove(producer) + } + } + } + + void cleanTemporaryFiles(Collection paths) { + for( Path path : paths ) { + log.trace "Cleaning temporary file: ${path}" + + // get original file metadata + final attrs = Files.readAttributes(path, BasicFileAttributes.class) + + // delete file contents + try { + final file = new RandomAccessFile(path.toFile(), 'rw') + file.setLength(0) + file.setLength(attrs.size()) + file.close() + } + catch( UnsupportedOperationException e ) { + log.warn "Unable to truncate file, skipping: ${path.toUriString()}" + } + + // restore original file metadata + Files.setLastModifiedTime(path, attrs.lastModifiedTime()) + } + } + + @MapConstructor + static protected class Status { + Set paths + Set processBarriers + } + +} diff --git a/tests/temporary-outputs.nf b/tests/temporary-outputs.nf new file mode 100644 index 0000000000..5c73ab19ae --- /dev/null +++ b/tests/temporary-outputs.nf @@ -0,0 +1,49 @@ + +process foo { + input: + val meta_id + output: + tuple val(meta_id), path('a.txt', temporary: true) + + script: + """ + touch a.txt + echo 'foo was here' >> a.txt + """ +} + +process bar { + input: + tuple val(meta_id), path('a.txt') + output: + tuple val(meta_id), path('b.txt', temporary: true) + + script: + """ + cat a.txt > b.txt + echo 'bar was here' >> b.txt + """ +} + +process baz { + publishDir '.' + + input: + tuple val(meta_id), path('a.txt'), path('b.txt') + output: + tuple val(meta_id), path('c.txt') + + script: + """ + cat b.txt > c.txt + echo 'baz was here' >> c.txt + """ +} + +workflow { + meta_ids = Channel.of( '1', '2', '3' ) + ch_a = foo(meta_ids) + ch_b = bar(ch_a) + ch_ab = ch_a.join(ch_b) + ch_c = baz(ch_ab) +} From 3e55ad5d303042f8f3107d9bdcda51ded62e9665 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 31 Mar 2023 16:43:05 -0500 Subject: [PATCH 07/34] Fix failing test Signed-off-by: Ben Sherman --- .../src/test/groovy/nextflow/SessionTest.groovy | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy index ea1ce7e1dc..98efce7139 100644 --- a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy @@ -25,9 +25,10 @@ import nextflow.container.ContainerConfig import nextflow.exception.AbortOperationException import nextflow.script.ScriptFile import nextflow.script.WorkflowMetadata +import nextflow.trace.TemporaryFileObserver +import nextflow.trace.TraceFileObserver import nextflow.trace.TraceHelper import nextflow.trace.WorkflowStatsObserver -import nextflow.trace.TraceFileObserver import nextflow.util.Duration import nextflow.util.VersionNumber import spock.lang.Specification @@ -248,16 +249,16 @@ class SessionTest extends Specification { session = [:] as Session result = session.createObservers() then: - result.size()==1 + result.size() == 2 result.any { it instanceof WorkflowStatsObserver } + result.any { it instanceof TemporaryFileObserver } when: session = [:] as Session session.config = [trace: [enabled: true, file:'name.txt']] result = session.createObservers() - observer = result[1] as TraceFileObserver + observer = result.find { it instanceof TraceFileObserver } then: - result.size() == 2 observer.tracePath == Paths.get('name.txt').complete() observer.separator == '\t' @@ -265,9 +266,8 @@ class SessionTest extends Specification { session = [:] as Session session.config = [trace: [enabled: true, sep: 'x', fields: 'task_id,name,exit', file: 'alpha.txt']] result = session.createObservers() - observer = result[1] as TraceFileObserver + observer = result.find { it instanceof TraceFileObserver } then: - result.size() == 2 observer.tracePath == Paths.get('alpha.txt').complete() observer.separator == 'x' observer.fields == ['task_id','name','exit'] @@ -283,9 +283,8 @@ class SessionTest extends Specification { session = [:] as Session session.config = [trace: [enabled: true, fields: 'task_id,name,exit,vmem']] result = session.createObservers() - observer = result[1] as TraceFileObserver + observer = result.find { it instanceof TraceFileObserver } then: - result.size() == 2 observer.tracePath == Paths.get('trace-20221001.txt').complete() observer.separator == '\t' observer.fields == ['task_id','name','exit','vmem'] From e307f750b6bbbd6ada72ea7e166515e1c4c0f98b Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 31 Mar 2023 17:09:19 -0500 Subject: [PATCH 08/34] Add caveat about overlapping output channels [ci skip] Signed-off-by: Ben Sherman --- docs/process.rst | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/process.rst b/docs/process.rst index 082660ee18..e801395304 100644 --- a/docs/process.rst +++ b/docs/process.rst @@ -1180,6 +1180,10 @@ The following caveats apply when using temporary outputs: - A temporary output should not be forwarded by a downstream process using the ``includeInputs`` option. In this case, the temporary output will be cleaned prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. +- If a file captured by a temporary output path is also captured by a regular output path, it will still be treated as a temporary + file. While it is safe to declare multiple output channels in this way, if the regular output path is also published, it may lead + to some workflow outputs being empty. + - Directories and remote paths (e.g. S3) are not currently supported. From 08881b020848cbb9a6c98d851291dbeb8bdd75ca Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 31 Mar 2023 23:33:02 -0500 Subject: [PATCH 09/34] Delete files instead of emptying them (now supports directories and remote paths) Signed-off-by: Ben Sherman --- docs/process.rst | 24 ++++++------------ .../trace/TemporaryFileObserver.groovy | 25 ++++++------------- 2 files changed, 14 insertions(+), 35 deletions(-) diff --git a/docs/process.rst b/docs/process.rst index e801395304..5bbb73d179 100644 --- a/docs/process.rst +++ b/docs/process.rst @@ -1157,34 +1157,24 @@ Temporary output files .. warning:: This feature is experimental and may change in a future release. -When ``path`` output is declared with ``temporary: true``, any file associated with this output will be automatically "emptied" -during pipeline execution, as soon as it is no longer needed by downstream tasks. This feature is useful for deleting large -intermediate files that can be discarded once a pipeline run is finished. +When a ``path`` output is declared with ``temporary: true``, any file associated with this output will be automatically deleted +during pipeline execution, as soon as it is no longer needed by downstream tasks. This feature is useful for cleaning up large +intermediate files in order to free up disk storage. The lifetime of a temporary file is determined by the processes that are downstream of the file's originating process, either directly or indirectly through channel operators. When all of these processes finish (i.e. all of their tasks finish), all temporary files produced by the original process can be deleted. -"Emptying" a file means that the file contents are deleted, but the file metadata (size, last modified timestamp) is preserved, -such that it will be cached on a resumed run. - The following caveats apply when using temporary outputs: -- A pipeline run with temporary outputs should not be resumed if any regular downstream outputs have also been deleted. If a - temporary output is cached but a downstream output is not cached (e.g. because it was deleted or the process script was modified), - the downstream task will fail or produce incorrect output. - -- Temporary outputs will not be cached when the ``cache`` directive is set to ``'deep'``, because this mode includes the file - contents in the cache key. +- This feature will break the resumability of your pipeline. If you try to resume a run with temporary outputs, any tasks that + were cleaned will have to be re-run. - A temporary output should not be forwarded by a downstream process using the ``includeInputs`` option. In this case, the temporary - output will be cleaned prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. + output will be deleted prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. - If a file captured by a temporary output path is also captured by a regular output path, it will still be treated as a temporary - file. While it is safe to declare multiple output channels in this way, if the regular output path is also published, it may lead - to some workflow outputs being empty. - -- Directories and remote paths (e.g. S3) are not currently supported. + file. Declare multiple output channels in this way is safe to do as long as the regular output path isn't also published. .. _process-env: diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy index 62a1ba2415..db6ccde226 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -26,6 +26,7 @@ import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.DAG import nextflow.dag.ConcreteDAG +import nextflow.extension.FilesEx import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor import nextflow.script.params.FileOutParam @@ -146,32 +147,20 @@ class TemporaryFileObserver implements TraceObserver { if( consumers.isEmpty() ) { log.trace "All consumers of process ${producer} are complete, time to clean up temporary files" - cleanTemporaryFiles(status.paths) + deleteTemporaryFiles(status.paths) statusMap.remove(producer) } } } - void cleanTemporaryFiles(Collection paths) { + void deleteTemporaryFiles(Collection paths) { for( Path path : paths ) { log.trace "Cleaning temporary file: ${path}" - // get original file metadata - final attrs = Files.readAttributes(path, BasicFileAttributes.class) - - // delete file contents - try { - final file = new RandomAccessFile(path.toFile(), 'rw') - file.setLength(0) - file.setLength(attrs.size()) - file.close() - } - catch( UnsupportedOperationException e ) { - log.warn "Unable to truncate file, skipping: ${path.toUriString()}" - } - - // restore original file metadata - Files.setLastModifiedTime(path, attrs.lastModifiedTime()) + if( Files.isDirectory(path) ) + FilesEx.deleteDir(path) + else + Files.delete(path) } } From 0cf07ec4d031e825260ba90be3d6f18a6571dc2e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 10 Apr 2023 11:29:46 -0500 Subject: [PATCH 10/34] Add `eager' cleanup option Signed-off-by: Ben Sherman --- docs/config.rst | 21 ++++++++ docs/process.rst | 27 ---------- .../src/main/groovy/nextflow/Session.groovy | 2 +- .../script/params/FileOutParam.groovy | 5 -- .../trace/DefaultObserverFactory.groovy | 3 +- .../trace/TemporaryFileObserver.groovy | 48 +++++++----------- .../test/groovy/nextflow/SessionTest.groovy | 8 ++- tests/temporary-outputs.nf | 49 ------------------- 8 files changed, 50 insertions(+), 113 deletions(-) delete mode 100644 tests/temporary-outputs.nf diff --git a/docs/config.rst b/docs/config.rst index 770da9bb2b..cc473700e9 100644 --- a/docs/config.rst +++ b/docs/config.rst @@ -993,6 +993,27 @@ dumpHashes If ``true``, dump task hash keys in the log file, for debugg The use of the ``cleanup`` option will prevent the use of the *resume* feature on subsequent executions of that pipeline run. Also, be aware that deleting all scratch files can take a lot of time, especially when using a shared file system or remote cloud storage. +Since version ``23.10.0``, it is possible to use an "eager" cleanup strategy by setting ``cleanup = 'eager'``. This strategy +will try to delete each task directory as soon as it is no longer needed by downstream tasks, rather than at the end of the +workflow run. This feature can is useful for freeing up disk storage during the workflow run. + +The lifetime of a task is determined by the processes that are downstream of the underlying process, either directly or indirectly +through channel operators. When all of these "consumer" processes finish (i.e. all of their tasks finish), all tasks produced by the +upstream process can be deleted. + +The following caveats apply when using ``cleanup = 'eager'``: + +- Eager cleanup will break the resumability of your pipeline. If a workflow run fails, you will have to restart from the beginning, whereas + with ``cleanup = true`` the cleanup would not have happened. As a result, eager cleanup is designed to be used only when you are confident + that the workflow run will not fail. + +- Output files should not be published via symlink when using eager cleanup, because the symlinks will be invalidated when the original task + directory is deleted. Avoid using the following ``publishDir`` modes: ``copyNoFollow``, ``rellink``, ``symlink``. + +- Eager cleanup currently does not work properly with processes that forward input files with the ``includeInputs`` option. In this case, + the forwarded input files will be deleted prematurely, and any process that consumes the forwarded output channel may fail or produce + incorrect output. + .. _config-profiles: diff --git a/docs/process.rst b/docs/process.rst index 5bbb73d179..e24de605c8 100644 --- a/docs/process.rst +++ b/docs/process.rst @@ -1026,7 +1026,6 @@ Name Description ``type`` Type of paths returned, either ``file``, ``dir`` or ``any`` (default: ``any``, or ``file`` if the specified file name pattern contains a double star (``**``)) ``maxDepth`` Maximum number of directory levels to visit (default: no limit) ``includeInputs`` When ``true`` any input files matching an output file glob pattern are included. -``temporary`` When ``true`` the file will be "emptied" once it is no longer needed by downstream tasks. ================== ===================== The parenthesis are optional for input and output qualifiers, but when you want to set an additional option and there @@ -1151,32 +1150,6 @@ on the actual value of the ``species`` input. because it will result in simpler and more portable code. -Temporary output files ----------------------- - -.. warning:: - This feature is experimental and may change in a future release. - -When a ``path`` output is declared with ``temporary: true``, any file associated with this output will be automatically deleted -during pipeline execution, as soon as it is no longer needed by downstream tasks. This feature is useful for cleaning up large -intermediate files in order to free up disk storage. - -The lifetime of a temporary file is determined by the processes that are downstream of the file's originating process, -either directly or indirectly through channel operators. When all of these processes finish (i.e. all of their tasks finish), -all temporary files produced by the original process can be deleted. - -The following caveats apply when using temporary outputs: - -- This feature will break the resumability of your pipeline. If you try to resume a run with temporary outputs, any tasks that - were cleaned will have to be re-run. - -- A temporary output should not be forwarded by a downstream process using the ``includeInputs`` option. In this case, the temporary - output will be deleted prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. - -- If a file captured by a temporary output path is also captured by a regular output path, it will still be treated as a temporary - file. Declare multiple output channels in this way is safe to do as long as the regular output path isn't also published. - - .. _process-env: Output type ``env`` diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 87660008ac..37125ef5f0 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -1129,7 +1129,7 @@ class Session implements ISession { * Delete the workflow work directory from tasks temporary files */ void cleanup() { - if( !workDir || !config.cleanup ) + if( !workDir || config.cleanup != true ) return if( aborted || cancelled || error ) diff --git a/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy b/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy index 58bc52a446..e305c46c3e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/params/FileOutParam.groovy @@ -82,11 +82,6 @@ class FileOutParam extends BaseOutParam implements OutParam, OptionalParam, Path boolean glob = true - /** - * When true the file will be "cleaned" once it is no longer needed by downstream tasks. - */ - boolean temporary = false - private GString gstring private Closure dynamicObj private String filePattern diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index cd7bd72d7e..d8ef45eae9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -119,7 +119,8 @@ class DefaultObserverFactory implements TraceObserverFactory { } protected void createTemporaryFileObserver(Collection result) { - result << new TemporaryFileObserver() + if( session.config.cleanup == 'eager' ) + result << new TemporaryFileObserver() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy index db6ccde226..98b3fb9859 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -16,9 +16,7 @@ package nextflow.trace -import java.nio.file.Files import java.nio.file.Path -import java.nio.file.attribute.BasicFileAttributes import java.util.concurrent.ConcurrentHashMap import groovy.transform.MapConstructor @@ -26,10 +24,9 @@ import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.DAG import nextflow.dag.ConcreteDAG -import nextflow.extension.FilesEx +import nextflow.file.FileHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor -import nextflow.script.params.FileOutParam /** * Watch for temporary output files and "clean" them once they * are no longer needed. @@ -57,31 +54,21 @@ class TemporaryFileObserver implements TraceObserver { * @param trace */ @Override - void onProcessComplete(TaskHandler handler, TraceRecord trace) { - // find all temporary output files + synchronized void onProcessComplete(TaskHandler handler, TraceRecord trace) { + // add task directory to status map final task = handler.task - final temporaryOutputs = task - .getOutputsByType(FileOutParam) - .findAll { param, paths -> param.temporary } - .values() - .flatten() - - if( temporaryOutputs.isEmpty() ) - return - - // update status tracker for the task's process final processName = task.processor.name if( processName !in statusMap ) { - log.trace "Process ${processName} has temporary output files, tracking for automatic cleanup" + log.trace "Tracking process `${processName}` for automatic cleanup" statusMap[processName] = new Status( paths: [] as Set, - processBarriers: findAllConsumers(processName) + processBarriers: findAllConsumers(processName) ?: [processName] as Set ) } - statusMap[processName].paths.addAll(temporaryOutputs) + statusMap[processName].paths.add(task.workDir) } /** @@ -110,6 +97,10 @@ class TemporaryFileObserver implements TraceObserver { def node = edge.to + // skip if process is terminal + if( !node ) + continue + // add process nodes to the list of consumers if( node.process != null ) consumers.add(node.process.name) @@ -119,7 +110,7 @@ class TemporaryFileObserver implements TraceObserver { } } - log.trace "Process ${processName} has the following consumers: ${consumers.join(', ')}" + log.trace "Process `${processName}` is consumed by the following processes: ${consumers.collect({ "`${it}`" }).join(', ')}" return consumers } @@ -134,18 +125,20 @@ class TemporaryFileObserver implements TraceObserver { * @param process */ @Override - void onProcessTerminate(TaskProcessor process) { + synchronized void onProcessTerminate(TaskProcessor process) { + log.trace "Process `${process.name}` is complete, updating barriers for upstream processes" + for( def entry : statusMap ) { // remove barrier from each upstream process final producer = entry.key final status = entry.value - final consumers = status.processBarriers + final barriers = status.processBarriers - consumers.remove(process.name) + barriers.remove(process.name) // if a process has no more barriers, trigger the cleanup - if( consumers.isEmpty() ) { - log.trace "All consumers of process ${producer} are complete, time to clean up temporary files" + if( barriers.isEmpty() ) { + log.trace "All barriers for process `${producer}` are complete, time to clean up temporary files" deleteTemporaryFiles(status.paths) statusMap.remove(producer) @@ -157,10 +150,7 @@ class TemporaryFileObserver implements TraceObserver { for( Path path : paths ) { log.trace "Cleaning temporary file: ${path}" - if( Files.isDirectory(path) ) - FilesEx.deleteDir(path) - else - Files.delete(path) + FileHelper.deletePath(path) } } diff --git a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy index 98efce7139..2202c56764 100644 --- a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy @@ -249,8 +249,14 @@ class SessionTest extends Specification { session = [:] as Session result = session.createObservers() then: - result.size() == 2 + result.size() == 1 result.any { it instanceof WorkflowStatsObserver } + + when: + session = [config: [cleanup: 'eager']] as Session + result = session.createObservers() + then: + result.size() == 2 result.any { it instanceof TemporaryFileObserver } when: diff --git a/tests/temporary-outputs.nf b/tests/temporary-outputs.nf deleted file mode 100644 index 5c73ab19ae..0000000000 --- a/tests/temporary-outputs.nf +++ /dev/null @@ -1,49 +0,0 @@ - -process foo { - input: - val meta_id - output: - tuple val(meta_id), path('a.txt', temporary: true) - - script: - """ - touch a.txt - echo 'foo was here' >> a.txt - """ -} - -process bar { - input: - tuple val(meta_id), path('a.txt') - output: - tuple val(meta_id), path('b.txt', temporary: true) - - script: - """ - cat a.txt > b.txt - echo 'bar was here' >> b.txt - """ -} - -process baz { - publishDir '.' - - input: - tuple val(meta_id), path('a.txt'), path('b.txt') - output: - tuple val(meta_id), path('c.txt') - - script: - """ - cat b.txt > c.txt - echo 'baz was here' >> c.txt - """ -} - -workflow { - meta_ids = Channel.of( '1', '2', '3' ) - ch_a = foo(meta_ids) - ch_b = bar(ch_a) - ch_ab = ch_a.join(ch_b) - ch_c = baz(ch_ab) -} From 73b2f3bf0157032c0cc988ea6ff8e14750a921a7 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 10 Apr 2023 16:09:32 -0500 Subject: [PATCH 11/34] Fix concurrency issues [ci fast] Signed-off-by: Ben Sherman --- .../trace/TemporaryFileObserver.groovy | 106 +++++++++--------- .../test/groovy/nextflow/SessionTest.groovy | 3 +- 2 files changed, 52 insertions(+), 57 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy index 98b3fb9859..6aedebfac2 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -46,81 +46,75 @@ class TemporaryFileObserver implements TraceObserver { this.statusMap = new ConcurrentHashMap<>() } - /** - * When a task is completed, track any temporary output files - * for automatic cleanup. - * - * @param handler - * @param trace - */ @Override - synchronized void onProcessComplete(TaskHandler handler, TraceRecord trace) { - // add task directory to status map - final task = handler.task - final processName = task.processor.name + void onFlowBegin() { + + for( def processNode : dag.vertices ) { + // skip nodes that are not processes + if( !processNode.process ) + continue + + // find all downstream processes in the abstract dag + def processName = processNode.process.name + def consumers = [] as Set + def queue = [ processNode ] + + while( !queue.isEmpty() ) { + // remove a node from the search queue + final sourceNode = queue.remove(0) + + // search each outgoing edge from the source node + for( def edge : dag.edges ) { + if( edge.from != sourceNode ) + continue + + def node = edge.to + + // skip if process is terminal + if( !node ) + continue + + // add process nodes to the list of consumers + if( node.process != null ) + consumers.add(node.process.name) + // add operator nodes to the queue to keep searching + else + queue.add(node) + } + } - if( processName !in statusMap ) { - log.trace "Tracking process `${processName}` for automatic cleanup" + log.trace "Process `${processName}` is consumed by the following processes: ${consumers.collect({ "`${it}`" }).join(', ')}" statusMap[processName] = new Status( paths: [] as Set, - processBarriers: findAllConsumers(processName) ?: [processName] as Set + processBarriers: consumers ?: [processName] as Set ) } - - statusMap[processName].paths.add(task.workDir) } /** - * Find all processes which are consumers of a given process. + * When a task is started, track the task directory for automatic cleanup. * - * @param processName + * @param handler + * @param trace */ - Set findAllConsumers(String processName) { - - // find the task's process node in the abstract dag - final processNode = dag.vertices - .find { node -> node.process?.name == processName } - - // find all downstream processes in the abstract dag - def consumers = [] as Set - def queue = [ processNode ] - - while( !queue.isEmpty() ) { - // remove a node from the search queue - final sourceNode = queue.remove(0) - - // search each outgoing edge from the source node - for( def edge : dag.edges ) { - if( edge.from != sourceNode ) - continue - - def node = edge.to - - // skip if process is terminal - if( !node ) - continue - - // add process nodes to the list of consumers - if( node.process != null ) - consumers.add(node.process.name) - // add operator nodes to the queue to keep searching - else - queue.add(node) - } - } + @Override + synchronized void onProcessStart(TaskHandler handler, TraceRecord trace) { + // add task directory to status map + final task = handler.task + final processName = task.processor.name - log.trace "Process `${processName}` is consumed by the following processes: ${consumers.collect({ "`${it}`" }).join(', ')}" + log.trace "Task completed from process `${processName}`, tracking task directory for automatic cleanup" - return consumers + statusMap[processName].paths.add(task.workDir) } /** - * When a process is completed, update the status of any processes - * that are waiting on it in order to cleanup temporary outputs. + * When all tasks of a process are completed, update the status of any processes + * that are waiting on it in order to be cleaned up. * * If, after this update is removed, a process has no more barriers, - * then clean all temporary files for that process. + * then delete all temporary files for that process. * * @param process */ diff --git a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy index 2202c56764..ab2497f801 100644 --- a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy @@ -253,7 +253,8 @@ class SessionTest extends Specification { result.any { it instanceof WorkflowStatsObserver } when: - session = [config: [cleanup: 'eager']] as Session + session = [:] as Session + session.config = [cleanup: 'eager'] result = session.createObservers() then: result.size() == 2 From e81e584c27172f76dd65d8ed3b189e4b3e61ed9f Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 26 Apr 2023 11:54:54 -0500 Subject: [PATCH 12/34] Replace synchronized with lock Signed-off-by: Ben Sherman --- docs/config.md | 3 +- .../groovy/nextflow/dag/ConcreteDAG.groovy | 38 ++++++++++++++----- .../groovy/nextflow/dag/DagRenderer.groovy | 2 +- 3 files changed, 31 insertions(+), 12 deletions(-) diff --git a/docs/config.md b/docs/config.md index fec1b46930..a6bccaca22 100644 --- a/docs/config.md +++ b/docs/config.md @@ -484,8 +484,7 @@ The following settings are available: : When `true` overwrites any existing DAG file with the same name. `dag.type` -: Can be `abstract` to render the abstract DAG or `concrete` to render the concrete (task) DAG (default: `abstract`). - +: Can be `abstract` to render the abstract (process) DAG or `concrete` to render the concrete (task) DAG (default: `abstract`). Read the {ref}`dag-visualisation` page to learn more about the execution graph that can be generated by Nextflow. diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy index 5768916d16..c580f62147 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy @@ -18,6 +18,8 @@ package nextflow.dag import java.nio.file.Path import java.util.regex.Pattern +import java.util.concurrent.locks.Lock +import java.util.concurrent.locks.ReentrantLock import groovy.transform.MapConstructor import groovy.transform.ToString @@ -32,14 +34,20 @@ import nextflow.script.params.FileOutParam @Slf4j class ConcreteDAG { - Map nodes = new HashMap<>(100) + private Lock sync = new ReentrantLock() + + private Map nodes = new HashMap<>(100) + + Map getNodes() { + nodes + } /** * Add a task to the graph * * @param task */ - synchronized void addTask( TaskRun task ) { + void addTask(TaskRun task) { final hash = task.hash.toString() final label = "[${hash.substring(0,2)}/${hash.substring(2,8)}] ${task.name}" final inputs = task.getInputFilesMap() @@ -47,11 +55,17 @@ class ConcreteDAG { new Input(name: name, path: path, predecessor: getPredecessorHash(path)) } - nodes[hash] = new Task( - index: nodes.size(), - label: label, - inputs: inputs - ) + sync.lock() + try { + nodes[hash] = new Task( + index: nodes.size(), + label: label, + inputs: inputs + ) + } + finally { + sync.unlock() + } } static public String getPredecessorHash(Path path) { @@ -66,7 +80,7 @@ class ConcreteDAG { * * @param task */ - synchronized void addTaskOutputs( TaskRun task ) { + void addTaskOutputs(TaskRun task) { final hash = task.hash.toString() final outputs = task.getOutputsByType(FileOutParam) .values() @@ -75,7 +89,13 @@ class ConcreteDAG { new Output(name: path.name, path: path) } - nodes[hash].outputs = outputs + sync.lock() + try { + nodes[hash].outputs = outputs + } + finally { + sync.unlock() + } } @MapConstructor diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy index ccdb8fecc0..768c9ef11e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy @@ -26,7 +26,7 @@ import java.nio.file.Path trait DagRenderer { /** - * Render an abstract (process/operator) DAG. + * Render an abstract (process) DAG. */ void renderAbstractGraph(DAG dag, Path file) { throw new UnsupportedOperationException("Abstract graph rendering is not supported for this file format") From f7bcfa80bc2f257c42a0e5d48fb66ee334dcb8bd Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 28 Apr 2023 13:51:29 -0500 Subject: [PATCH 13/34] Remove dependency on task graph branch Signed-off-by: Ben Sherman --- docs/config.md | 3 - .../src/main/groovy/nextflow/Session.groovy | 9 -- .../groovy/nextflow/dag/ConcreteDAG.groovy | 125 ------------------ .../nextflow/dag/CytoscapeHtmlRenderer.groovy | 2 +- .../nextflow/dag/CytoscapeJsRenderer.groovy | 2 +- .../src/main/groovy/nextflow/dag/DAG.groovy | 2 +- .../groovy/nextflow/dag/DagRenderer.groovy | 15 +-- .../groovy/nextflow/dag/DotRenderer.groovy | 2 +- .../groovy/nextflow/dag/GexfRenderer.groovy | 2 +- .../nextflow/dag/GraphVizRenderer.groovy | 2 +- .../nextflow/dag/MermaidRenderer.groovy | 53 +------- .../groovy/nextflow/dag/NodeMarker.groovy | 6 +- .../nextflow/processor/TaskHandler.groovy | 28 ---- .../nextflow/processor/TaskProcessor.groovy | 2 +- .../groovy/nextflow/processor/TaskRun.groovy | 1 - .../trace/DefaultObserverFactory.groovy | 1 - .../nextflow/trace/GraphObserver.groovy | 66 ++++----- .../trace/TemporaryFileObserver.groovy | 1 - .../nextflow/dag/ConcreteDAGTest.groovy | 86 ------------ .../nextflow/dag/DotRendererTest.groovy | 2 +- .../nextflow/dag/GexfRendererTest.groovy | 2 +- .../nextflow/dag/MermaidRendererTest.groovy | 70 +--------- .../nextflow/processor/TaskHandlerTest.groovy | 25 ---- .../nextflow/trace/GraphObserverTest.groovy | 34 ++--- .../main/nextflow/extension/FilesEx.groovy | 20 --- .../main/nextflow/file/ETagAwareFile.groovy | 29 ---- .../main/nextflow/cloud/aws/nio/S3Path.java | 11 +- .../cloud/azure/nio/AzFileAttributes.groovy | 8 -- .../nextflow/cloud/azure/nio/AzPath.groovy | 8 +- 29 files changed, 73 insertions(+), 544 deletions(-) delete mode 100644 modules/nextflow/src/main/groovy/nextflow/dag/ConcreteDAG.groovy delete mode 100644 modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy delete mode 100644 modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy diff --git a/docs/config.md b/docs/config.md index 7ef580ea82..57465263e4 100644 --- a/docs/config.md +++ b/docs/config.md @@ -483,9 +483,6 @@ The following settings are available: `dag.overwrite` : When `true` overwrites any existing DAG file with the same name. -`dag.type` -: Can be `abstract` to render the abstract (process) DAG or `concrete` to render the concrete (task) DAG (default: `abstract`). - Read the {ref}`dag-visualisation` page to learn more about the execution graph that can be generated by Nextflow. (config-docker)= diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 37125ef5f0..5e6e330cf5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -39,7 +39,6 @@ import nextflow.conda.CondaConfig import nextflow.config.Manifest import nextflow.container.ContainerConfig import nextflow.dag.DAG -import nextflow.dag.ConcreteDAG import nextflow.exception.AbortOperationException import nextflow.exception.AbortSignalException import nextflow.exception.IllegalConfigException @@ -194,8 +193,6 @@ class Session implements ISession { private DAG dag - private ConcreteDAG concreteDag - private CacheDB cache private Barrier processesBarrier = new Barrier() @@ -348,7 +345,6 @@ class Session implements ISession { // -- DAG object this.dag = new DAG() - this.concreteDag = new ConcreteDAG() // -- init work dir this.workDir = ((config.workDir ?: 'work') as Path).complete() @@ -803,8 +799,6 @@ class Session implements ISession { DAG getDag() { this.dag } - ConcreteDAG getConcreteDAG() { this.concreteDag } - ExecutorService getExecService() { execService } /** @@ -1013,9 +1007,6 @@ class Session implements ISession { final trace = handler.safeTraceRecord() cache.putTaskAsync(handler, trace) - // save the task meta file to the task directory - handler.writeMetaFile() - // notify the event to the observers for( int i=0; i - */ -@Slf4j -class ConcreteDAG { - - private Lock sync = new ReentrantLock() - - private Map nodes = new HashMap<>(100) - - Map getNodes() { - nodes - } - - /** - * Add a task to the graph - * - * @param task - */ - void addTask(TaskRun task) { - final hash = task.hash.toString() - final label = "[${hash.substring(0,2)}/${hash.substring(2,8)}] ${task.name}" - final inputs = task.getInputFilesMap() - .collect { name, path -> - new Input(name: name, path: path, predecessor: getPredecessorHash(path)) - } - - sync.lock() - try { - nodes[hash] = new Task( - index: nodes.size(), - label: label, - inputs: inputs - ) - } - finally { - sync.unlock() - } - } - - static public String getPredecessorHash(Path path) { - final pattern = Pattern.compile('.*/([0-9a-f]{2}/[0-9a-f]{30})') - final matcher = pattern.matcher(path.toString()) - - matcher.find() ? matcher.group(1).replace('/', '') : null - } - - /** - * Add a task's outputs to the graph - * - * @param task - */ - void addTaskOutputs(TaskRun task) { - final hash = task.hash.toString() - final outputs = task.getOutputsByType(FileOutParam) - .values() - .flatten() - .collect { path -> - new Output(name: path.name, path: path) - } - - sync.lock() - try { - nodes[hash].outputs = outputs - } - finally { - sync.unlock() - } - } - - @MapConstructor - @ToString(includeNames = true, includes = 'label', includePackage=false) - static protected class Task { - int index - String label - List inputs - List outputs - - String getSlug() { "t${index}" } - } - - @MapConstructor - static protected class Input { - String name - Path path - String predecessor - } - - @MapConstructor - static protected class Output { - String name - Path path - } - -} diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy index e839824da8..5aff50ffe3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeHtmlRenderer.groovy @@ -28,7 +28,7 @@ import java.nio.file.Path class CytoscapeHtmlRenderer implements DagRenderer { @Override - void renderAbstractGraph(DAG dag, Path file) { + void renderDocument(DAG dag, Path file) { String tmplPage = readTemplate() String network = CytoscapeJsRenderer.renderNetwork(dag) file.text = tmplPage.replaceAll(~/\/\* REPLACE_WITH_NETWORK_DATA \*\//, network) diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy index af2a09b98e..dfcb8246df 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/CytoscapeJsRenderer.groovy @@ -28,7 +28,7 @@ import java.nio.file.Path class CytoscapeJsRenderer implements DagRenderer { @Override - void renderAbstractGraph(DAG dag, Path file) { + void renderDocument(DAG dag, Path file) { file.text = renderNetwork(dag) } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy index da4e5887ce..d1ef0c0666 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DAG.groovy @@ -43,7 +43,7 @@ import nextflow.script.params.TupleOutParam import java.util.concurrent.atomic.AtomicLong /** - * Model the abstract graph of a pipeline execution. + * Model a direct acyclic graph of the pipeline execution. * * @author Paolo Di Tommaso */ diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy index 768c9ef11e..0d3bfcd252 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DagRenderer.groovy @@ -23,19 +23,10 @@ import java.nio.file.Path * @author Paolo Di Tommaso * @author Mike Smoot */ -trait DagRenderer { +interface DagRenderer { /** - * Render an abstract (process) DAG. + * Render the dag to the specified file. */ - void renderAbstractGraph(DAG dag, Path file) { - throw new UnsupportedOperationException("Abstract graph rendering is not supported for this file format") - } - - /** - * Render a concrete (task) DAG. - */ - void renderConcreteGraph(ConcreteDAG dag, Path file) { - throw new UnsupportedOperationException("Concrete graph rendering is not supported for this file format") - } + void renderDocument(DAG dag, Path file); } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy index c5411f820b..f9b5a36e41 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/DotRenderer.groovy @@ -45,7 +45,7 @@ class DotRenderer implements DagRenderer { static String normalise(String str) { str.replaceAll(/[^0-9_A-Za-z]/,'') } @Override - void renderAbstractGraph(DAG dag, Path file) { + void renderDocument(DAG dag, Path file) { file.text = renderNetwork(dag) } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy index 43aaba6e9c..c2bac51110 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/GexfRenderer.groovy @@ -41,7 +41,7 @@ class GexfRenderer implements DagRenderer { } @Override - void renderAbstractGraph(DAG dag, Path file) { + void renderDocument(DAG dag, Path file) { final Charset charset = Charset.defaultCharset() Writer bw = Files.newBufferedWriter(file, charset) final XMLOutputFactory xof = XMLOutputFactory.newFactory() diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy index 0fddcee58d..3106890b9c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/GraphVizRenderer.groovy @@ -41,7 +41,7 @@ class GraphvizRenderer implements DagRenderer { * See http://www.graphviz.org for more info. */ @Override - void renderAbstractGraph(DAG dag, Path target) { + void renderDocument(DAG dag, Path target) { def result = Files.createTempFile('nxf-',".$format") def temp = Files.createTempFile('nxf-','.dot') // save the DAG as `dot` to a temp file diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy index fa8f6f602f..3afdb7cf42 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/MermaidRenderer.groovy @@ -26,7 +26,11 @@ import java.nio.file.Path class MermaidRenderer implements DagRenderer { @Override - void renderAbstractGraph(DAG dag, Path file) { + void renderDocument(DAG dag, Path file) { + file.text = renderNetwork(dag) + } + + String renderNetwork(DAG dag) { def lines = [] lines << "flowchart TD" @@ -40,7 +44,7 @@ class MermaidRenderer implements DagRenderer { lines << "" - file.text = lines.join('\n') + return lines.join('\n') } private String renderVertex(DAG.Vertex vertex) { @@ -71,49 +75,4 @@ class MermaidRenderer implements DagRenderer { return "${edge.from.name} -->${label} ${edge.to.name}" } - - @Override - void renderConcreteGraph(ConcreteDAG graph, Path file) { - def renderedOutputs = [] as Set - def numInputs = 0 - def numOutputs = 0 - - def lines = [] - lines << "flowchart TD" - - // render tasks and task inputs - graph.nodes.values().each { task -> - // render task node - lines << " ${task.getSlug()}[\"${task.label}\"]" - - task.inputs.each { input -> - // render task input from predecessor - if( input.predecessor != null ) { - final pred = graph.nodes[input.predecessor] - lines << " ${pred.getSlug()} -->|${input.name}| ${task.getSlug()}" - renderedOutputs << input.path - } - - // render task input from source node - else { - numInputs += 1 - lines << " i${numInputs}(( )) -->|${input.name}| ${task.getSlug()}" - } - } - } - - // render task outputs with sink nodes - graph.nodes.values().each { task -> - task.outputs.each { output -> - if( output.path !in renderedOutputs ) { - numOutputs += 1 - lines << " ${task.getSlug()} -->|${output.name}| o${numOutputs}(( ))" - } - } - } - - lines << "" - - file.text = lines.join('\n') - } } diff --git a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy index 18550b4cc6..68d56a12d5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/dag/NodeMarker.groovy @@ -40,7 +40,7 @@ class NodeMarker { static private Session getSession() { Global.session as Session } /** - * Creates a vertex in the abstract DAG representing a computing `process` + * Creates a new vertex in the DAG representing a computing `process` * * @param label The label associated to the process * @param inputs The list of inputs entering in the process @@ -52,7 +52,7 @@ class NodeMarker { } /** - * Creates a vertex in the abstract DAG representing a dataflow operator + * Creates a new DAG vertex representing a dataflow operator * * @param label The operator label * @param inputs The operator input(s). It can be either a single channel or a list of channels. @@ -66,7 +66,7 @@ class NodeMarker { } /** - * Creates a vertex in the abstract DAG representing a dataflow channel source. + * Creates a vertex in the DAG representing a dataflow channel source. * * @param label The node description * @param source Either a dataflow channel or a list of channel. diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy index 42ee522185..a94ded210f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskHandler.groovy @@ -18,14 +18,9 @@ package nextflow.processor import static nextflow.processor.TaskStatus.* -import java.nio.file.Files import java.nio.file.NoSuchFileException -import groovy.json.JsonBuilder import groovy.util.logging.Slf4j -import nextflow.dag.ConcreteDAG -import nextflow.extension.FilesEx -import nextflow.script.params.FileOutParam import nextflow.trace.TraceRecord /** * Actions to handle the underlying job running the user task. @@ -218,29 +213,6 @@ abstract class TaskHandler { return record } - void writeMetaFile() { - final record = [ - hash: task.hash.toString(), - inputs: task.getInputFilesMap().collect { name, path -> - [ - name: name, - path: path.toString(), - predecessor: ConcreteDAG.getPredecessorHash(path) - ] - }, - outputs: task.getOutputsByType(FileOutParam).values().flatten().collect { path -> - [ - name: path.name, - path: path.toString(), - size: Files.size(path), - checksum: FilesEx.getChecksum(path) - ] - } - ] - - task.workDir.resolve(TaskRun.CMD_META).text = new JsonBuilder(record).toString() - } - /** * Determine if a process can be forked i.e. can launch * a parallel task execution. This is only enforced when diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 147de2d0d6..45af17c644 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -543,7 +543,7 @@ class TaskProcessor { def invoke = new InvokeTaskAdapter(this, opInputs.size()) session.allOperators << (operator = new DataflowOperator(group, params, invoke)) - // notify the creation of a new process in the abstract DAG + // notify the creation of a new vertex the execution DAG NodeMarker.addProcessNode(this, config.getInputs(), config.getOutputs()) // fix issue #41 diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index 703109d9c6..a45382a144 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -550,7 +550,6 @@ class TaskRun implements Cloneable { static final public String CMD_STAGE = '.command.stage' static final public String CMD_TRACE = '.command.trace' static final public String CMD_ENV = '.command.env' - static final public String CMD_META = '.command.meta.json' String toString( ) { diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index d8ef45eae9..b0cb7f03a3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -94,7 +94,6 @@ class DefaultObserverFactory implements TraceObserverFactory { if( !fileName ) fileName = GraphObserver.DEF_FILE_NAME def traceFile = (fileName as Path).complete() def observer = new GraphObserver(traceFile) - config.navigate('dag.type') { observer.type = it ?: 'abstract' } config.navigate('dag.overwrite') { observer.overwrite = it } result << observer } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy index 23c565277d..0a61d43eda 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/GraphObserver.groovy @@ -23,7 +23,6 @@ import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.CytoscapeHtmlRenderer import nextflow.dag.DAG -import nextflow.dag.ConcreteDAG import nextflow.dag.DagRenderer import nextflow.dag.DotRenderer import nextflow.dag.GexfRenderer @@ -46,11 +45,7 @@ class GraphObserver implements TraceObserver { private Path file - private String type = 'abstract' - - private DAG abstractDag - - private ConcreteDAG concreteDag + private DAG dag private String name @@ -71,8 +66,7 @@ class GraphObserver implements TraceObserver { @Override void onFlowCreate(Session session) { - this.abstractDag = session.dag - this.concreteDag = session.concreteDag + this.dag = session.dag // check file existance final attrs = FileHelper.readAttributes(file) if( attrs ) { @@ -83,40 +77,16 @@ class GraphObserver implements TraceObserver { } } - @Override - void onProcessSubmit(TaskHandler handler, TraceRecord trace) { - concreteDag.addTask( handler.task ) - } - - @Override - void onProcessComplete(TaskHandler handler, TraceRecord trace) { - concreteDag.addTaskOutputs( handler.task ) - } - - @Override - void onProcessCached(TaskHandler handler, TraceRecord trace) { - concreteDag.addTask( handler.task ) - concreteDag.addTaskOutputs( handler.task ) - } - @Override void onFlowComplete() { - if( type == 'abstract' ) { - // -- normalise the DAG - abstractDag.normalize() - // -- render it to a file - createRenderer().renderAbstractGraph(abstractDag,file) - } - else if( type == 'concrete' ) { - createRenderer().renderConcreteGraph(concreteDag,file) - } - else { - log.warn("Invalid DAG type '${type}', should be 'abstract' or 'concrete'") - } + // -- normalise the DAG + dag.normalize() + // -- render it to a file + createRender().renderDocument(dag,file) } @PackageScope - DagRenderer createRenderer() { + DagRenderer createRender() { if( format == 'dot' ) new DotRenderer(name) @@ -133,6 +103,28 @@ class GraphObserver implements TraceObserver { new GraphvizRenderer(name, format) } + + @Override + void onProcessCreate(TaskProcessor process) { + + } + + + @Override + void onProcessSubmit(TaskHandler handler, TraceRecord trace) { + + } + + @Override + void onProcessStart(TaskHandler handler, TraceRecord trace) { + + } + + @Override + void onProcessComplete(TaskHandler handler, TraceRecord trace) { + + } + @Override boolean enableMetrics() { return false diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy index 6aedebfac2..398f6e65fe 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -23,7 +23,6 @@ import groovy.transform.MapConstructor import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.DAG -import nextflow.dag.ConcreteDAG import nextflow.file.FileHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy deleted file mode 100644 index a666372169..0000000000 --- a/modules/nextflow/src/test/groovy/nextflow/dag/ConcreteDAGTest.groovy +++ /dev/null @@ -1,86 +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.dag - -import java.nio.file.Paths - -import com.google.common.hash.HashCode -import nextflow.processor.TaskRun -import spock.lang.Specification -/** - * - * @author Ben Sherman - */ -class ConcreteDAGTest extends Specification { - - def 'should add task nodes and outputs' () { - - given: - def task1 = Mock(TaskRun) { - getHash() >> HashCode.fromString('00112233445566778899aabbccddeeff') - getName() >> 'foo' - getInputFilesMap() >> [ - 'data.txt': Paths.get('/inputs/data.txt') - ] - getOutputsByType(_) >> [ - 'data.foo': Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') - ] - } - def task2 = Mock(TaskRun) { - getHash() >> HashCode.fromString('aabbccddeeff00112233445566778899') - getName() >> 'bar' - getInputFilesMap() >> [ - 'data.foo': Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') - ] - getOutputsByType(_) >> [ - 'data.bar': Paths.get('/work/aa/bbccddeeff00112233445566778899/data.bar') - ] - } - def dag = new ConcreteDAG() - - when: - dag.addTask( task1 ) - dag.addTask( task2 ) - def node1 = dag.nodes['00112233445566778899aabbccddeeff'] - def node2 = dag.nodes['aabbccddeeff00112233445566778899'] - then: - node1.index == 0 - node1.label == '[00/112233] foo' - node1.inputs.size() == 1 - node1.inputs[0].name == 'data.txt' - node1.inputs[0].path == Paths.get('/inputs/data.txt') - node1.inputs[0].predecessor == null - node2.index == 1 - node2.label == '[aa/bbccdd] bar' - node2.inputs.size() == 1 - node2.inputs[0].name == 'data.foo' - node2.inputs[0].path == Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') - node2.inputs[0].predecessor == '00112233445566778899aabbccddeeff' - - when: - dag.addTaskOutputs( task1 ) - dag.addTaskOutputs( task2 ) - then: - node1.outputs.size() == 1 - node1.outputs[0].name == 'data.foo' - node1.outputs[0].path == Paths.get('/work/00/112233445566778899aabbccddeeff/data.foo') - node2.outputs.size() == 1 - node2.outputs[0].name == 'data.bar' - node2.outputs[0].path == Paths.get('/work/aa/bbccddeeff00112233445566778899/data.bar') - } - -} diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy index 7e5921aeef..2ff86ebaf7 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/DotRendererTest.groovy @@ -54,7 +54,7 @@ class DotRendererTest extends Specification { dag.normalize() when: - new DotRenderer('TheGraph').renderAbstractGraph(dag, file) + new DotRenderer('TheGraph').renderDocument(dag, file) then: file.text == ''' diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy index 6d4a9e3539..f112732b2e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/GexfRendererTest.groovy @@ -47,7 +47,7 @@ class GexfRendererTest extends Specification { dag.normalize() when: - new GexfRenderer('TheGraph').renderAbstractGraph(dag, file.toPath()) + new GexfRenderer('TheGraph').renderDocument(dag, file.toPath()) then: def graph = new XmlSlurper().parse(file); assert graph.name() == 'gexf' diff --git a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy index b0d1226727..220422ac77 100644 --- a/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/dag/MermaidRendererTest.groovy @@ -15,14 +15,13 @@ */ package nextflow.dag - import java.nio.file.Files -import java.nio.file.Paths import groovyx.gpars.dataflow.DataflowQueue -import nextflow.Session import spock.lang.Specification +import nextflow.Session + /** * * @author Ben Sherman @@ -33,7 +32,7 @@ class MermaidRendererTest extends Specification { new Session() } - def 'should render an abstract graph using the `mmd` format' () { + def 'should render a graph using the `mmd` format' () { given: def file = Files.createTempFile('test', null) def ch1 = new DataflowQueue() @@ -47,7 +46,7 @@ class MermaidRendererTest extends Specification { dag.normalize() when: - new MermaidRenderer().renderAbstractGraph(dag, file) + new MermaidRenderer().renderDocument(dag, file) then: file.text == ''' @@ -65,65 +64,4 @@ class MermaidRendererTest extends Specification { cleanup: file.delete() } - - def 'should render a concrete graph using the `mmd` format' () { - given: - def file = Files.createTempFile('test', null) - - def dag = Mock(ConcreteDAG) { - nodes >> [ - '012345': new ConcreteDAG.Task( - index: 1, - label: 'foo', - inputs: [ - new ConcreteDAG.Input( - name: 'data.txt', - path: Paths.get('/inputs/data.txt'), - predecessor: null - ) - ], - outputs: [ - new ConcreteDAG.Output( - name: 'data.foo', - path: Paths.get('/work/012345/data.foo'), - ) - ] - ), - 'abcdef': new ConcreteDAG.Task( - index: 2, - label: 'bar', - inputs: [ - new ConcreteDAG.Input( - name: 'data.foo', - path: Paths.get('/work/012345/data.foo'), - predecessor: '012345' - ) - ], - outputs: [ - new ConcreteDAG.Output( - name: 'data.bar', - path: Paths.get('/work/abcdef/data.bar'), - ) - ] - ) - ] - } - - when: - new MermaidRenderer().renderConcreteGraph(dag, file) - then: - file.text == - ''' - flowchart TD - t1["foo"] - i1(( )) -->|data.txt| t1 - t2["bar"] - t1 -->|data.foo| t2 - t2 -->|data.bar| o1(( )) - ''' - .stripIndent().leftTrim() - - cleanup: - file.delete() - } } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy index ad94ce47a7..1846fb0838 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskHandlerTest.groovy @@ -16,10 +16,8 @@ package nextflow.processor -import java.nio.file.Paths import java.util.concurrent.atomic.LongAdder -import com.google.common.hash.HashCode import nextflow.Session import nextflow.executor.Executor import nextflow.util.Duration @@ -137,29 +135,6 @@ class TaskHandlerTest extends Specification { } - def 'should write meta file' () { - - given: - def folder = File.createTempDir() - def outputFile = new File(folder, 'bar.txt') ; outputFile.text = 'bar' - def task = Mock(TaskRun) { - hash >> HashCode.fromString('aabbccddeeff00112233445566778899') - workDir >> folder.toPath() - getInputFilesMap() >> [ 'foo.txt': Paths.get('/tmp/00/112233445566778899aabbccddeeff/foo.txt') ] - getOutputsByType(_) >> [ 'bar.txt': outputFile.toPath() ] - } - def handler = [:] as TaskHandler - handler.task = task - - when: - handler.writeMetaFile() - then: - task.workDir.resolve(TaskRun.CMD_META).text == """{"hash":"aabbccddeeff00112233445566778899","inputs":[{"name":"foo.txt","path":"/tmp/00/112233445566778899aabbccddeeff/foo.txt","predecessor":"00112233445566778899aabbccddeeff"}],"outputs":[{"name":"bar.txt","path":"${folder}/bar.txt","size":3,"checksum":"37b51d194a7513e45b56f6524f2d51f2"}]}""" - - cleanup: - folder.delete() - } - LongAdder _adder(Integer x) { if( x != null ) { def adder = new LongAdder() diff --git a/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy b/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy index 384c737944..f3d13bc507 100644 --- a/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/trace/GraphObserverTest.groovy @@ -35,7 +35,7 @@ import test.TestHelper */ class GraphObserverTest extends Specification { - DAG dag + DAG test_dag def setup() { new Session() @@ -45,28 +45,28 @@ class GraphObserverTest extends Specification { def ch2 = new DataflowQueue() def ch3 = new DataflowQueue() - dag = new DAG() + test_dag = new DAG() - dag.addVertex( + test_dag.addVertex( DAG.Type.ORIGIN, 'Source', null, [ new DAG.ChannelHandler(channel: src, label: 'src') ] ) - dag.addVertex( + test_dag.addVertex( DAG.Type.PROCESS, 'Process 1', [ new DAG.ChannelHandler(channel: src, label: 'Source') ], [ new DAG.ChannelHandler(channel: ch1, label: 'Channel 1') ] ) - dag.addVertex( + test_dag.addVertex( DAG.Type.OPERATOR, 'Filter', [ new DAG.ChannelHandler(channel: ch1, label: 'Channel 1') ], [ new DAG.ChannelHandler(channel: ch2, label: 'Channel 2') ] ) - dag.addVertex( + test_dag.addVertex( DAG.Type.PROCESS, 'Process 2', [ new DAG.ChannelHandler(channel: ch2, label: 'Channel 2') ], @@ -78,7 +78,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf_','.dot') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -105,7 +105,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.html') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -133,7 +133,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.svg') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -151,7 +151,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.png') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -168,7 +168,7 @@ class GraphObserverTest extends Specification { given: def file = Files.createTempFile('nxf-','.pdf') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -185,7 +185,7 @@ class GraphObserverTest extends Specification { def folder = Files.createTempDirectory('test') def file = folder.resolve('nope') def gr = new GraphObserver(file) - gr.abstractDag = dag + gr.dag = test_dag when: gr.onFlowComplete() @@ -216,34 +216,34 @@ class GraphObserverTest extends Specification { then: observer.name == 'hello-world' observer.format == 'dot' - observer.createRenderer() instanceof DotRenderer + observer.createRender() instanceof DotRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.html')) then: observer.name == 'TheGraph' observer.format == 'html' - observer.createRenderer() instanceof CytoscapeHtmlRenderer + observer.createRender() instanceof CytoscapeHtmlRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.mmd')) then: observer.name == 'TheGraph' observer.format == 'mmd' - observer.createRenderer() instanceof MermaidRenderer + observer.createRender() instanceof MermaidRenderer when: observer = new GraphObserver(Paths.get('/path/to/TheGraph.SVG')) then: observer.name == 'TheGraph' observer.format == 'svg' - observer.createRenderer() instanceof GraphvizRenderer + observer.createRender() instanceof GraphvizRenderer when: observer = new GraphObserver(Paths.get('/path/to/anonymous')) then: observer.name == 'anonymous' observer.format == 'dot' - observer.createRenderer() instanceof DotRenderer + observer.createRender() instanceof DotRenderer } } diff --git a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy index 026d0dbc92..389dab5107 100644 --- a/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy +++ b/modules/nf-commons/src/main/nextflow/extension/FilesEx.groovy @@ -34,14 +34,12 @@ import java.nio.file.attribute.FileAttribute import java.nio.file.attribute.FileTime import java.nio.file.attribute.PosixFilePermission import java.nio.file.attribute.PosixFilePermissions -import java.security.MessageDigest import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.transform.stc.ClosureParams import groovy.transform.stc.FromString import groovy.util.logging.Slf4j -import nextflow.file.ETagAwareFile import nextflow.file.FileHelper import nextflow.file.FileSystemPathFactory import nextflow.io.ByteBufferBackedInputStream @@ -1601,22 +1599,4 @@ class FilesEx { static String getScheme(Path path) { path.getFileSystem().provider().getScheme() } - - static String getChecksum(Path path) { - if( Files.isDirectory(path) ) - return null - - if( path instanceof ETagAwareFile ) - return ((ETagAwareFile)path).getETag() - - final is = Files.newInputStream(path) - final md = MessageDigest.getInstance('MD5') - final buf = new byte[16 << 10] - - int len - while( (len=is.read(buf)) != -1 ) - md.update(buf, 0, len) - - new BigInteger(1, md.digest()).toString(16) - } } diff --git a/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy b/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy deleted file mode 100644 index f1c40073b3..0000000000 --- a/modules/nf-commons/src/main/nextflow/file/ETagAwareFile.groovy +++ /dev/null @@ -1,29 +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.file - -/** - * Defines the interface for files that have an ETag - * - * @author Ben Sherman - */ -interface ETagAwareFile { - - String getETag() - -} diff --git a/plugins/nf-amazon/src/main/nextflow/cloud/aws/nio/S3Path.java b/plugins/nf-amazon/src/main/nextflow/cloud/aws/nio/S3Path.java index a7b7a4e1ed..5ce117ac6f 100644 --- a/plugins/nf-amazon/src/main/nextflow/cloud/aws/nio/S3Path.java +++ b/plugins/nf-amazon/src/main/nextflow/cloud/aws/nio/S3Path.java @@ -42,14 +42,13 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import nextflow.file.ETagAwareFile; import nextflow.file.TagAwareFile; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Iterables.transform; import static java.lang.String.format; -public class S3Path implements Path, ETagAwareFile, TagAwareFile { +public class S3Path implements Path, TagAwareFile { public static final String PATH_SEPARATOR = "/"; /** @@ -555,14 +554,6 @@ public String getContentType() { return contentType; } - @Override - public String getETag() { - return fileSystem - .getClient() - .getObjectMetadata(getBucket(), getKey()) - .getETag(); - } - public String getStorageClass() { return storageClass; } diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy index ba07cda22f..a955888d70 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzFileAttributes.groovy @@ -46,8 +46,6 @@ class AzFileAttributes implements BasicFileAttributes { private String objectId - private String etag - static AzFileAttributes root() { new AzFileAttributes(size: 0, objectId: '/', directory: true) } @@ -62,7 +60,6 @@ class AzFileAttributes implements BasicFileAttributes { updateTime = time(props.getLastModified()) directory = client.blobName.endsWith('/') size = props.getBlobSize() - etag = props.getETag() } AzFileAttributes(String containerName, BlobItem item) { @@ -72,7 +69,6 @@ class AzFileAttributes implements BasicFileAttributes { creationTime = time(item.properties.getCreationTime()) updateTime = time(item.properties.getLastModified()) size = item.properties.getContentLength() - etag = item.properties.getETag() } } @@ -148,10 +144,6 @@ class AzFileAttributes implements BasicFileAttributes { return objectId } - String getETag() { - return etag - } - @Override boolean equals( Object obj ) { if( this.class != obj?.class ) return false diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy index 3bdd222f6b..2f654b4ad8 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/nio/AzPath.groovy @@ -29,7 +29,6 @@ import com.azure.storage.blob.models.BlobItem import groovy.transform.CompileStatic import groovy.transform.EqualsAndHashCode import groovy.transform.PackageScope -import nextflow.file.ETagAwareFile /** * Implements Azure path object @@ -38,7 +37,7 @@ import nextflow.file.ETagAwareFile */ @CompileStatic @EqualsAndHashCode(includes = 'fs,path,directory', includeFields = true) -class AzPath implements Path, ETagAwareFile { +class AzPath implements Path { private AzFileSystem fs @@ -306,11 +305,6 @@ class AzPath implements Path, ETagAwareFile { return this.toString() <=> other.toString() } - @Override - String getETag() { - return attributes.getETag() - } - String getContainerName() { if( path.isAbsolute() ) { path.nameCount==0 ? '/' : path.getName(0) From 4d90e277fc339e649f6e789e884f8a1e8aaf139d Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 28 Apr 2023 14:12:19 -0500 Subject: [PATCH 14/34] Use downstream tasks to determine lifetime for task cleanup Signed-off-by: Ben Sherman --- .../src/main/groovy/nextflow/Session.groovy | 11 ++ .../nextflow/processor/TaskProcessor.groovy | 5 + .../trace/TemporaryFileObserver.groovy | 155 +++++++++++++----- .../nextflow/trace/TraceObserver.groovy | 5 + 4 files changed, 133 insertions(+), 43 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 5e6e330cf5..e74d515e30 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -935,6 +935,17 @@ class Session implements ISession { } } + void notifyProcessClose(TaskProcessor 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 it.incCompleted() } } + protected void closeProcess() { + session.notifyProcessClose(this) + } + protected void terminateProcess() { log.trace "<${name}> Sending poison pills and terminating process" sendPoisonPill() @@ -2440,6 +2444,7 @@ class TaskProcessor { @Override void afterStop(final DataflowProcessor processor) { log.trace "<${name}> After stop" + closeProcess() } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy index 398f6e65fe..cc5fb13a24 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy @@ -18,33 +18,46 @@ package nextflow.trace import java.nio.file.Path import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.locks.Lock +import java.util.concurrent.locks.ReentrantLock -import groovy.transform.MapConstructor +import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.DAG import nextflow.file.FileHelper import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor +import nextflow.processor.TaskRun +import nextflow.script.params.FileOutParam /** - * Watch for temporary output files and "clean" them once they - * are no longer needed. + * Delete task directories once they are no longer needed. * * @author Ben Sherman */ @Slf4j +@CompileStatic class TemporaryFileObserver implements TraceObserver { private DAG dag - private Map statusMap + private Map processes = new HashMap<>() + + private Map tasks = new HashMap<>() + + private Map taskLookup = new HashMap<>() + + private Lock sync = new ReentrantLock() @Override void onFlowCreate(Session session) { this.dag = session.dag - this.statusMap = new ConcurrentHashMap<>() } + /** + * When the workflow begins, determine the consumers of each process + * in the DAG. + */ @Override void onFlowBegin() { @@ -84,73 +97,129 @@ class TemporaryFileObserver implements TraceObserver { log.trace "Process `${processName}` is consumed by the following processes: ${consumers.collect({ "`${it}`" }).join(', ')}" - statusMap[processName] = new Status( - paths: [] as Set, - processBarriers: consumers ?: [processName] as Set - ) + processes[processName] = new ProcessState(consumers ?: [processName] as Set) } } /** - * When a task is started, track the task directory for automatic cleanup. + * When a task is created, add it to the state map and add it as a consumer + * of any task whose output it takes as input. * * @param handler * @param trace */ @Override - synchronized void onProcessStart(TaskHandler handler, TraceRecord trace) { - // add task directory to status map + void onProcessPending(TaskHandler handler, TraceRecord trace) { + // query task input files final task = handler.task - final processName = task.processor.name + final inputs = task.getInputFilesMap().values() - log.trace "Task completed from process `${processName}`, tracking task directory for automatic cleanup" + sync.lock() + try { + // add task to the task state map + tasks[task] = new TaskState() - statusMap[processName].paths.add(task.workDir) + // add task as consumer to each task whoes output it takes as input + for( Path path : inputs ) + if( path in taskLookup ) + tasks[taskLookup[path]].consumers.add(task) + } + finally { + sync.unlock() + } } /** - * When all tasks of a process are completed, update the status of any processes - * that are waiting on it in order to be cleaned up. + * When a task is completed, track any temporary output files + * for automatic cleanup. * - * If, after this update is removed, a process has no more barriers, - * then delete all temporary files for that process. + * @param handler + * @param trace + */ + @Override + void onProcessComplete(TaskHandler handler, TraceRecord trace) { + // query task output files + final task = handler.task + final outputs = task + .getOutputsByType(FileOutParam) + .values() + .flatten() as List + + sync.lock() + try { + // mark task as completed + tasks[task].completed = true + + // scan tasks for cleanup + cleanup0() + + // add new output files to task lookup + for( Path path : outputs ) + taskLookup[path] = task + } + finally { + sync.unlock() + } + } + + /** + * When a process is closed (all tasks of the process have been created), + * mark the process as closed and scan tasks for cleanup. * * @param process */ @Override - synchronized void onProcessTerminate(TaskProcessor process) { - log.trace "Process `${process.name}` is complete, updating barriers for upstream processes" - - for( def entry : statusMap ) { - // remove barrier from each upstream process - final producer = entry.key - final status = entry.value - final barriers = status.processBarriers - - barriers.remove(process.name) - - // if a process has no more barriers, trigger the cleanup - if( barriers.isEmpty() ) { - log.trace "All barriers for process `${producer}` are complete, time to clean up temporary files" + void onProcessClose(TaskProcessor process) { + sync.lock() + try { + processes[process.name].closed = true + cleanup0() + } + finally { + sync.unlock() + } + } - deleteTemporaryFiles(status.paths) - statusMap.remove(producer) + /** + * Delete any task directories that have no more barriers. + */ + private void cleanup0() { + for( TaskRun task : tasks.keySet() ) { + final taskState = tasks[task] + if( taskState.completed && !taskState.deleted && canDelete(task) ) { + log.trace "Deleting task directory: ${task.workDir.toUriString()}" + FileHelper.deletePath(task.workDir) + taskState.deleted = true } } } - void deleteTemporaryFiles(Collection paths) { - for( Path path : paths ) { - log.trace "Cleaning temporary file: ${path}" + /** + * Determine whether a task directory can be deleted. + * + * A task directory can be deleted if all of its process consumers + * are closed and all of its task consumers are completed. + */ + private boolean canDelete(TaskRun task) { + final taskState = tasks[task] + final processConsumers = processes[task.processor.name].consumers + final taskConsumers = taskState.consumers + processConsumers.every { p -> processes[p].closed } && taskConsumers.every { t -> tasks[t].completed } + } + + static private class ProcessState { + Set consumers + boolean closed = false - FileHelper.deletePath(path) + ProcessState(Set consumers) { + this.consumers = consumers } } - @MapConstructor - static protected class Status { - Set paths - Set processBarriers + static private class TaskState { + Set consumers = [] as Set + boolean completed = false + boolean deleted = false } } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy index 45e414764e..aacdbe95de 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TraceObserver.groovy @@ -50,6 +50,11 @@ trait TraceObserver { */ void onProcessCreate( TaskProcessor process ){} + /** + * Invoked when the process is closed (all tasks have been created). + */ + void onProcessClose( TaskProcessor process ){} + /* * Invoked when all tak have been executed and process ends. */ From dd23b2ac19571bf2a581f62d6d6f63e561fe9401 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 6 Jul 2023 23:50:03 -0500 Subject: [PATCH 15/34] Rename TemporaryFileObserver to TaskCleanupObserver Signed-off-by: Ben Sherman --- .../groovy/nextflow/trace/DefaultObserverFactory.groovy | 6 +++--- ...poraryFileObserver.groovy => TaskCleanupObserver.groovy} | 4 ++-- .../nextflow/src/test/groovy/nextflow/SessionTest.groovy | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) rename modules/nextflow/src/main/groovy/nextflow/trace/{TemporaryFileObserver.groovy => TaskCleanupObserver.groovy} (97%) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index b0cb7f03a3..017b461e9d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -26,7 +26,7 @@ class DefaultObserverFactory implements TraceObserverFactory { createDagObserver(result) createWebLogObserver(result) createAnsiLogObserver(result) - createTemporaryFileObserver(result) + createTaskCleanupObserver(result) return result } @@ -117,9 +117,9 @@ class DefaultObserverFactory implements TraceObserverFactory { result << observer } - protected void createTemporaryFileObserver(Collection result) { + protected void createTaskCleanupObserver(Collection result) { if( session.config.cleanup == 'eager' ) - result << new TemporaryFileObserver() + result << new TaskCleanupObserver() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy similarity index 97% rename from modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy rename to modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index cc5fb13a24..0e2079b2b5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TemporaryFileObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -37,7 +37,7 @@ import nextflow.script.params.FileOutParam */ @Slf4j @CompileStatic -class TemporaryFileObserver implements TraceObserver { +class TaskCleanupObserver implements TraceObserver { private DAG dag @@ -204,7 +204,7 @@ class TemporaryFileObserver implements TraceObserver { final taskState = tasks[task] final processConsumers = processes[task.processor.name].consumers final taskConsumers = taskState.consumers - processConsumers.every { p -> processes[p].closed } && taskConsumers.every { t -> tasks[t].completed } + processConsumers.every( p -> processes[p].closed ) && taskConsumers.every( t -> tasks[t].completed ) } static private class ProcessState { diff --git a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy index ab2497f801..5a66fe3dee 100644 --- a/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/SessionTest.groovy @@ -25,7 +25,7 @@ import nextflow.container.ContainerConfig import nextflow.exception.AbortOperationException import nextflow.script.ScriptFile import nextflow.script.WorkflowMetadata -import nextflow.trace.TemporaryFileObserver +import nextflow.trace.TaskCleanupObserver import nextflow.trace.TraceFileObserver import nextflow.trace.TraceHelper import nextflow.trace.WorkflowStatsObserver @@ -258,7 +258,7 @@ class SessionTest extends Specification { result = session.createObservers() then: result.size() == 2 - result.any { it instanceof TemporaryFileObserver } + result.any { it instanceof TaskCleanupObserver } when: session = [:] as Session From ff08984be1d5e2335401f823955f8c62f092f299 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 7 Jul 2023 02:01:35 -0500 Subject: [PATCH 16/34] Wait for output files to be published Signed-off-by: Ben Sherman --- .../nextflow/processor/PublishDir.groovy | 26 ++-- .../nextflow/trace/TaskCleanupObserver.groovy | 111 +++++++++++++++--- 2 files changed, 112 insertions(+), 25 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy index e6f6b64223..0693ae867e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy @@ -290,13 +290,8 @@ class PublishDir { @CompileStatic protected void apply1(Path source, boolean inProcess ) { - def target = sourceDir ? sourceDir.relativize(source) : source.getFileName() - if( matcher && !matcher.matches(target) ) { - // skip not matching file - return - } - - if( saveAs && !(target=saveAs.call(target.toString()))) { + final target = getPublishTarget(source) + if( !target ) { // skip this file return } @@ -328,6 +323,23 @@ class PublishDir { } + @CompileStatic + boolean canPublish(Path source) { + return getPublishTarget(source) != null + } + + @CompileStatic + protected def getPublishTarget(Path source) { + def target = sourceDir ? sourceDir.relativize(source) : source.getFileName() + if( matcher && !matcher.matches(target) ) + return null + + if( saveAs && !(target=saveAs.call(target.toString()))) + return null + + return target + } + @CompileStatic protected Path resolveDestination(target) { diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 0e2079b2b5..ab7404124c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -17,7 +17,6 @@ package nextflow.trace import java.nio.file.Path -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.locks.Lock import java.util.concurrent.locks.ReentrantLock @@ -41,11 +40,13 @@ class TaskCleanupObserver implements TraceObserver { private DAG dag - private Map processes = new HashMap<>() + private Map processes = [:] - private Map tasks = new HashMap<>() + private Map tasks = [:] - private Map taskLookup = new HashMap<>() + private Map taskLookup = [:] + + private Set publishedOutputs = [] private Lock sync = new ReentrantLock() @@ -95,7 +96,7 @@ class TaskCleanupObserver implements TraceObserver { } } - log.trace "Process `${processName}` is consumed by the following processes: ${consumers.collect({ "`${it}`" }).join(', ')}" + log.trace "Process `${processName}` is consumed by the following processes: ${consumers}" processes[processName] = new ProcessState(consumers ?: [processName] as Set) } @@ -145,11 +146,27 @@ class TaskCleanupObserver implements TraceObserver { .values() .flatten() as List + // get publish outputs + final publishDirs = task.config.getPublishDir() + final publishOutputs = publishDirs + ? outputs.findAll( p -> publishDirs.any( publishDir -> publishDir.canPublish(p) ) ) + : [] + + log.trace "Task ${task.name} will publish the following files: ${publishOutputs*.toUriString()}" + sync.lock() try { // mark task as completed tasks[task].completed = true + // remove any outputs have already been published + final alreadyPublished = publishedOutputs.intersect(publishOutputs) + publishedOutputs.removeAll(alreadyPublished) + publishOutputs.removeAll(alreadyPublished) + + // add publish outputs to wait on + tasks[task].publishOutputs = publishOutputs as Set + // scan tasks for cleanup cleanup0() @@ -162,6 +179,45 @@ class TaskCleanupObserver implements TraceObserver { } } + /** + * When a file is published, mark it as published and check + * the corresponding task for cleanup. + * + * If the file is published before the corresponding task is + * marked as completed, save it for later. + * + * @param destination + * @param source + */ + @Override + void onFilePublish(Path destination, Path source) { + sync.lock() + try { + // get the corresponding task + final task = taskLookup[source] + + if( task ) { + log.trace "File ${source.toUriString()} was published by task ${task?.name}" + + // remove file from task barriers + tasks[task].publishOutputs.remove(source) + + // delete task if it can be deleted + if( canDelete(task) ) + deleteTask(task) + } + else { + log.trace "File ${source.toUriString()} was published, but task isn't marked as completed yet" + + // save file to be processed later + publishedOutputs << source + } + } + finally { + sync.unlock() + } + } + /** * When a process is closed (all tasks of the process have been created), * mark the process as closed and scan tasks for cleanup. @@ -181,30 +237,48 @@ class TaskCleanupObserver implements TraceObserver { } /** - * Delete any task directories that have no more barriers. + * Delete any task directories that can be deleted. */ private void cleanup0() { - for( TaskRun task : tasks.keySet() ) { - final taskState = tasks[task] - if( taskState.completed && !taskState.deleted && canDelete(task) ) { - log.trace "Deleting task directory: ${task.workDir.toUriString()}" - FileHelper.deletePath(task.workDir) - taskState.deleted = true - } - } + for( TaskRun task : tasks.keySet() ) + if( canDelete(task) ) + deleteTask(task) } /** * Determine whether a task directory can be deleted. * - * A task directory can be deleted if all of its process consumers - * are closed and all of its task consumers are completed. + * A task directory can be deleted if: + * - the task has completed + * - the task directory hasn't already been deleted + * - all of its publish outputs have been published + * - all of its process consumers are closed + * - all of its task consumers are completed + * + * @param task */ private boolean canDelete(TaskRun task) { final taskState = tasks[task] final processConsumers = processes[task.processor.name].consumers final taskConsumers = taskState.consumers - processConsumers.every( p -> processes[p].closed ) && taskConsumers.every( t -> tasks[t].completed ) + + taskState.completed + && !taskState.deleted + && taskState.publishOutputs.isEmpty() + && processConsumers.every( p -> processes[p].closed ) + && taskConsumers.every( t -> tasks[t].completed ) + } + + /** + * Delete a task directory. + * + * @param task + */ + private void deleteTask(TaskRun task) { + log.trace "Deleting task directory: ${task.workDir.toUriString()}" + + FileHelper.deletePath(task.workDir) + tasks[task].deleted = true } static private class ProcessState { @@ -217,7 +291,8 @@ class TaskCleanupObserver implements TraceObserver { } static private class TaskState { - Set consumers = [] as Set + Set consumers = [] + Set publishOutputs = [] boolean completed = false boolean deleted = false } From 9b343b6d11096a1c6d70eb4e5af82a4096cbacf9 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 7 Jul 2023 02:48:30 -0500 Subject: [PATCH 17/34] Log warning if eager cleanup is used with incompatible publish modes Signed-off-by: Ben Sherman --- .../groovy/nextflow/processor/PublishDir.groovy | 3 ++- .../nextflow/processor/TaskProcessor.groovy | 9 +++++++++ .../nextflow/trace/TaskCleanupObserver.groovy | 17 +++++++++++++++++ 3 files changed, 28 insertions(+), 1 deletion(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy index 0693ae867e..f57717a7b4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy @@ -199,6 +199,8 @@ class PublishDir { if( params.storageClass ) result.storageClass = params.storageClass as String + result.validatePublishMode() + return result } @@ -207,7 +209,6 @@ class PublishDir { assert path createPublishDir() - validatePublishMode() /* * when the publishing is using links, create them in process diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 0a2318419c..ca759a64f4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -367,6 +367,15 @@ class TaskProcessor { boolean hasErrors() { errorCount>0 } + /** + * Get a preview task config. + */ + TaskConfig getPreviewConfig() { + def config = config.createTaskConfig() + config.context = new TaskContext(this) + return config + } + protected void checkWarn(String msg, Map opts=null) { if( NF.isStrictMode() ) throw new ProcessUnrecoverableException(msg) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index ab7404124c..219666cf21 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -25,6 +25,7 @@ import groovy.util.logging.Slf4j import nextflow.Session import nextflow.dag.DAG import nextflow.file.FileHelper +import nextflow.processor.PublishDir.Mode import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun @@ -102,6 +103,22 @@ class TaskCleanupObserver implements TraceObserver { } } + static private final Set INVALID_PUBLISH_MODES = [Mode.COPY_NO_FOLLOW, Mode.RELLINK, Mode.SYMLINK] + + /** + * Log warning for any process that uses an incompatible + * publish mode. + * + * @param process + */ + void onProcessCreate( TaskProcessor process ) { + final taskConfig = process.getPreviewConfig() + final publishDirs = taskConfig.getPublishDir() + + if( publishDirs.any( p -> p.mode in INVALID_PUBLISH_MODES ) ) + log.warn "Process `${process.name}` is publishing files as symlinks, which may be invalidated by eager cleanup -- consider using 'copy' or 'link' instead" + } + /** * When a task is created, add it to the state map and add it as a consumer * of any task whose output it takes as input. From 6b5a82084a8d7484d1a3a8e5f7103917a1104712 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Fri, 7 Jul 2023 03:53:31 -0500 Subject: [PATCH 18/34] Add eager cleanup for individual output files Signed-off-by: Ben Sherman --- .../nextflow/trace/TaskCleanupObserver.groovy | 113 ++++++++++++++---- 1 file changed, 87 insertions(+), 26 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 219666cf21..800ab40937 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -45,7 +45,7 @@ class TaskCleanupObserver implements TraceObserver { private Map tasks = [:] - private Map taskLookup = [:] + private Map paths = [:] private Set publishedOutputs = [] @@ -90,10 +90,10 @@ class TaskCleanupObserver implements TraceObserver { // add process nodes to the list of consumers if( node.process != null ) - consumers.add(node.process.name) + consumers << node.process.name // add operator nodes to the queue to keep searching else - queue.add(node) + queue << node } } @@ -121,7 +121,7 @@ class TaskCleanupObserver implements TraceObserver { /** * When a task is created, add it to the state map and add it as a consumer - * of any task whose output it takes as input. + * of any upstream tasks and output files. * * @param handler * @param trace @@ -137,10 +137,15 @@ class TaskCleanupObserver implements TraceObserver { // add task to the task state map tasks[task] = new TaskState() - // add task as consumer to each task whoes output it takes as input - for( Path path : inputs ) - if( path in taskLookup ) - tasks[taskLookup[path]].consumers.add(task) + // add task as consumer of each upstream task and output file + for( Path path : inputs ) { + if( path in paths ) { + final pathState = paths[path] + final taskState = tasks[pathState.task] + taskState.consumers << task + pathState.consumers << task + } + } } finally { sync.unlock() @@ -148,7 +153,7 @@ class TaskCleanupObserver implements TraceObserver { } /** - * When a task is completed, track any temporary output files + * When a task is completed, track the task and its output files * for automatic cleanup. * * @param handler @@ -187,9 +192,14 @@ class TaskCleanupObserver implements TraceObserver { // scan tasks for cleanup cleanup0() - // add new output files to task lookup - for( Path path : outputs ) - taskLookup[path] = task + // add each output file to the path state map + for( Path path : outputs ) { + final pathState = new PathState(task) + if( path !in publishOutputs ) + pathState.published = true + + paths[path] = pathState + } } finally { sync.unlock() @@ -211,22 +221,26 @@ class TaskCleanupObserver implements TraceObserver { sync.lock() try { // get the corresponding task - final task = taskLookup[source] + final pathState = paths[source] + if( pathState ) { + final task = pathState.task - if( task ) { - log.trace "File ${source.toUriString()} was published by task ${task?.name}" + log.trace "File ${source.toUriString()} was published by task ${task.name}" - // remove file from task barriers + // mark file as published tasks[task].publishOutputs.remove(source) + pathState.published = true // delete task if it can be deleted - if( canDelete(task) ) + if( canDeleteTask(task) ) deleteTask(task) + else if( canDeleteFile(source) ) + deleteFile(source) } else { - log.trace "File ${source.toUriString()} was published, but task isn't marked as completed yet" + log.trace "File ${source.toUriString()} was published before task was marked as completed" - // save file to be processed later + // save file to be processed when task completes publishedOutputs << source } } @@ -254,12 +268,16 @@ class TaskCleanupObserver implements TraceObserver { } /** - * Delete any task directories that can be deleted. + * Delete any task directories and output files that can be deleted. */ private void cleanup0() { for( TaskRun task : tasks.keySet() ) - if( canDelete(task) ) + if( canDeleteTask(task) ) deleteTask(task) + + for( Path path : paths.keySet() ) + if( canDeleteFile(path) ) + deleteFile(path) } /** @@ -274,16 +292,15 @@ class TaskCleanupObserver implements TraceObserver { * * @param task */ - private boolean canDelete(TaskRun task) { + private boolean canDeleteTask(TaskRun task) { final taskState = tasks[task] - final processConsumers = processes[task.processor.name].consumers - final taskConsumers = taskState.consumers + final processState = processes[task.processor.name] taskState.completed && !taskState.deleted && taskState.publishOutputs.isEmpty() - && processConsumers.every( p -> processes[p].closed ) - && taskConsumers.every( t -> tasks[t].completed ) + && processState.consumers.every( p -> processes[p].closed ) + && taskState.consumers.every( t -> tasks[t].completed ) } /** @@ -298,6 +315,39 @@ class TaskCleanupObserver implements TraceObserver { tasks[task].deleted = true } + /** + * Determine whether a file can be deleted. + * + * A file can be deleted if: + * - the file has been published (or doesn't need to be published) + * - the file hasn't already been deleted + * - all of its process consumers are closed + * - all of its task consumers are completed + * + * @param path + */ + private boolean canDeleteFile(Path path) { + final pathState = paths[path] + final processState = processes[pathState.task.processor.name] + + pathState.published + && !pathState.deleted + && processState.consumers.every( p -> processes[p].closed ) + && pathState.consumers.every( t -> tasks[t].completed ) + } + + /** + * Delete a file. + * + * @param path + */ + private void deleteFile(Path path) { + log.trace "Deleting file: ${path.toUriString()}" + + FileHelper.deletePath(path) + paths[path].deleted = true + } + static private class ProcessState { Set consumers boolean closed = false @@ -314,4 +364,15 @@ class TaskCleanupObserver implements TraceObserver { boolean deleted = false } + static private class PathState { + TaskRun task + Set consumers = [] + boolean deleted = false + boolean published = false + + PathState(TaskRun task) { + this.task = task + } + } + } From c42f249ea04c6aabd25ec4110176310157801725 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Mon, 10 Jul 2023 10:33:58 -0500 Subject: [PATCH 19/34] Add warning about includeInputs with eager cleanup Signed-off-by: Ben Sherman --- .../groovy/nextflow/trace/TaskCleanupObserver.groovy | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 800ab40937..b7be5b178c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -106,12 +106,18 @@ class TaskCleanupObserver implements TraceObserver { static private final Set INVALID_PUBLISH_MODES = [Mode.COPY_NO_FOLLOW, Mode.RELLINK, Mode.SYMLINK] /** - * Log warning for any process that uses an incompatible - * publish mode. + * Log warning for any process that uses any incompatible features. * * @param process */ void onProcessCreate( TaskProcessor process ) { + // check for includeInputs + final outputs = process.config.getOutputs() + + if( outputs.any( p -> p instanceof FileOutParam && p.includeInputs ) ) + log.warn "Process `${process.name}` is forwarding input files with includeInputs, which may be invalidated by eager cleanup" + + // check for incompatible publish modes final taskConfig = process.getPreviewConfig() final publishDirs = taskConfig.getPublishDir() From 8b66c393309ae74094ca49977f5f965959a2c498 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 11 Jul 2023 15:29:21 -0500 Subject: [PATCH 20/34] Add resumability Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/cache/CacheDB.groovy | 45 +++- .../nextflow/processor/TaskEntry.groovy | 6 + .../nextflow/processor/TaskProcessor.groovy | 196 ++++++++++-------- .../nextflow/trace/TaskCleanupObserver.groovy | 48 ++++- 4 files changed, 202 insertions(+), 93 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy index 214e3235e3..2a2806f3cd 100644 --- a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy @@ -77,7 +77,7 @@ class CacheDB implements Closeable { * @param processor The {@link TaskProcessor} instance to be assigned to the retrieved task * @return A {link TaskEntry} instance or {@code null} if a task for the given hash does not exist */ - TaskEntry getTaskEntry(HashCode taskHash, TaskProcessor processor) { + TaskEntry getTaskEntry(HashCode taskHash, Map processorLookup=[:]) { final payload = store.getEntry(taskHash) if( !payload ) @@ -85,9 +85,11 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) TraceRecord trace = TraceRecord.deserialize( (byte[])record[0] ) + final processor = processorLookup[trace.get('process')] TaskContext ctx = record[1]!=null && processor!=null ? TaskContext.deserialize(processor, (byte[])record[1]) : null + final consumers = record[3]!=null ? ((List)record[3]).collect( str -> HashCode.fromString(str) ) : null - return new TaskEntry(trace,ctx) + return new TaskEntry(processor, trace, ctx, consumers) } void incTaskEntry( HashCode hash ) { @@ -99,7 +101,7 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) // third record contains the reference count for this record - record[2] = ((Integer)record[2]) +1 + record[2] = ((Integer)record[2]) + 1 // save it again store.putEntry(hash, KryoHelper.serialize(record)) @@ -114,7 +116,7 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) // third record contains the reference count for this record - def count = record[2] = ((Integer)record[2]) -1 + def count = record[2] = ((Integer)record[2]) - 1 // save or delete if( count > 0 ) { store.putEntry(hash, KryoHelper.serialize(record)) @@ -128,9 +130,10 @@ class CacheDB implements Closeable { /** - * Save task runtime information to th cache DB + * Save task runtime information to the cache DB * - * @param handler A {@link TaskHandler} instance + * @param handler + * @param trace */ @PackageScope void writeTaskEntry0( TaskHandler handler, TraceRecord trace ) { @@ -143,10 +146,11 @@ class CacheDB implements Closeable { // only the 'cache' is active and TaskContext ctx = proc.isCacheable() && task.hasCacheableValues() ? task.context : null - def record = new ArrayList(3) + final record = new ArrayList(4) record[0] = trace.serialize() record[1] = ctx != null ? ctx.serialize() : null record[2] = 1 + record[3] = null // -- save in the db store.putEntry( key, KryoHelper.serialize(record) ) @@ -157,6 +161,31 @@ class CacheDB implements Closeable { writer.send { writeTaskEntry0(handler, trace) } } + /** + * Finalize task entry in the cache DB with the list of + * consumer tasks. + * + * @param hash + * @param consumers + */ + @PackageScope + void finalizeTaskEntry0( HashCode hash, List consumers ) { + + final payload = store.getEntry(hash) + if( !payload ) { + log.debug "Unable to finalize task with key: $hash" + return + } + + final record = (List)KryoHelper.deserialize(payload) + record[3] = consumers.collect( t -> t.toString() ) + store.putEntry(hash, KryoHelper.serialize(record)) + } + + void finalizeTaskAsync( HashCode hash, List consumers ) { + writer.send { finalizeTaskEntry0(hash, consumers) } + } + void cacheTaskAsync( TaskHandler handler ) { writer.send { writeTaskIndex0(handler,true) @@ -224,7 +253,7 @@ class CacheDB implements Closeable { } TraceRecord getTraceRecord( HashCode hashCode ) { - final result = getTaskEntry(hashCode, null) + final result = getTaskEntry(hashCode) return result ? result.trace : null } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy index ed4ac32362..7738dfd016 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy @@ -16,9 +16,11 @@ package nextflow.processor +import com.google.common.hash.HashCode import groovy.transform.EqualsAndHashCode import groovy.transform.ToString import groovy.transform.TupleConstructor +import nextflow.processor.TaskProcessor import nextflow.trace.TraceRecord /** * Model a task entry persisted in the {@link nextflow.cache.CacheDB} @@ -30,8 +32,12 @@ import nextflow.trace.TraceRecord @TupleConstructor class TaskEntry { + TaskProcessor processor + TraceRecord trace TaskContext context + List consumers + } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index ca759a64f4..6af2e5ee6d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -23,6 +23,7 @@ import java.nio.file.LinkOption import java.nio.file.NoSuchFileException import java.nio.file.Path import java.nio.file.Paths +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicIntegerArray @@ -123,11 +124,11 @@ class TaskProcessor { RunType(String str) { message=str }; } - static final public String TASK_CONTEXT_PROPERTY_NAME = 'task' + static final String TASK_CONTEXT_PROPERTY_NAME = 'task' - final private static Pattern ENV_VAR_NAME = ~/[a-zA-Z_]+[a-zA-Z0-9_]*/ + static private final Pattern ENV_VAR_NAME = ~/[a-zA-Z_]+[a-zA-Z0-9_]*/ - final private static Pattern QUESTION_MARK = ~/(\?+)/ + static private final Pattern QUESTION_MARK = ~/(\?+)/ @Memoized static boolean getInvalidateCacheOnTaskDirectiveChange() { @@ -135,7 +136,7 @@ class TaskProcessor { return value==null || value =='true' } - @TestOnly private static volatile TaskProcessor currentProcessor0 + @TestOnly static private volatile TaskProcessor currentProcessor0 @TestOnly static TaskProcessor currentProcessor() { currentProcessor0 } @@ -147,7 +148,7 @@ class TaskProcessor { /** * Unique task index number (run) */ - final protected AtomicInteger indexCount = new AtomicInteger() + protected final AtomicInteger indexCount = new AtomicInteger() /** * The current workflow execution session @@ -200,7 +201,7 @@ class TaskProcessor { * Note: it is declared static because the error must be shown only the * very first time for all processes */ - private static final AtomicBoolean errorShown = new AtomicBoolean() + static private final AtomicBoolean errorShown = new AtomicBoolean() /** * Flag set {@code true} when the processor termination has been invoked @@ -243,9 +244,9 @@ class TaskProcessor { private int maxForks - private static int processCount + static private int processCount - private static LockManager lockManager = new LockManager() + static private LockManager lockManager = new LockManager() private List> fairBuffers = new ArrayList<>() @@ -253,6 +254,17 @@ class TaskProcessor { private Boolean isFair0 + /** + * Map of all task processors by name. + */ + static private Map processorLookup = [:] + + /** + * Set of tasks (across all processors) that were deleted in a + * previous run and successfully restored from the cache db. + */ + static private Map restoredTasks = new ConcurrentHashMap<>() + private CompilerConfiguration compilerConfig() { final config = new CompilerConfiguration() config.addCompilationCustomizers( new ASTTransformationCustomizer(TaskTemplateVarsXform) ) @@ -307,6 +319,8 @@ class TaskProcessor { this.maxForks = config.maxForks ? config.maxForks as int : 0 this.forksCount = maxForks ? new LongAdder() : null this.isFair0 = config.getFair() + + processorLookup[name] = this } /** @@ -703,6 +717,10 @@ class TaskProcessor { return null } + protected TaskStartParams createTaskStartParams() { + return new TaskStartParams(TaskId.next(), indexCount.incrementAndGet()) + } + /** * Create a new {@code TaskRun} instance, initializing the following properties : *
  • {@code TaskRun#id} @@ -774,23 +792,17 @@ class TaskProcessor { while( true ) { hash = CacheHelper.defaultHasher().newHasher().putBytes(hash.asBytes()).putInt(tries).hash() - Path resumeDir = null - boolean exists = false try { - def entry = session.cache.getTaskEntry(hash, this) - resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null - if( resumeDir ) - exists = resumeDir.exists() - - log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" - def cached = shouldTryCache && exists && checkCachedOutput(task.clone(), resumeDir, hash, entry) - if( cached ) + if( shouldTryCache && checkCachedOutput(task.clone(), hash) ) break } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) } + final entry = session.cache.getTaskEntry(hash, processorLookup) + Path resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null + boolean exists = resumeDir?.exists() if( exists ) { tries++ continue @@ -876,88 +888,110 @@ class TaskProcessor { } /** - * Check whenever the outputs for the specified task already exist + * Attempt to restore a cached task by verifying either its outputs + * or the outputs of its consumers. * - * @param task The task instance - * @param folder The folder where the outputs are stored (eventually) - * @return {@code true} when all outputs are available, {@code false} otherwise + * @param seedTask + * @param seedHash + * @return {@code true} if all outputs are available, {@code false} otherwise */ - final boolean checkCachedOutput(TaskRun task, Path folder, HashCode hash, TaskEntry entry) { + @CompileStatic + final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash) { - // check if exists the task exit code file - def exitCode = null - def exitFile = folder.resolve(TaskRun.CMD_EXIT) - if( task.type == ScriptType.SCRIPTLET ) { - def str - try { - str = exitFile.text?.trim() + // -- recursively check for cached outputs + final queue = [ seedHash ] + final handlersMap = [:] as Map + + while( !queue.isEmpty() ) { + final hash = queue.pop() + + // -- skip tasks that have already been restored + if( hash in handlersMap ) + continue + + // -- get cache entry + final entry = session.cache.getTaskEntry(hash, processorLookup) + if( !entry ) + return false + + // -- get or create task run + def task + if( hash == seedHash ) { + task = seedTask } - catch( IOException e ) { - log.trace "[${safeTaskName(task)}] Exit file can't be read > $exitFile -- return false -- Cause: ${e.message}" + else if( entry.processor ) { + final params = entry.processor.createTaskStartParams() + task = entry.processor.createTaskRun(params) + log.trace "[${safeTaskName(task)}] Restoring deleted task hash=${hash} context=${entry.context}" + } + else { return false } - exitCode = str.isInteger() ? str.toInteger() : null - if( !task.isSuccess(exitCode) ) { - log.trace "[${safeTaskName(task)}] Exit code is not valid > $str -- return false" + // -- verify the task context map + if( task.hasCacheableValues() && !entry.context ) { + log.trace "[${safeTaskName(task)}] Missing cache context -- return false" return false } - } - /* - * verify cached context map - */ - if( !entry ) { - log.trace "[${safeTaskName(task)}] Missing cache entry -- return false" - return false - } + if( entry.context != null ) { + task.context = entry.context + task.config.context = entry.context + task.code?.delegate = entry.context + } - if( task.hasCacheableValues() && !entry.context ) { - log.trace "[${safeTaskName(task)}] Missing cache context -- return false" - return false - } + // -- verify the task exit code + final exitCode = entry.trace.get('exit') as Integer + if( task.type == ScriptType.SCRIPTLET && !task.isSuccess(exitCode) ) { + log.trace "[${safeTaskName(task)}] Exit code is not valid > $exitCode -- return false" + return false + } - /* - * verify stdout file - */ - final stdoutFile = folder.resolve( TaskRun.CMD_OUTFILE ) + // -- set the remaining task properties + task.cached = true + task.hash = hash + task.workDir = FileHelper.asPath(entry.trace.getWorkDir()) + task.stdout = task.workDir.resolve(TaskRun.CMD_OUTFILE) + task.exitStatus = exitCode + task.config.exitStatus = exitCode - if( entry.context != null ) { - task.context = entry.context - task.config.context = entry.context - task.code?.delegate = entry.context + // -- check if all downstream outputs are available + if( entry.consumers != null ) { + queue.addAll( entry.consumers ) + } + + // -- otherwise check if all task outputs are available + else { + try { + collectOutputs(task) + } + catch( MissingFileException | MissingValueException e ) { + log.trace "[${safeTaskName(task)}] Missed cache > ${e.getMessage()} -- folder: ${task.workDir}" + return false + } + } + + // -- create task handler + handlersMap[hash] = new CachedTaskHandler(task, entry.trace) } - try { - // -- expose task exit status to make accessible as output value - task.config.exitStatus = exitCode - // -- check if all output resources are available - collectOutputs(task, folder, stdoutFile, task.context) + // -- finalize all cached tasks + handlersMap.each { hash, handler -> + if( hash in restoredTasks ) + return - // set the exit code in to the task object - task.cached = true - task.hash = hash - task.workDir = folder - task.stdout = stdoutFile - if( exitCode != null ) { - task.exitStatus = exitCode - } + final task = handler.task log.info "[${task.hashLog}] Cached process > ${task.name}" + // -- update the set of restored tasks + restoredTasks[hash] = true // -- notify cached event - if( entry ) - session.notifyTaskCached(new CachedTaskHandler(task,entry.trace)) - - // -- now bind the results - finalizeTask0(task) - return true - } - catch( MissingFileException | MissingValueException e ) { - log.trace "[${safeTaskName(task)}] Missed cache > ${e.getMessage()} -- folder: $folder" - task.exitStatus = Integer.MAX_VALUE - task.workDir = null - return false + session.notifyTaskCached(handler) + // -- bind the results + task.processor.finalizeTask0(task) } + + return true } /** @@ -2406,7 +2440,7 @@ class TaskProcessor { state.update { StateObj it -> it.incSubmitted() } // task index must be created here to guarantee consistent ordering // with the sequence of messages arrival since this method is executed in a thread safe manner - final params = new TaskStartParams(TaskId.next(), indexCount.incrementAndGet()) + final params = createTaskStartParams() final result = new ArrayList(2) result[0] = params result[1] = messages diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index b7be5b178c..67eae531d1 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -23,6 +23,7 @@ import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.Session +import nextflow.cache.CacheDB import nextflow.dag.DAG import nextflow.file.FileHelper import nextflow.processor.PublishDir.Mode @@ -41,6 +42,8 @@ class TaskCleanupObserver implements TraceObserver { private DAG dag + private CacheDB cache + private Map processes = [:] private Map tasks = [:] @@ -54,6 +57,7 @@ class TaskCleanupObserver implements TraceObserver { @Override void onFlowCreate(Session session) { this.dag = session.dag + this.cache = session.cache } /** @@ -167,8 +171,15 @@ class TaskCleanupObserver implements TraceObserver { */ @Override void onProcessComplete(TaskHandler handler, TraceRecord trace) { - // query task output files final task = handler.task + + // handle failed tasks separately + if( !task.isSuccess() ) { + handleTaskFailure(task) + return + } + + // query task output files final outputs = task .getOutputsByType(FileOutParam) .values() @@ -212,6 +223,24 @@ class TaskCleanupObserver implements TraceObserver { } } + /** + * When a task fails, mark it as completed without tracking its + * output files. Failed tasks are not included as consumers of + * upstream tasks in the cache. + * + * @param task + */ + void handleTaskFailure(TaskRun task) { + sync.lock() + try { + // mark task as completed + tasks[task].completed = true + } + finally { + sync.unlock() + } + } + /** * When a file is published, mark it as published and check * the corresponding task for cleanup. @@ -317,8 +346,16 @@ class TaskCleanupObserver implements TraceObserver { private void deleteTask(TaskRun task) { log.trace "Deleting task directory: ${task.workDir.toUriString()}" + // delete task + final taskState = tasks[task] FileHelper.deletePath(task.workDir) - tasks[task].deleted = true + taskState.deleted = true + + // finalize task in the cache db + final consumers = taskState.consumers + .findAll( t -> t.isSuccess() ) + .collect( t -> t.hash ) + cache.finalizeTaskAsync(task.hash, consumers) } /** @@ -350,8 +387,11 @@ class TaskCleanupObserver implements TraceObserver { private void deleteFile(Path path) { log.trace "Deleting file: ${path.toUriString()}" - FileHelper.deletePath(path) - paths[path].deleted = true + final pathState = paths[path] + final taskState = tasks[pathState.task] + if( !taskState.deleted ) + FileHelper.deletePath(path) + pathState.deleted = true } static private class ProcessState { From 43c939d79de9f5219012384df46caa5d27b98232 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 11 Jul 2023 15:49:55 -0500 Subject: [PATCH 21/34] Upgrade kryo to 5.4.0 (#3562) Signed-off-by: Ben Sherman --- modules/nextflow/build.gradle | 2 +- .../nextflow/util/SerializationHelper.groovy | 26 +++++++++---------- .../UnmodifiableCollectionsSerializer.java | 2 +- .../nextflow/file/http/XPathSerializer.groovy | 2 +- .../cloud/azure/file/AzPathSerializer.groovy | 2 +- .../cloud/google/util/GsPathSerializer.groovy | 2 +- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/modules/nextflow/build.gradle b/modules/nextflow/build.gradle index 66f39fad28..83d179ea94 100644 --- a/modules/nextflow/build.gradle +++ b/modules/nextflow/build.gradle @@ -34,7 +34,7 @@ dependencies { api "commons-codec:commons-codec:1.15" api "commons-io:commons-io:2.11.0" api "com.beust:jcommander:1.35" - api("com.esotericsoftware.kryo:kryo:2.24.0") { exclude group: 'com.esotericsoftware.minlog', module: 'minlog' } + api("com.esotericsoftware:kryo:5.4.0") api('org.iq80.leveldb:leveldb:0.12') api('org.eclipse.jgit:org.eclipse.jgit:6.5.0.202303070854-r') api ('javax.activation:activation:1.1.1') diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index d6668fde2d..0129044656 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -270,7 +270,7 @@ class PathSerializer extends Serializer { } @Override - Path read(Kryo kryo, Input input, Class type) { + Path read(Kryo kryo, Input input, Class type) { final scheme = input.readString() final path = input.readString() log.trace "Path de-serialization > scheme: $scheme; path: $path" @@ -302,16 +302,16 @@ class GStringSerializer extends Serializer { static final private Class STR_ARRAY_CLASS = (Class)(new String[0]).getClass() @Override - void write(Kryo kryo, Output stream, GString object) { + void write(Kryo kryo, Output output, GString object) { log.trace "GString serialization: values: ${object?.getValues()} - strings: ${object?.getStrings()}" - kryo.writeObject( stream, object.getValues() ) - kryo.writeObject( stream, object.getStrings() ) + kryo.writeObject( output, object.getValues() ) + kryo.writeObject( output, object.getStrings() ) } @Override - GString read(Kryo kryo, Input stream, Class type) { - Object[] values = kryo.readObject(stream, OBJ_ARRAY_CLASS) - String[] strings = kryo.readObject(stream, STR_ARRAY_CLASS) + GString read(Kryo kryo, Input input, Class type) { + Object[] values = kryo.readObject(input, OBJ_ARRAY_CLASS) + String[] strings = kryo.readObject(input, STR_ARRAY_CLASS) log.trace "GString de-serialize: values: ${values} - strings: ${strings}" new GStringImpl(values, strings) } @@ -329,7 +329,7 @@ class URLSerializer extends Serializer { } @Override - URL read(Kryo kryo, Input input, Class type) { + URL read(Kryo kryo, Input input, Class type) { log.trace "URL de-serialization" return new URL(input.readString()) } @@ -347,7 +347,7 @@ class UUIDSerializer extends Serializer { } @Override - UUID read(Kryo kryo, Input input, Class type) { + UUID read(Kryo kryo, Input input, Class type) { log.trace "UUID de-serialization" long mostBits = input.readLong() long leastBits = input.readLong() @@ -367,7 +367,7 @@ class FileSerializer extends Serializer { } @Override - File read(Kryo kryo, Input input, Class type) { + File read(Kryo kryo, Input input, Class type) { log.trace "File de-serialization" return new File(input.readString()) } @@ -396,7 +396,7 @@ class PatternSerializer extends Serializer { } @Override - Pattern read(Kryo kryo, Input input, Class type) { + Pattern read(Kryo kryo, Input input, Class type) { def len = input.readInt() def buffer = new byte[len] @@ -421,7 +421,7 @@ class ArrayTupleSerializer extends Serializer { } @Override - ArrayTuple read(Kryo kryo, Input input, Class type) { + ArrayTuple read(Kryo kryo, Input input, Class type) { final len = input.readInt() def list = new ArrayList(len) for( int i=0; i { } @Override - Map.Entry read(Kryo kryo, Input input, Class type) { + Map.Entry read(Kryo kryo, Input input, Class type) { def key = kryo.readClassAndObject(input) def val = kryo.readClassAndObject(input) new MapEntry(key,val) diff --git a/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java b/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java index 5971ecd4a9..5b7256f2a7 100644 --- a/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java +++ b/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java @@ -67,7 +67,7 @@ public class UnmodifiableCollectionsSerializer extends Serializer { } @Override - public Object read(final Kryo kryo, final Input input, final Class clazz) { + public Object read(final Kryo kryo, final Input input, final Class clazz) { final int ordinal = input.readInt( true ); final UnmodifiableCollection unmodifiableCollection = UnmodifiableCollection.values()[ordinal]; final Object sourceCollection = kryo.readClassAndObject( input ); diff --git a/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy b/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy index 761a11c11d..330656fd00 100644 --- a/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy +++ b/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy @@ -41,7 +41,7 @@ class XPathSerializer extends Serializer { } @Override - XPath read(Kryo kryo, Input input, Class type) { + XPath read(Kryo kryo, Input input, Class type) { final uri = input.readString() log.trace "Path de-serialization > uri=$uri" (XPath) FileHelper.asPath(new URI(uri)) diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy index 05fcff02b2..fab1ce4bb5 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy @@ -41,7 +41,7 @@ class AzPathSerializer extends Serializer implements SerializerRegistran } @Override - AzPath read(Kryo kryo, Input input, Class type) { + AzPath read(Kryo kryo, Input input, Class type) { final path = input.readString() log.trace "Azure Blob storage path > path=$path" return (AzPath)FileHelper.asPath(path) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy index e042061495..3af5f2b74b 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy @@ -47,7 +47,7 @@ class GsPathSerializer extends Serializer implements Serialize } @Override - CloudStoragePath read(Kryo kryo, Input input, Class type) { + CloudStoragePath read(Kryo kryo, Input input, Class type) { final path = input.readString() log.trace "Google CloudStoragePath de-serialization > path=$path" def uri = CloudStorageFileSystem.URI_SCHEME + '://' + path From 5a78cf97a9e48c864e5d9e6cc3091c21958ca83b Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 11 Jul 2023 15:50:28 -0500 Subject: [PATCH 22/34] Add serializer for HashCode Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/cache/CacheDB.groovy | 4 ++-- .../nextflow/util/SerializationHelper.groovy | 22 +++++++++++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy index 2a2806f3cd..a9349e52fe 100644 --- a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy @@ -87,7 +87,7 @@ class CacheDB implements Closeable { TraceRecord trace = TraceRecord.deserialize( (byte[])record[0] ) final processor = processorLookup[trace.get('process')] TaskContext ctx = record[1]!=null && processor!=null ? TaskContext.deserialize(processor, (byte[])record[1]) : null - final consumers = record[3]!=null ? ((List)record[3]).collect( str -> HashCode.fromString(str) ) : null + final consumers = record[3]!=null ? ((List)record[3]) : null return new TaskEntry(processor, trace, ctx, consumers) } @@ -178,7 +178,7 @@ class CacheDB implements Closeable { } final record = (List)KryoHelper.deserialize(payload) - record[3] = consumers.collect( t -> t.toString() ) + record[3] = consumers store.putEntry(hash, KryoHelper.serialize(record)) } diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index 0129044656..5fa9670f3f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -27,6 +27,7 @@ import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.Serializer import com.esotericsoftware.kryo.io.Input import com.esotericsoftware.kryo.io.Output +import com.google.common.hash.HashCode import de.javakaffee.kryoserializers.UnmodifiableCollectionsSerializer import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -215,6 +216,7 @@ class DefaultSerializers implements SerializerRegistrant { serializers.put( Pattern, PatternSerializer ) serializers.put( ArrayTuple, ArrayTupleSerializer ) serializers.put( SerializableMarker, null ) + serializers.put( HashCode, HashCodeSerializer ) } } @@ -448,3 +450,23 @@ class MapEntrySerializer extends Serializer { new MapEntry(key,val) } } + +@CompileStatic +class HashCodeSerializer extends Serializer { + + @Override + void write(Kryo kryo, Output output, HashCode entry) { + final bytes = entry.asBytes() + output.writeInt(bytes.length) + output.write(bytes) + } + + @Override + HashCode read(Kryo kryo, Input input, Class type) { + final len = input.readInt() + final bytes = new byte[len] + input.read(bytes) + + HashCode.fromBytes(bytes) + } +} From 7b02b8e7cef9ab3a410f1be9d50cb09d8e1b6deb Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Tue, 11 Jul 2023 17:31:01 -0500 Subject: [PATCH 23/34] Fix failing tests Signed-off-by: Ben Sherman --- modules/nextflow/build.gradle | 2 +- .../nextflow/processor/PublishDir.groovy | 23 ++++++++++++------- .../nextflow/trace/TaskCleanupObserver.groovy | 2 +- .../nextflow/util/SerializationHelper.groovy | 2 ++ .../groovy/nextflow/cache/CacheDBTest.groovy | 2 +- 5 files changed, 20 insertions(+), 11 deletions(-) diff --git a/modules/nextflow/build.gradle b/modules/nextflow/build.gradle index 83d179ea94..117b63f6d2 100644 --- a/modules/nextflow/build.gradle +++ b/modules/nextflow/build.gradle @@ -34,7 +34,7 @@ dependencies { api "commons-codec:commons-codec:1.15" api "commons-io:commons-io:2.11.0" api "com.beust:jcommander:1.35" - api("com.esotericsoftware:kryo:5.4.0") + api("com.esotericsoftware:kryo:5.5.0") api('org.iq80.leveldb:leveldb:0.12') api('org.eclipse.jgit:org.eclipse.jgit:6.5.0.202303070854-r') api ('javax.activation:activation:1.1.1') diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy index f57717a7b4..70c88b5864 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy @@ -168,7 +168,7 @@ class PublishDir { def result = new PublishDir() if( params.path ) - result.path = params.path + result.path = params.path if( params.mode ) result.mode = params.mode @@ -199,16 +199,28 @@ class PublishDir { if( params.storageClass ) result.storageClass = params.storageClass as String - result.validatePublishMode() - return result } + @CompileStatic + boolean canPublish(Path source, TaskRun task) { + if( !sourceDir ) { + this.sourceDir = task.targetDir + this.sourceFileSystem = sourceDir.fileSystem + this.stageInMode = task.config.stageInMode + this.taskName = task.name + validatePublishMode() + } + + return getPublishTarget(source) != null + } + @CompileStatic protected void apply0(Set files) { assert path createPublishDir() + validatePublishMode() /* * when the publishing is using links, create them in process @@ -324,11 +336,6 @@ class PublishDir { } - @CompileStatic - boolean canPublish(Path source) { - return getPublishTarget(source) != null - } - @CompileStatic protected def getPublishTarget(Path source) { def target = sourceDir ? sourceDir.relativize(source) : source.getFileName() diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 67eae531d1..3662c662d5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -188,7 +188,7 @@ class TaskCleanupObserver implements TraceObserver { // get publish outputs final publishDirs = task.config.getPublishDir() final publishOutputs = publishDirs - ? outputs.findAll( p -> publishDirs.any( publishDir -> publishDir.canPublish(p) ) ) + ? outputs.findAll( p -> publishDirs.any( publishDir -> publishDir.canPublish(p, task) ) ) : [] log.trace "Task ${task.name} will publish the following files: ${publishOutputs*.toUriString()}" diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index 5fa9670f3f..f69de6ee51 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -87,6 +87,8 @@ class KryoHelper { static private Kryo newInstance() { def kryo = new Kryo() kryo.setInstantiatorStrategy( InstantiationStrategy.instance ) + kryo.setReferences(true) + kryo.setRegistrationRequired(false) // special serializers UnmodifiableCollectionsSerializer.registerSerializers(kryo) diff --git a/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy b/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy index 8191c29695..0276a9ae79 100644 --- a/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy @@ -83,7 +83,7 @@ class CacheDBTest extends Specification { 1 * task.getContext() >> ctx when: - def entry = cache.getTaskEntry(hash, proc) + def entry = cache.getTaskEntry(hash, ['foo': proc]) then: entry instanceof TaskEntry entry.trace instanceof TraceRecord From c727c056a07f666e0c4fdd21bd370c69b766e057 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 10:29:18 -0500 Subject: [PATCH 24/34] Revert Kryo upgrade Signed-off-by: Ben Sherman --- modules/nextflow/build.gradle | 2 +- .../nextflow/util/SerializationHelper.groovy | 30 +++++++++---------- .../UnmodifiableCollectionsSerializer.java | 2 +- .../nextflow/file/http/XPathSerializer.groovy | 2 +- .../cloud/azure/file/AzPathSerializer.groovy | 2 +- .../cloud/google/util/GsPathSerializer.groovy | 2 +- 6 files changed, 19 insertions(+), 21 deletions(-) diff --git a/modules/nextflow/build.gradle b/modules/nextflow/build.gradle index 117b63f6d2..66f39fad28 100644 --- a/modules/nextflow/build.gradle +++ b/modules/nextflow/build.gradle @@ -34,7 +34,7 @@ dependencies { api "commons-codec:commons-codec:1.15" api "commons-io:commons-io:2.11.0" api "com.beust:jcommander:1.35" - api("com.esotericsoftware:kryo:5.5.0") + api("com.esotericsoftware.kryo:kryo:2.24.0") { exclude group: 'com.esotericsoftware.minlog', module: 'minlog' } api('org.iq80.leveldb:leveldb:0.12') api('org.eclipse.jgit:org.eclipse.jgit:6.5.0.202303070854-r') api ('javax.activation:activation:1.1.1') diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index f69de6ee51..512fc344b1 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -87,8 +87,6 @@ class KryoHelper { static private Kryo newInstance() { def kryo = new Kryo() kryo.setInstantiatorStrategy( InstantiationStrategy.instance ) - kryo.setReferences(true) - kryo.setRegistrationRequired(false) // special serializers UnmodifiableCollectionsSerializer.registerSerializers(kryo) @@ -274,7 +272,7 @@ class PathSerializer extends Serializer { } @Override - Path read(Kryo kryo, Input input, Class type) { + Path read(Kryo kryo, Input input, Class type) { final scheme = input.readString() final path = input.readString() log.trace "Path de-serialization > scheme: $scheme; path: $path" @@ -306,16 +304,16 @@ class GStringSerializer extends Serializer { static final private Class STR_ARRAY_CLASS = (Class)(new String[0]).getClass() @Override - void write(Kryo kryo, Output output, GString object) { + void write(Kryo kryo, Output stream, GString object) { log.trace "GString serialization: values: ${object?.getValues()} - strings: ${object?.getStrings()}" - kryo.writeObject( output, object.getValues() ) - kryo.writeObject( output, object.getStrings() ) + kryo.writeObject( stream, object.getValues() ) + kryo.writeObject( stream, object.getStrings() ) } @Override - GString read(Kryo kryo, Input input, Class type) { - Object[] values = kryo.readObject(input, OBJ_ARRAY_CLASS) - String[] strings = kryo.readObject(input, STR_ARRAY_CLASS) + GString read(Kryo kryo, Input stream, Class type) { + Object[] values = kryo.readObject(stream, OBJ_ARRAY_CLASS) + String[] strings = kryo.readObject(stream, STR_ARRAY_CLASS) log.trace "GString de-serialize: values: ${values} - strings: ${strings}" new GStringImpl(values, strings) } @@ -333,7 +331,7 @@ class URLSerializer extends Serializer { } @Override - URL read(Kryo kryo, Input input, Class type) { + URL read(Kryo kryo, Input input, Class type) { log.trace "URL de-serialization" return new URL(input.readString()) } @@ -351,7 +349,7 @@ class UUIDSerializer extends Serializer { } @Override - UUID read(Kryo kryo, Input input, Class type) { + UUID read(Kryo kryo, Input input, Class type) { log.trace "UUID de-serialization" long mostBits = input.readLong() long leastBits = input.readLong() @@ -371,7 +369,7 @@ class FileSerializer extends Serializer { } @Override - File read(Kryo kryo, Input input, Class type) { + File read(Kryo kryo, Input input, Class type) { log.trace "File de-serialization" return new File(input.readString()) } @@ -400,7 +398,7 @@ class PatternSerializer extends Serializer { } @Override - Pattern read(Kryo kryo, Input input, Class type) { + Pattern read(Kryo kryo, Input input, Class type) { def len = input.readInt() def buffer = new byte[len] @@ -425,7 +423,7 @@ class ArrayTupleSerializer extends Serializer { } @Override - ArrayTuple read(Kryo kryo, Input input, Class type) { + ArrayTuple read(Kryo kryo, Input input, Class type) { final len = input.readInt() def list = new ArrayList(len) for( int i=0; i { } @Override - Map.Entry read(Kryo kryo, Input input, Class type) { + Map.Entry read(Kryo kryo, Input input, Class type) { def key = kryo.readClassAndObject(input) def val = kryo.readClassAndObject(input) new MapEntry(key,val) @@ -464,7 +462,7 @@ class HashCodeSerializer extends Serializer { } @Override - HashCode read(Kryo kryo, Input input, Class type) { + HashCode read(Kryo kryo, Input input, Class type) { final len = input.readInt() final bytes = new byte[len] input.read(bytes) diff --git a/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java b/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java index 5b7256f2a7..5971ecd4a9 100644 --- a/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java +++ b/modules/nextflow/src/main/java/de/javakaffee/kryoserializers/UnmodifiableCollectionsSerializer.java @@ -67,7 +67,7 @@ public class UnmodifiableCollectionsSerializer extends Serializer { } @Override - public Object read(final Kryo kryo, final Input input, final Class clazz) { + public Object read(final Kryo kryo, final Input input, final Class clazz) { final int ordinal = input.readInt( true ); final UnmodifiableCollection unmodifiableCollection = UnmodifiableCollection.values()[ordinal]; final Object sourceCollection = kryo.readClassAndObject( input ); diff --git a/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy b/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy index 330656fd00..761a11c11d 100644 --- a/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy +++ b/modules/nf-httpfs/src/main/nextflow/file/http/XPathSerializer.groovy @@ -41,7 +41,7 @@ class XPathSerializer extends Serializer { } @Override - XPath read(Kryo kryo, Input input, Class type) { + XPath read(Kryo kryo, Input input, Class type) { final uri = input.readString() log.trace "Path de-serialization > uri=$uri" (XPath) FileHelper.asPath(new URI(uri)) diff --git a/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy b/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy index fab1ce4bb5..05fcff02b2 100644 --- a/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy +++ b/plugins/nf-azure/src/main/nextflow/cloud/azure/file/AzPathSerializer.groovy @@ -41,7 +41,7 @@ class AzPathSerializer extends Serializer implements SerializerRegistran } @Override - AzPath read(Kryo kryo, Input input, Class type) { + AzPath read(Kryo kryo, Input input, Class type) { final path = input.readString() log.trace "Azure Blob storage path > path=$path" return (AzPath)FileHelper.asPath(path) diff --git a/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy b/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy index 3af5f2b74b..e042061495 100644 --- a/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy +++ b/plugins/nf-google/src/main/nextflow/cloud/google/util/GsPathSerializer.groovy @@ -47,7 +47,7 @@ class GsPathSerializer extends Serializer implements Serialize } @Override - CloudStoragePath read(Kryo kryo, Input input, Class type) { + CloudStoragePath read(Kryo kryo, Input input, Class type) { final path = input.readString() log.trace "Google CloudStoragePath de-serialization > path=$path" def uri = CloudStorageFileSystem.URI_SCHEME + '://' + path From d37ee1f514daf3a5cfab75555d07c94ae427ea2f Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 11:27:03 -0500 Subject: [PATCH 25/34] Add kryo instantiator for HashCode Signed-off-by: Ben Sherman --- .../groovy/nextflow/util/SerializationHelper.groovy | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index 512fc344b1..9c7dc8b5fc 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -236,6 +236,16 @@ class MatcherInstantiator implements ObjectInstantiator { } } +@Singleton +@CompileStatic +class HashCodeInstantiator implements ObjectInstantiator { + + @Override + Object newInstance() { + HashCode.fromInt(0) + } +} + @Singleton @CompileStatic class InstantiationStrategy extends Kryo.DefaultInstantiatorStrategy { @@ -245,6 +255,9 @@ class InstantiationStrategy extends Kryo.DefaultInstantiatorStrategy { if( type == Matcher ) { MatcherInstantiator.instance } + else if( type == HashCode.BytesHashCode ) { + HashCodeInstantiator.instance + } else { super.newInstantiatorOf(type) } From 859e96bfd3a57eb1e1b3f3404482eaeea6dc5054 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 11:40:16 -0500 Subject: [PATCH 26/34] Remove HashCode serializer Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/cache/CacheDB.groovy | 4 +-- .../nextflow/util/SerializationHelper.groovy | 35 ------------------- 2 files changed, 2 insertions(+), 37 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy index a9349e52fe..07302166d5 100644 --- a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy @@ -87,7 +87,7 @@ class CacheDB implements Closeable { TraceRecord trace = TraceRecord.deserialize( (byte[])record[0] ) final processor = processorLookup[trace.get('process')] TaskContext ctx = record[1]!=null && processor!=null ? TaskContext.deserialize(processor, (byte[])record[1]) : null - final consumers = record[3]!=null ? ((List)record[3]) : null + final consumers = record[3]!=null ? ((List)record[3]).collect( s -> HashCode.fromString(s) ) : null return new TaskEntry(processor, trace, ctx, consumers) } @@ -178,7 +178,7 @@ class CacheDB implements Closeable { } final record = (List)KryoHelper.deserialize(payload) - record[3] = consumers + record[3] = consumers.collect( h -> h.toString() ) store.putEntry(hash, KryoHelper.serialize(record)) } diff --git a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy index 9c7dc8b5fc..d6668fde2d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/util/SerializationHelper.groovy @@ -27,7 +27,6 @@ import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.Serializer import com.esotericsoftware.kryo.io.Input import com.esotericsoftware.kryo.io.Output -import com.google.common.hash.HashCode import de.javakaffee.kryoserializers.UnmodifiableCollectionsSerializer import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -216,7 +215,6 @@ class DefaultSerializers implements SerializerRegistrant { serializers.put( Pattern, PatternSerializer ) serializers.put( ArrayTuple, ArrayTupleSerializer ) serializers.put( SerializableMarker, null ) - serializers.put( HashCode, HashCodeSerializer ) } } @@ -236,16 +234,6 @@ class MatcherInstantiator implements ObjectInstantiator { } } -@Singleton -@CompileStatic -class HashCodeInstantiator implements ObjectInstantiator { - - @Override - Object newInstance() { - HashCode.fromInt(0) - } -} - @Singleton @CompileStatic class InstantiationStrategy extends Kryo.DefaultInstantiatorStrategy { @@ -255,9 +243,6 @@ class InstantiationStrategy extends Kryo.DefaultInstantiatorStrategy { if( type == Matcher ) { MatcherInstantiator.instance } - else if( type == HashCode.BytesHashCode ) { - HashCodeInstantiator.instance - } else { super.newInstantiatorOf(type) } @@ -463,23 +448,3 @@ class MapEntrySerializer extends Serializer { new MapEntry(key,val) } } - -@CompileStatic -class HashCodeSerializer extends Serializer { - - @Override - void write(Kryo kryo, Output output, HashCode entry) { - final bytes = entry.asBytes() - output.writeInt(bytes.length) - output.write(bytes) - } - - @Override - HashCode read(Kryo kryo, Input input, Class type) { - final len = input.readInt() - final bytes = new byte[len] - input.read(bytes) - - HashCode.fromBytes(bytes) - } -} From c89811cc4cd882ceeabcf81206884f5f68c8f87e Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 11:52:45 -0500 Subject: [PATCH 27/34] Add log messages, fix infinite hang Signed-off-by: Ben Sherman --- .../nextflow/processor/TaskProcessor.groovy | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 6af2e5ee6d..e7c535ec5f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -792,17 +792,22 @@ class TaskProcessor { while( true ) { hash = CacheHelper.defaultHasher().newHasher().putBytes(hash.asBytes()).putInt(tries).hash() + Path resumeDir = null + boolean exists = false try { - if( shouldTryCache && checkCachedOutput(task.clone(), hash) ) + final entry = session.cache.getTaskEntry(hash, processorLookup) + resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null + if( resumeDir ) + exists = resumeDir.exists() + + log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" + if( shouldTryCache && checkCachedOutput(task.clone(), hash, entry) ) break } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) } - final entry = session.cache.getTaskEntry(hash, processorLookup) - Path resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null - boolean exists = resumeDir?.exists() if( exists ) { tries++ continue @@ -893,10 +898,10 @@ class TaskProcessor { * * @param seedTask * @param seedHash + * @param seedEntry * @return {@code true} if all outputs are available, {@code false} otherwise */ - @CompileStatic - final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash) { + final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash, TaskEntry seedEntry) { // -- recursively check for cached outputs final queue = [ seedHash ] @@ -910,9 +915,13 @@ class TaskProcessor { continue // -- get cache entry - final entry = session.cache.getTaskEntry(hash, processorLookup) - if( !entry ) + final entry = (hash == seedHash) + ? seedEntry + : session.cache.getTaskEntry(hash, processorLookup) + if( !entry ) { + log.trace "[${safeTaskName(seedTask)}] Missing cache entry for downstream hash=${hash} -- return false" return false + } // -- get or create task run def task @@ -925,6 +934,7 @@ class TaskProcessor { log.trace "[${safeTaskName(task)}] Restoring deleted task hash=${hash} context=${entry.context}" } else { + log.trace "[${safeTaskName(seedTask)}] Missing processor for downstream hash=${hash} entry=${entry} -- return false" return false } @@ -984,6 +994,8 @@ class TaskProcessor { log.info "[${task.hashLog}] Cached process > ${task.name}" // -- update the set of restored tasks + if( task.processor != this ) + task.processor.state.update { StateObj it -> it.incSubmitted() } restoredTasks[hash] = true // -- notify cached event session.notifyTaskCached(handler) From a034f6058552d74b688e9e7265149ba7713eb59d Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 12:59:42 -0500 Subject: [PATCH 28/34] Add support for includeInputs Signed-off-by: Ben Sherman --- .../nextflow/trace/TaskCleanupObserver.groovy | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 3662c662d5..ff06dc9484 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -67,6 +67,9 @@ class TaskCleanupObserver implements TraceObserver { @Override void onFlowBegin() { + // construct process lookup + final withIncludeInputs = [] as Set + for( def processNode : dag.vertices ) { // skip nodes that are not processes if( !processNode.process ) @@ -101,9 +104,27 @@ class TaskCleanupObserver implements TraceObserver { } } - log.trace "Process `${processName}` is consumed by the following processes: ${consumers}" - processes[processName] = new ProcessState(consumers ?: [processName] as Set) + + // check if process uses includeInputs + final hasIncludeInputs = processNode.process + .config.getOutputs() + .any( p -> p instanceof FileOutParam && p.includeInputs ) + + if( hasIncludeInputs ) + withIncludeInputs << processName + } + + // update producers of processes that use includeInputs + processes.each { processName, processState -> + final consumers = processState.consumers + for( def consumer : consumers.intersect(withIncludeInputs) ) { + log.trace "Process `${consumer}` uses includeInputs, adding its consumers to `${processName}`" + final consumerState = processes[consumer] + consumers.addAll(consumerState.consumers) + } + + log.trace "Process `${processName}` is consumed by the following processes: ${consumers}" } } @@ -115,12 +136,6 @@ class TaskCleanupObserver implements TraceObserver { * @param process */ void onProcessCreate( TaskProcessor process ) { - // check for includeInputs - final outputs = process.config.getOutputs() - - if( outputs.any( p -> p instanceof FileOutParam && p.includeInputs ) ) - log.warn "Process `${process.name}` is forwarding input files with includeInputs, which may be invalidated by eager cleanup" - // check for incompatible publish modes final taskConfig = process.getPreviewConfig() final publishDirs = taskConfig.getPublishDir() From 9741b1cd92fa6bab3633f7e3dfc4d2a1755707d1 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 13:10:09 -0500 Subject: [PATCH 29/34] Update log messages Signed-off-by: Ben Sherman --- .../nextflow/trace/TaskCleanupObserver.groovy | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index ff06dc9484..9077d4cb84 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -206,7 +206,7 @@ class TaskCleanupObserver implements TraceObserver { ? outputs.findAll( p -> publishDirs.any( publishDir -> publishDir.canPublish(p, task) ) ) : [] - log.trace "Task ${task.name} will publish the following files: ${publishOutputs*.toUriString()}" + log.trace "[${task.name}] will publish the following files: ${publishOutputs*.toUriString()}" sync.lock() try { @@ -275,7 +275,7 @@ class TaskCleanupObserver implements TraceObserver { if( pathState ) { final task = pathState.task - log.trace "File ${source.toUriString()} was published by task ${task.name}" + log.trace "File ${source.toUriString()} was published by task <${task.name}>" // mark file as published tasks[task].publishOutputs.remove(source) @@ -359,7 +359,7 @@ class TaskCleanupObserver implements TraceObserver { * @param task */ private void deleteTask(TaskRun task) { - log.trace "Deleting task directory: ${task.workDir.toUriString()}" + log.trace "[${task.name}] Deleting task directory: ${task.workDir.toUriString()}" // delete task final taskState = tasks[task] @@ -400,12 +400,14 @@ class TaskCleanupObserver implements TraceObserver { * @param path */ private void deleteFile(Path path) { - log.trace "Deleting file: ${path.toUriString()}" - final pathState = paths[path] - final taskState = tasks[pathState.task] - if( !taskState.deleted ) + final task = pathState.task + final taskState = tasks[task] + + if( !taskState.deleted ) { + log.trace "[${task.name}] Deleting file: ${path.toUriString()}" FileHelper.deletePath(path) + } pathState.deleted = true } From 3aae1afd3c4794d30ebf8f16fc1a158f75b73275 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 12 Jul 2023 14:27:18 -0500 Subject: [PATCH 30/34] Add lazy, eager, aggressive cleanup strategies Signed-off-by: Ben Sherman --- docs/config.md | 34 +++++++------ .../src/main/groovy/nextflow/Session.groovy | 34 ++----------- .../nextflow/script/ScriptRunner.groovy | 1 - .../nextflow/trace/CleanupStrategy.groovy | 49 +++++++++++++++++++ .../trace/DefaultObserverFactory.groovy | 7 ++- .../nextflow/trace/TaskCleanupObserver.groovy | 29 ++++++++++- .../nextflow/trace/CleanupStrategyTest.groovy | 48 ++++++++++++++++++ 7 files changed, 152 insertions(+), 50 deletions(-) create mode 100644 modules/nextflow/src/main/groovy/nextflow/trace/CleanupStrategy.groovy create mode 100644 modules/nextflow/src/test/groovy/nextflow/trace/CleanupStrategyTest.groovy diff --git a/docs/config.md b/docs/config.md index 086399854e..2f85bb3468 100644 --- a/docs/config.md +++ b/docs/config.md @@ -1425,26 +1425,32 @@ Detailed information about the JSON fields can be found in the {ref}`weblog desc There are additional variables that can be defined within a configuration file that do not have a dedicated scope. `cleanup` -: If `true`, on a successful completion of a run all files in *work* directory are automatically deleted. - - :::{warning} - The use of the `cleanup` option will prevent the use of the *resume* feature on subsequent executions of that pipeline run. Also, be aware that deleting all scratch files can take a lot of time, especially when using a shared file system or remote cloud storage. - ::: - - :::{versionadded} 23.10.0 +: :::{versionchanged} 23.10.0 + Added `'lazy'`, `'eager'`, and `'aggressive'` strategies. ::: +: Automatically delete task directories using one of the following strategies: - Setting `cleanup = 'eager'` enables the "eager" cleanup strategy. This strategy will delete each task directory as soon as it is no longer needed by downstream tasks, rather than at the end of the workflow run. This feature is useful for minimizing disk usage during the workflow run. - - The lifetime of a task is determined by the downstream tasks that use the task's output files. When all of these tasks finish, the upstream task can be deleted. + `false` (default) + : Disable automatic cleanup. - The following caveats apply when using eager cleanup: + `true` + : Equivalent to `'lazy'`. - - Eager cleanup will break the resumability of your pipeline. If a workflow run fails, you will have to restart from the beginning, whereas with `cleanup = true` the cleanup would not have happened. As a result, eager cleanup is designed to be used only when you are confident that the workflow run will not fail. + `'lazy'` + : If a workflow completes successfully, delete all task directories. + : This strategy supports resumability for both successful and failed runs. + : Note that deleting all work directories at once can take a lot of time, especially when using a shared file system or remote cloud storage. - - Output files should not be published via symlink when using eager cleanup, because the symlinks will be invalidated when the original task directory is deleted. Avoid using the following publish modes: `copyNoFollow`, `rellink`, `symlink`. + `'eager'` + : Delete each task directory as soon as it is no longer needed by downstream tasks. + : A task can be deleted once all of the tasks that use any of its output files have completed. + : This strategy supports resumability for both successful and failed runs. + : Output files that are published via symlink will be invalidated when the original task directory is deleted. Avoid using the following publish modes: `copyNoFollow`, `rellink`, `symlink`. - - Eager cleanup currently does not work properly with processes that forward input files with the `includeInputs` option. In this case, the forwarded input files will be deleted prematurely, and any process that consumes the forwarded output channel may fail or produce incorrect output. + `'aggressive'` + : Equivalent to `'eager'`, but also deletes individual output files as soon as they are no longer needed. + : An output file can be deleted once the tasks that use it have completed. In some cases, an output file can be deleted sooner than its originating task. + : This strategy supports resumability for successful runs, but not necessarily for failed runs. Therefore, it is recommended when you want to minimize disk storage during the pipeline and you don't need resumability. `dumpHashes` : If `true`, dump task hash keys in the log file, for debugging purposes. diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index 6d0ce3007e..662d175921 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -354,6 +354,9 @@ class Session implements ISession { // -- file porter config this.filePorter = new FilePorter(this) + // -- normalize cleanup config + if( config.cleanup == true ) + config.cleanup = 'lazy' } /** @@ -1130,37 +1133,6 @@ class Session implements ISession { errorAction = action } - /** - * Delete the workflow work directory from tasks temporary files - */ - void cleanup() { - if( !workDir || config.cleanup != true ) - return - - if( aborted || cancelled || error ) - return - - CacheDB db = null - try { - log.trace "Cleaning-up workdir" - db = CacheFactory.create(uniqueId, runName).openForRead() - db.eachRecord { HashCode hash, TraceRecord record -> - def deleted = db.removeTaskEntry(hash) - if( deleted ) { - // delete folder - FileHelper.deletePath(FileHelper.asPath(record.workDir)) - } - } - log.trace "Clean workdir complete" - } - catch( Exception e ) { - log.warn("Failed to cleanup work dir: ${workDir.toUriString()}") - } - finally { - db.close() - } - } - @Memoized CondaConfig getCondaConfig() { final cfg = config.conda as Map ?: Collections.emptyMap() diff --git a/modules/nextflow/src/main/groovy/nextflow/script/ScriptRunner.groovy b/modules/nextflow/src/main/groovy/nextflow/script/ScriptRunner.groovy index 3cab2bddb0..3f4b0a1638 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/ScriptRunner.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/ScriptRunner.groovy @@ -237,7 +237,6 @@ class ScriptRunner { protected shutdown() { session.destroy() - session.cleanup() Global.cleanUp() log.debug "> Execution complete -- Goodbye" } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/CleanupStrategy.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/CleanupStrategy.groovy new file mode 100644 index 0000000000..2ed75b0a7b --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/trace/CleanupStrategy.groovy @@ -0,0 +1,49 @@ +/* + * 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.trace + +import groovy.transform.CompileStatic + +/** + * Strategies to automatically cleanup task directories. + * + * @author Ben Sherman + */ +@CompileStatic +enum CleanupStrategy { + LAZY(1), + EAGER(2), + AGGRESSIVE(3) + + final int level + + CleanupStrategy(int level) { + this.level = level + } + + static boolean isValid(CharSequence name) { + if( !name ) + return false + try { + valueOf(name.toString().toUpperCase()) + return true + } + catch( IllegalArgumentException e ) { + return false + } + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy index 017b461e9d..9bc6e4577e 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/DefaultObserverFactory.groovy @@ -118,8 +118,11 @@ class DefaultObserverFactory implements TraceObserverFactory { } protected void createTaskCleanupObserver(Collection result) { - if( session.config.cleanup == 'eager' ) - result << new TaskCleanupObserver() + final strategy = session.config.cleanup + if( strategy instanceof CharSequence && !CleanupStrategy.isValid(strategy) ) + throw new IllegalArgumentException("Invalid cleanup strategy '${strategy}' -- available strategies are ${CleanupStrategy.values().join(',').toLowerCase()}") + if( strategy ) + result << new TaskCleanupObserver(strategy.toString().toUpperCase() as CleanupStrategy) } } diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index 9077d4cb84..98514a52b9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -40,6 +40,8 @@ import nextflow.script.params.FileOutParam @CompileStatic class TaskCleanupObserver implements TraceObserver { + private CleanupStrategy strategy + private DAG dag private CacheDB cache @@ -54,6 +56,10 @@ class TaskCleanupObserver implements TraceObserver { private Lock sync = new ReentrantLock() + TaskCleanupObserver(CleanupStrategy strategy) { + this.strategy = strategy + } + @Override void onFlowCreate(Session session) { this.dag = session.dag @@ -282,9 +288,9 @@ class TaskCleanupObserver implements TraceObserver { pathState.published = true // delete task if it can be deleted - if( canDeleteTask(task) ) + if( strategy >= CleanupStrategy.EAGER && canDeleteTask(task) ) deleteTask(task) - else if( canDeleteFile(source) ) + else if( strategy >= CleanupStrategy.AGGRESSIVE && canDeleteFile(source) ) deleteFile(source) } else { @@ -317,14 +323,33 @@ class TaskCleanupObserver implements TraceObserver { } } + /** + * When the workflow completes, delete all task directories (only + * when using the 'lazy' strategy). + */ + @Override + void onFlowComplete() { + if( strategy != CleanupStrategy.LAZY ) + return + + for( TaskRun task : tasks.keySet() ) + deleteTask(task) + } + /** * Delete any task directories and output files that can be deleted. */ private void cleanup0() { + if( strategy < CleanupStrategy.EAGER ) + return + for( TaskRun task : tasks.keySet() ) if( canDeleteTask(task) ) deleteTask(task) + if( strategy < CleanupStrategy.AGGRESSIVE ) + return + for( Path path : paths.keySet() ) if( canDeleteFile(path) ) deleteFile(path) diff --git a/modules/nextflow/src/test/groovy/nextflow/trace/CleanupStrategyTest.groovy b/modules/nextflow/src/test/groovy/nextflow/trace/CleanupStrategyTest.groovy new file mode 100644 index 0000000000..f25a0439b3 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/trace/CleanupStrategyTest.groovy @@ -0,0 +1,48 @@ +/* + * 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.trace + +import spock.lang.Specification + +/** + * + * @author Ben Sherman + */ +class CleanupStrategyTest extends Specification { + + def "should check if it's a valid cleanup strategy" () { + + expect: + CleanupStrategy.isValid(NAME) == EXPECTED + + where: + NAME | EXPECTED + null | false + '' | false + 'foo' | false + and: + 'lazy' | true + 'eager' | true + 'aggressive' | true + and: + 'LAZY' | true + 'EAGER' | true + 'AGGRESSIVE' | true + } + +} From 3498c5d650d9a9c1712416f8a4c51ca39a2e8613 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 14 Sep 2023 16:51:46 -0500 Subject: [PATCH 31/34] Add thread pool for task cleanup Signed-off-by: Ben Sherman --- .../nextflow/trace/TaskCleanupObserver.groovy | 70 +++++++++---------- 1 file changed, 32 insertions(+), 38 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index aa4c523f5e..c4e4abe459 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -17,6 +17,7 @@ package nextflow.trace import java.nio.file.Path +import java.util.concurrent.ExecutorService import java.util.concurrent.locks.Lock import java.util.concurrent.locks.ReentrantLock @@ -31,6 +32,7 @@ import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor import nextflow.processor.TaskRun import nextflow.script.params.FileOutParam +import nextflow.util.ThreadPoolManager /** * Delete task directories once they are no longer needed. * @@ -42,9 +44,7 @@ class TaskCleanupObserver implements TraceObserver { private CleanupStrategy strategy - private DAG dag - - private CacheDB cache + private Session session private Map processes = [:] @@ -56,14 +56,18 @@ class TaskCleanupObserver implements TraceObserver { private Lock sync = new ReentrantLock() + private ExecutorService threadPool + TaskCleanupObserver(CleanupStrategy strategy) { this.strategy = strategy } @Override void onFlowCreate(Session session) { - this.dag = session.dag - this.cache = session.cache + this.session = session + this.threadPool = new ThreadPoolManager('TaskCleanup') + .withConfig(session.config) + .create() } /** @@ -74,6 +78,7 @@ class TaskCleanupObserver implements TraceObserver { void onFlowBegin() { // construct process lookup + final dag = session.dag final withIncludeInputs = [] as Set for( def processNode : dag.vertices ) { @@ -163,8 +168,7 @@ class TaskCleanupObserver implements TraceObserver { final task = handler.task final inputs = task.getInputFilesMap().values() - sync.lock() - try { + sync.withLock { // add task to the task state map tasks[task] = new TaskState() @@ -178,9 +182,6 @@ class TaskCleanupObserver implements TraceObserver { } } } - finally { - sync.unlock() - } } /** @@ -214,8 +215,7 @@ class TaskCleanupObserver implements TraceObserver { log.trace "[${task.name}] will publish the following files: ${publishOutputs*.toUriString()}" - sync.lock() - try { + sync.withLock { // mark task as completed tasks[task].completed = true @@ -239,9 +239,6 @@ class TaskCleanupObserver implements TraceObserver { paths[path] = pathState } } - finally { - sync.unlock() - } } /** @@ -252,14 +249,10 @@ class TaskCleanupObserver implements TraceObserver { * @param task */ void handleTaskFailure(TaskRun task) { - sync.lock() - try { + sync.withLock { // mark task as completed tasks[task].completed = true } - finally { - sync.unlock() - } } /** @@ -274,8 +267,7 @@ class TaskCleanupObserver implements TraceObserver { */ @Override void onFilePublish(Path destination, Path source) { - sync.lock() - try { + sync.withLock { // get the corresponding task final pathState = paths[source] if( pathState ) { @@ -300,9 +292,6 @@ class TaskCleanupObserver implements TraceObserver { publishedOutputs << source } } - finally { - sync.unlock() - } } /** @@ -313,14 +302,10 @@ class TaskCleanupObserver implements TraceObserver { */ @Override void onProcessClose(TaskProcessor process) { - sync.lock() - try { + sync.withLock { processes[process.name].closed = true cleanup0() } - finally { - sync.unlock() - } } /** @@ -329,11 +314,14 @@ class TaskCleanupObserver implements TraceObserver { */ @Override void onFlowComplete() { - if( strategy != CleanupStrategy.LAZY ) - return + if( strategy == CleanupStrategy.LAZY && session.isSuccess() ) { + log.info 'Deleting task directories (this might take a moment)...' - for( TaskRun task : tasks.keySet() ) - deleteTask(task) + for( TaskRun task : tasks.keySet() ) + deleteTask(task) + } + + threadPool.shutdown() } /** @@ -387,15 +375,22 @@ class TaskCleanupObserver implements TraceObserver { log.trace "[${task.name}] Deleting task directory: ${task.workDir.toUriString()}" // delete task + threadPool.submit({ + try { + FileHelper.deletePath(task.workDir) + } + catch( Exception e ) {} + } as Runnable) + + // mark task as deleted final taskState = tasks[task] - FileHelper.deletePath(task.workDir) taskState.deleted = true // finalize task in the cache db final consumers = taskState.consumers .findAll( t -> t.isSuccess() ) .collect( t -> t.hash ) - cache.finalizeTaskAsync(task.hash, consumers) + session.cache.finalizeTaskAsync(task.hash, consumers) } /** @@ -427,9 +422,8 @@ class TaskCleanupObserver implements TraceObserver { private void deleteFile(Path path) { final pathState = paths[path] final task = pathState.task - final taskState = tasks[task] - if( !taskState.deleted ) { + if( !tasks[task].deleted ) { log.trace "[${task.name}] Deleting file: ${path.toUriString()}" FileHelper.deletePath(path) } From c4216172b42be89cf97b6fd4cc89fc6096a68c88 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Thu, 14 Sep 2023 16:52:35 -0500 Subject: [PATCH 32/34] Minor improvement to checkCachedOutput() Signed-off-by: Ben Sherman --- .../nextflow/processor/TaskProcessor.groovy | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 41e9cfc669..22a4ee5c5c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -808,22 +808,17 @@ class TaskProcessor { while( true ) { hash = CacheHelper.defaultHasher().newHasher().putBytes(hash.asBytes()).putInt(tries).hash() - Path resumeDir = null - boolean exists = false try { - final entry = session.cache.getTaskEntry(hash, processorLookup) - resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null - if( resumeDir ) - exists = resumeDir.exists() - - log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" - if( shouldTryCache && checkCachedOutput(task.clone(), hash, entry) ) + if( shouldTryCache && checkCachedOutput(task.clone(), hash) ) break } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) } + final entry = session.cache.getTaskEntry(hash, processorLookup) + Path resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null + boolean exists = resumeDir?.exists() if( exists ) { tries++ continue @@ -914,10 +909,9 @@ class TaskProcessor { * * @param seedTask * @param seedHash - * @param seedEntry * @return {@code true} if all outputs are available, {@code false} otherwise */ - final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash, TaskEntry seedEntry) { + final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash) { // -- recursively check for cached outputs final queue = [ seedHash ] @@ -931,18 +925,15 @@ class TaskProcessor { continue // -- get cache entry - final entry = (hash == seedHash) - ? seedEntry - : session.cache.getTaskEntry(hash, processorLookup) - if( !entry ) { - log.trace "[${safeTaskName(seedTask)}] Missing cache entry for downstream hash=${hash} -- return false" + final entry = session.cache.getTaskEntry(hash, processorLookup) + if( !entry ) return false - } // -- get or create task run def task if( hash == seedHash ) { task = seedTask + log.trace "[${safeTaskName(task)}] Cacheable task hash=${hash} entry=${entry}" } else if( entry.processor ) { final params = entry.processor.createTaskStartParams() @@ -969,7 +960,7 @@ class TaskProcessor { // -- verify the task exit code final exitCode = entry.trace.get('exit') as Integer if( task.type == ScriptType.SCRIPTLET && !task.isSuccess(exitCode) ) { - log.trace "[${safeTaskName(task)}] Exit code is not valid > $exitCode -- return false" + log.trace "[${safeTaskName(task)}] Exit code is not valid > ${exitCode} -- return false" return false } From 9ce10dc4e2e144c198e345907581faf74b222ce2 Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 Jan 2024 20:36:38 -0600 Subject: [PATCH 33/34] minor edits Signed-off-by: Ben Sherman --- .../main/groovy/nextflow/processor/PublishDir.groovy | 2 +- .../groovy/nextflow/processor/TaskProcessor.groovy | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy index 4b812a8483..f037a1adaa 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/PublishDir.groovy @@ -222,7 +222,7 @@ class PublishDir { this.sourceDir = task.targetDir this.sourceFileSystem = sourceDir.fileSystem this.stageInMode = task.config.stageInMode - this.taskName = task.name + this.task = task validatePublishMode() } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index 432d5bdcc1..dc9cd46564 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -728,7 +728,7 @@ class TaskProcessor { return null } - protected TaskStartParams createTaskStartParams() { + synchronized protected TaskStartParams createTaskStartParams() { return new TaskStartParams(TaskId.next(), indexCount.incrementAndGet()) } @@ -909,14 +909,14 @@ class TaskProcessor { final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash) { // -- recursively check for cached outputs - final queue = [ seedHash ] - final handlersMap = [:] as Map + List queue = [ seedHash ] + Map handlers = [:] while( !queue.isEmpty() ) { final hash = queue.pop() // -- skip tasks that have already been restored - if( hash in handlersMap ) + if( hash in handlers ) continue // -- get cache entry @@ -984,11 +984,11 @@ class TaskProcessor { } // -- create task handler - handlersMap[hash] = new CachedTaskHandler(task, entry.trace) + handlers[hash] = new CachedTaskHandler(task, entry.trace) } // -- finalize all cached tasks - handlersMap.each { hash, handler -> + handlers.each { hash, handler -> if( hash in restoredTasks ) return From b73c6bb070ddde87a466740711a19a52790b0ebe Mon Sep 17 00:00:00 2001 From: Ben Sherman Date: Wed, 31 Jan 2024 20:54:36 -0600 Subject: [PATCH 34/34] Remove WIP resumability Signed-off-by: Ben Sherman --- docs/config.md | 9 +- .../main/groovy/nextflow/cache/CacheDB.groovy | 45 +--- .../nextflow/processor/TaskEntry.groovy | 6 - .../nextflow/processor/TaskProcessor.groovy | 197 +++++++----------- .../nextflow/trace/TaskCleanupObserver.groovy | 10 +- .../groovy/nextflow/cache/CacheDBTest.groovy | 2 +- 6 files changed, 95 insertions(+), 174 deletions(-) diff --git a/docs/config.md b/docs/config.md index d74e79aac5..960f183ed4 100644 --- a/docs/config.md +++ b/docs/config.md @@ -1579,7 +1579,7 @@ Read the {ref}`trace-report` page to learn more about the execution report that There are additional variables that can be defined within a configuration file that do not have a dedicated scope. `cleanup` -: :::{versionchanged} 23.10.0 +: :::{versionchanged} 24.04.0 Added `'lazy'`, `'eager'`, and `'aggressive'` strategies. ::: : Automatically delete task directories using one of the following strategies: @@ -1592,19 +1592,20 @@ There are additional variables that can be defined within a configuration file t `'lazy'` : If a workflow completes successfully, delete all task directories. - : This strategy supports resumability for both successful and failed runs. : Note that deleting all work directories at once can take a lot of time, especially when using a shared file system or remote cloud storage. `'eager'` : Delete each task directory as soon as it is no longer needed by downstream tasks. : A task can be deleted once all of the tasks that use any of its output files have completed. - : This strategy supports resumability for both successful and failed runs. : Output files that are published via symlink will be invalidated when the original task directory is deleted. Avoid using the following publish modes: `copyNoFollow`, `rellink`, `symlink`. `'aggressive'` : Equivalent to `'eager'`, but also deletes individual output files as soon as they are no longer needed. : An output file can be deleted once the tasks that use it have completed. In some cases, an output file can be deleted sooner than its originating task. - : This strategy supports resumability for successful runs, but not necessarily for failed runs. Therefore, it is recommended when you want to minimize disk storage during the pipeline and you don't need resumability. + +: :::{note} + Resumability is not currently supported, but will be supported in a future release for the `'lazy'` and `'eager'` cleanup strategies. + ::: `dumpHashes` : If `true`, dump task hash keys in the log file, for debugging purposes. diff --git a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy index 07302166d5..214e3235e3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/cache/CacheDB.groovy @@ -77,7 +77,7 @@ class CacheDB implements Closeable { * @param processor The {@link TaskProcessor} instance to be assigned to the retrieved task * @return A {link TaskEntry} instance or {@code null} if a task for the given hash does not exist */ - TaskEntry getTaskEntry(HashCode taskHash, Map processorLookup=[:]) { + TaskEntry getTaskEntry(HashCode taskHash, TaskProcessor processor) { final payload = store.getEntry(taskHash) if( !payload ) @@ -85,11 +85,9 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) TraceRecord trace = TraceRecord.deserialize( (byte[])record[0] ) - final processor = processorLookup[trace.get('process')] TaskContext ctx = record[1]!=null && processor!=null ? TaskContext.deserialize(processor, (byte[])record[1]) : null - final consumers = record[3]!=null ? ((List)record[3]).collect( s -> HashCode.fromString(s) ) : null - return new TaskEntry(processor, trace, ctx, consumers) + return new TaskEntry(trace,ctx) } void incTaskEntry( HashCode hash ) { @@ -101,7 +99,7 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) // third record contains the reference count for this record - record[2] = ((Integer)record[2]) + 1 + record[2] = ((Integer)record[2]) +1 // save it again store.putEntry(hash, KryoHelper.serialize(record)) @@ -116,7 +114,7 @@ class CacheDB implements Closeable { final record = (List)KryoHelper.deserialize(payload) // third record contains the reference count for this record - def count = record[2] = ((Integer)record[2]) - 1 + def count = record[2] = ((Integer)record[2]) -1 // save or delete if( count > 0 ) { store.putEntry(hash, KryoHelper.serialize(record)) @@ -130,10 +128,9 @@ class CacheDB implements Closeable { /** - * Save task runtime information to the cache DB + * Save task runtime information to th cache DB * - * @param handler - * @param trace + * @param handler A {@link TaskHandler} instance */ @PackageScope void writeTaskEntry0( TaskHandler handler, TraceRecord trace ) { @@ -146,11 +143,10 @@ class CacheDB implements Closeable { // only the 'cache' is active and TaskContext ctx = proc.isCacheable() && task.hasCacheableValues() ? task.context : null - final record = new ArrayList(4) + def record = new ArrayList(3) record[0] = trace.serialize() record[1] = ctx != null ? ctx.serialize() : null record[2] = 1 - record[3] = null // -- save in the db store.putEntry( key, KryoHelper.serialize(record) ) @@ -161,31 +157,6 @@ class CacheDB implements Closeable { writer.send { writeTaskEntry0(handler, trace) } } - /** - * Finalize task entry in the cache DB with the list of - * consumer tasks. - * - * @param hash - * @param consumers - */ - @PackageScope - void finalizeTaskEntry0( HashCode hash, List consumers ) { - - final payload = store.getEntry(hash) - if( !payload ) { - log.debug "Unable to finalize task with key: $hash" - return - } - - final record = (List)KryoHelper.deserialize(payload) - record[3] = consumers.collect( h -> h.toString() ) - store.putEntry(hash, KryoHelper.serialize(record)) - } - - void finalizeTaskAsync( HashCode hash, List consumers ) { - writer.send { finalizeTaskEntry0(hash, consumers) } - } - void cacheTaskAsync( TaskHandler handler ) { writer.send { writeTaskIndex0(handler,true) @@ -253,7 +224,7 @@ class CacheDB implements Closeable { } TraceRecord getTraceRecord( HashCode hashCode ) { - final result = getTaskEntry(hashCode) + final result = getTaskEntry(hashCode, null) return result ? result.trace : null } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy index 7738dfd016..ed4ac32362 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskEntry.groovy @@ -16,11 +16,9 @@ package nextflow.processor -import com.google.common.hash.HashCode import groovy.transform.EqualsAndHashCode import groovy.transform.ToString import groovy.transform.TupleConstructor -import nextflow.processor.TaskProcessor import nextflow.trace.TraceRecord /** * Model a task entry persisted in the {@link nextflow.cache.CacheDB} @@ -32,12 +30,8 @@ import nextflow.trace.TraceRecord @TupleConstructor class TaskEntry { - TaskProcessor processor - TraceRecord trace TaskContext context - List consumers - } diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index dc9cd46564..cc260ddc0d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -23,7 +23,6 @@ import java.nio.file.LinkOption import java.nio.file.NoSuchFileException import java.nio.file.Path import java.nio.file.Paths -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicIntegerArray @@ -127,11 +126,11 @@ class TaskProcessor { RunType(String str) { message=str }; } - static final String TASK_CONTEXT_PROPERTY_NAME = 'task' + static final public String TASK_CONTEXT_PROPERTY_NAME = 'task' - static private final Pattern ENV_VAR_NAME = ~/[a-zA-Z_]+[a-zA-Z0-9_]*/ + final private static Pattern ENV_VAR_NAME = ~/[a-zA-Z_]+[a-zA-Z0-9_]*/ - static private final Pattern QUESTION_MARK = ~/(\?+)/ + final private static Pattern QUESTION_MARK = ~/(\?+)/ @TestOnly private static volatile TaskProcessor currentProcessor0 @@ -145,7 +144,7 @@ class TaskProcessor { /** * Unique task index number (run) */ - protected final AtomicInteger indexCount = new AtomicInteger() + final protected AtomicInteger indexCount = new AtomicInteger() /** * The current workflow execution session @@ -198,7 +197,7 @@ class TaskProcessor { * Note: it is declared static because the error must be shown only the * very first time for all processes */ - static private final AtomicBoolean errorShown = new AtomicBoolean() + private static final AtomicBoolean errorShown = new AtomicBoolean() /** * Flag set {@code true} when the processor termination has been invoked @@ -241,9 +240,9 @@ class TaskProcessor { private int maxForks - static private int processCount + private static int processCount - static private LockManager lockManager = new LockManager() + private static LockManager lockManager = new LockManager() private List> fairBuffers = new ArrayList<>() @@ -251,17 +250,6 @@ class TaskProcessor { private Boolean isFair0 - /** - * Map of all task processors by name. - */ - static private Map processorLookup = [:] - - /** - * Set of tasks (across all processors) that were deleted in a - * previous run and successfully restored from the cache db. - */ - static private Map restoredTasks = new ConcurrentHashMap<>() - private CompilerConfiguration compilerConfig() { final config = new CompilerConfiguration() config.addCompilationCustomizers( new ASTTransformationCustomizer(TaskTemplateVarsXform) ) @@ -316,8 +304,6 @@ class TaskProcessor { this.maxForks = config.maxForks ? config.maxForks as int : 0 this.forksCount = maxForks ? new LongAdder() : null this.isFair0 = config.getFair() - - processorLookup[name] = this } /** @@ -728,10 +714,6 @@ class TaskProcessor { return null } - synchronized protected TaskStartParams createTaskStartParams() { - return new TaskStartParams(TaskId.next(), indexCount.incrementAndGet()) - } - /** * Create a new {@code TaskRun} instance, initializing the following properties : *
  • {@code TaskRun#id} @@ -803,17 +785,23 @@ class TaskProcessor { while( true ) { hash = CacheHelper.defaultHasher().newHasher().putBytes(hash.asBytes()).putInt(tries).hash() + Path resumeDir = null + boolean exists = false try { - if( shouldTryCache && checkCachedOutput(task.clone(), hash) ) + final entry = session.cache.getTaskEntry(hash, this) + resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null + if( resumeDir ) + exists = resumeDir.exists() + + log.trace "[${safeTaskName(task)}] Cacheable folder=${resumeDir?.toUriString()} -- exists=$exists; try=$tries; shouldTryCache=$shouldTryCache; entry=$entry" + final cached = shouldTryCache && exists && entry.trace.isCompleted() && checkCachedOutput(task.clone(), resumeDir, hash, entry) + if( cached ) break } catch (Throwable t) { log.warn1("[${safeTaskName(task)}] Unable to resume cached task -- See log file for details", causedBy: t) } - final entry = session.cache.getTaskEntry(hash, processorLookup) - Path resumeDir = entry ? FileHelper.asPath(entry.trace.getWorkDir()) : null - boolean exists = resumeDir?.exists() if( exists ) { tries++ continue @@ -899,113 +887,88 @@ class TaskProcessor { } /** - * Attempt to restore a cached task by verifying either its outputs - * or the outputs of its consumers. + * Check whenever the outputs for the specified task already exist * - * @param seedTask - * @param seedHash - * @return {@code true} if all outputs are available, {@code false} otherwise + * @param task The task instance + * @param folder The folder where the outputs are stored (eventually) + * @return {@code true} when all outputs are available, {@code false} otherwise */ - final boolean checkCachedOutput(TaskRun seedTask, HashCode seedHash) { - - // -- recursively check for cached outputs - List queue = [ seedHash ] - Map handlers = [:] - - while( !queue.isEmpty() ) { - final hash = queue.pop() - - // -- skip tasks that have already been restored - if( hash in handlers ) - continue - - // -- get cache entry - final entry = session.cache.getTaskEntry(hash, processorLookup) - if( !entry || !entry.trace.isCompleted() ) - return false + final boolean checkCachedOutput(TaskRun task, Path folder, HashCode hash, TaskEntry entry) { - // -- get or create task run - def task - if( hash == seedHash ) { - task = seedTask - log.trace "[${safeTaskName(task)}] Cacheable task hash=${hash} entry=${entry}" - } - else if( entry.processor ) { - final params = entry.processor.createTaskStartParams() - task = entry.processor.createTaskRun(params) - log.trace "[${safeTaskName(task)}] Restoring deleted task hash=${hash} context=${entry.context}" - } - else { - log.trace "[${safeTaskName(seedTask)}] Missing processor for downstream hash=${hash} entry=${entry} -- return false" - return false + // check if exists the task exit code file + def exitCode = null + def exitFile = folder.resolve(TaskRun.CMD_EXIT) + if( task.type == ScriptType.SCRIPTLET ) { + def str + try { + str = exitFile.text?.trim() } - - // -- verify the task context map - if( task.hasCacheableValues() && !entry.context ) { - log.trace "[${safeTaskName(task)}] Missing cache context -- return false" + catch( IOException e ) { + log.trace "[${safeTaskName(task)}] Exit file can't be read > $exitFile -- return false -- Cause: ${e.message}" return false } - if( entry.context != null ) { - task.context = entry.context - task.config.context = entry.context - task.code?.delegate = entry.context - } - - // -- verify the task exit code - final exitCode = entry.trace.get('exit') as Integer - if( task.type == ScriptType.SCRIPTLET && !task.isSuccess(exitCode) ) { - log.trace "[${safeTaskName(task)}] Exit code is not valid > ${exitCode} -- return false" + exitCode = str.isInteger() ? str.toInteger() : null + if( !task.isSuccess(exitCode) ) { + log.trace "[${safeTaskName(task)}] Exit code is not valid > $str -- return false" return false } + } - // -- set the remaining task properties - task.cached = true - task.hash = hash - task.workDir = FileHelper.asPath(entry.trace.getWorkDir()) - task.stdout = task.workDir.resolve(TaskRun.CMD_OUTFILE) - task.exitStatus = exitCode - task.config.exitStatus = exitCode + /* + * verify cached context map + */ + if( !entry ) { + log.trace "[${safeTaskName(task)}] Missing cache entry -- return false" + return false + } - // -- check if all downstream outputs are available - if( entry.consumers != null ) { - queue.addAll( entry.consumers ) - } + if( task.hasCacheableValues() && !entry.context ) { + log.trace "[${safeTaskName(task)}] Missing cache context -- return false" + return false + } - // -- otherwise check if all task outputs are available - else { - try { - collectOutputs(task) - } - catch( MissingFileException | MissingValueException e ) { - log.trace "[${safeTaskName(task)}] Missed cache > ${e.getMessage()} -- folder: ${task.workDir}" - return false - } - } + /* + * verify stdout file + */ + final stdoutFile = folder.resolve( TaskRun.CMD_OUTFILE ) - // -- create task handler - handlers[hash] = new CachedTaskHandler(task, entry.trace) + if( entry.context != null ) { + task.context = entry.context + task.config.context = entry.context + task.code?.delegate = entry.context } - // -- finalize all cached tasks - handlers.each { hash, handler -> - if( hash in restoredTasks ) - return + try { + // -- expose task exit status to make accessible as output value + task.config.exitStatus = exitCode + // -- check if all output resources are available + collectOutputs(task, folder, stdoutFile, task.context) - final task = handler.task + // set the exit code in to the task object + task.cached = true + task.hash = hash + task.workDir = folder + task.stdout = stdoutFile + if( exitCode != null ) { + task.exitStatus = exitCode + } log.info "[${task.hashLog}] Cached process > ${task.name}" - // -- update the set of restored tasks - if( task.processor != this ) - task.processor.state.update { StateObj it -> it.incSubmitted() } - restoredTasks[hash] = true // -- notify cached event - session.notifyTaskCached(handler) - // -- bind the results - task.processor.finalizeTask0(task) - } + if( entry ) + session.notifyTaskCached(new CachedTaskHandler(task,entry.trace)) - return true + // -- now bind the results + finalizeTask0(task) + return true + } + catch( MissingFileException | MissingValueException e ) { + log.trace "[${safeTaskName(task)}] Missed cache > ${e.getMessage()} -- folder: $folder" + task.exitStatus = Integer.MAX_VALUE + task.workDir = null + return false + } } /** @@ -2479,7 +2442,7 @@ class TaskProcessor { state.update { StateObj it -> it.incSubmitted() } // task index must be created here to guarantee consistent ordering // with the sequence of messages arrival since this method is executed in a thread safe manner - final params = createTaskStartParams() + final params = new TaskStartParams(TaskId.next(), indexCount.incrementAndGet()) final result = new ArrayList(2) result[0] = params result[1] = messages diff --git a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy index c4e4abe459..7f9d67ed8a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/trace/TaskCleanupObserver.groovy @@ -24,7 +24,6 @@ import java.util.concurrent.locks.ReentrantLock import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import nextflow.Session -import nextflow.cache.CacheDB import nextflow.dag.DAG import nextflow.file.FileHelper import nextflow.processor.PublishDir.Mode @@ -243,8 +242,7 @@ class TaskCleanupObserver implements TraceObserver { /** * When a task fails, mark it as completed without tracking its - * output files. Failed tasks are not included as consumers of - * upstream tasks in the cache. + * output files or triggering a cleanup. * * @param task */ @@ -385,12 +383,6 @@ class TaskCleanupObserver implements TraceObserver { // mark task as deleted final taskState = tasks[task] taskState.deleted = true - - // finalize task in the cache db - final consumers = taskState.consumers - .findAll( t -> t.isSuccess() ) - .collect( t -> t.hash ) - session.cache.finalizeTaskAsync(task.hash, consumers) } /** diff --git a/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy b/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy index 0276a9ae79..8191c29695 100644 --- a/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/cache/CacheDBTest.groovy @@ -83,7 +83,7 @@ class CacheDBTest extends Specification { 1 * task.getContext() >> ctx when: - def entry = cache.getTaskEntry(hash, ['foo': proc]) + def entry = cache.getTaskEntry(hash, proc) then: entry instanceof TaskEntry entry.trace instanceof TraceRecord