diff --git a/modules/nextflow/src/main/groovy/nextflow/Channel.groovy b/modules/nextflow/src/main/groovy/nextflow/Channel.groovy index ee1bb55f43..6cb22e436a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Channel.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Channel.groovy @@ -522,7 +522,7 @@ class Channel { def groupChannel = isFlat ? new DataflowQueue<>() : CH.create() new GroupTupleOp(groupOpts, mapChannel) - .setTarget(groupChannel) + .withTarget(groupChannel) .apply() // -- flat the group resulting tuples diff --git a/modules/nextflow/src/main/groovy/nextflow/Session.groovy b/modules/nextflow/src/main/groovy/nextflow/Session.groovy index c9054e6673..863b82e815 100644 --- a/modules/nextflow/src/main/groovy/nextflow/Session.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/Session.groovy @@ -16,7 +16,6 @@ package nextflow - import java.nio.file.Files import java.nio.file.Path import java.nio.file.Paths @@ -53,6 +52,7 @@ import nextflow.processor.ErrorStrategy import nextflow.processor.TaskFault import nextflow.processor.TaskHandler import nextflow.processor.TaskProcessor +import nextflow.prov.Tracker import nextflow.script.BaseScript import nextflow.script.ProcessConfig import nextflow.script.ProcessFactory @@ -857,6 +857,8 @@ class Session implements ISession { DAG getDag() { this.dag } + Tracker getProvenance() { provenance } + ExecutorService getExecService() { execService } /** diff --git a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy index eca3a3fd27..0d6d611013 100644 --- a/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/executor/BashWrapperBuilder.groovy @@ -85,10 +85,8 @@ class BashWrapperBuilder { log.warn "Invalid value for `NXF_DEBUG` variable: $str -- See http://www.nextflow.io/docs/latest/config.html#environment-variables" } BASH = Collections.unmodifiableList( level > 0 ? ['/bin/bash','-uex'] : ['/bin/bash','-ue'] ) - } - @Delegate ScriptFileCopyStrategy copyStrategy @@ -480,6 +478,7 @@ class BashWrapperBuilder { protected String getTaskMetadata() { final lines = new StringBuilder() lines << '### ---\n' + lines << "### id: '${bean.taskId}'\n" lines << "### name: '${bean.name}'\n" if( bean.arrayIndexName ) { lines << '### array:\n' @@ -493,12 +492,18 @@ class BashWrapperBuilder { if( containerConfig?.isEnabled() ) lines << "### container: '${bean.containerImage}'\n" - if( outputFiles.size() > 0 ) { + if( outputFiles ) { lines << '### outputs:\n' for( final output : bean.outputFiles ) lines << "### - '${output}'\n" } + if( bean.upstreamTasks ) { + lines << '### upstream-tasks:\n' + for( final it : bean.upstreamTasks ) + lines << "### - '${it}'\n" + } + lines << '### ...\n' } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/BranchOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/BranchOp.groovy index 7fc5067620..0614fe1fd6 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/BranchOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/BranchOp.groovy @@ -20,7 +20,9 @@ import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.Op import nextflow.script.ChannelOut import nextflow.script.TokenBranchChoice import nextflow.script.TokenBranchDef @@ -50,20 +52,21 @@ class BranchOp { ChannelOut getOutput() { this.output } - protected void doNext(it) { + protected void doNext(DataflowProcessor dp, Object it) { TokenBranchChoice ret = switchDef.closure.call(it) if( ret ) { - targets[ret.choice].bind(ret.value) + Op.bind(dp, targets[ret.choice], ret.value) } } - protected void doComplete(nope) { + protected void doComplete(DataflowProcessor dp) { for( DataflowWriteChannel ch : targets.values() ) { if( ch instanceof DataflowExpression ) { - if( !ch.isBound()) ch.bind(Channel.STOP) + if( !ch.isBound() ) + Op.bind(dp, ch, Channel.STOP) } else { - ch.bind(Channel.STOP) + Op.bind(dp, ch, Channel.STOP) } } } @@ -72,7 +75,10 @@ class BranchOp { def events = new HashMap(2) events.put('onNext', this.&doNext) events.put('onComplete', this.&doComplete) - DataflowHelper.subscribeImpl(source, events) + new SubscribeOp() + .withSource(source) + .withEvents(events) + .apply() return this } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/BufferOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/BufferOp.groovy index 896381dc25..0a18688962 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/BufferOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/BufferOp.groovy @@ -16,6 +16,9 @@ package nextflow.extension +import static nextflow.extension.DataflowHelper.* +import static nextflow.util.CheckHelper.* + import groovy.transform.CompileDynamic import groovy.transform.CompileStatic import groovy.util.logging.Slf4j @@ -28,10 +31,11 @@ import groovyx.gpars.dataflow.operator.PoisonPill import nextflow.Channel import nextflow.Global import nextflow.Session +import nextflow.extension.op.ContextGrouping +import nextflow.extension.op.Op import org.codehaus.groovy.runtime.callsite.BooleanReturningMethodInvoker -import static nextflow.extension.DataflowHelper.newOperator -import static nextflow.util.CheckHelper.checkParams /** + * Implements the "buffer" operator * * @author Paolo Di Tommaso */ @@ -160,20 +164,16 @@ class BufferOp { final listener = new DataflowEventAdapter() { @Override - Object controlMessageArrived(final DataflowProcessor processor, final DataflowReadChannel channel, final int index, final Object message) { + Object controlMessageArrived(final DataflowProcessor dp, final DataflowReadChannel channel, final int index, final Object message) { if( message instanceof PoisonPill && remainder && buffer.size() ) { - target.bind(buffer) + Op.bind(dp, target, buffer) } return message } @Override - void afterRun(DataflowProcessor processor, List messages) { - if( !stopOnFirst ) - return - if( remainder && buffer) - target.bind(buffer) - target.bind(Channel.STOP) + void afterStop(DataflowProcessor dp) { + Op.bind(dp, target, Channel.STOP) } @Override @@ -187,8 +187,8 @@ class BufferOp { // -- open frame flag boolean isOpen = startingCriteria == null - // -- the operator collecting the elements - newOperator( source, target, listener ) { + // -- op code + final code = { if( isOpen ) { buffer << it } @@ -196,15 +196,27 @@ class BufferOp { isOpen = true buffer << it } - + final dp = getDelegate() as DataflowProcessor if( closeCriteria.call(it) ) { - ((DataflowProcessor) getDelegate()).bindOutput(buffer); + Op.bind(dp, target, buffer) buffer = [] // when a *startingCriteria* is defined, close the open frame flag isOpen = (startingCriteria == null) } - + if( stopOnFirst ) { + if( remainder && buffer ) + Op.bind(dp, target, buffer) + dp.terminate() + } } + + // -- the operator collecting the elements + new Op() + .withInput(source) + .withListener(listener) + .withContext(new ContextGrouping()) + .withCode(code) + .apply() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/ChainOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/ChainOp.groovy new file mode 100644 index 0000000000..3550c28fc8 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/ChainOp.groovy @@ -0,0 +1,86 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.ChainWithClosure +import groovyx.gpars.dataflow.operator.DataflowEventListener +import nextflow.extension.op.Op +/** + * Implements the chain operator + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class ChainOp { + + private DataflowReadChannel source + private DataflowWriteChannel target + private List listeners = List.of() + private Closure action + + static ChainOp create() { + new ChainOp() + } + + ChainOp withSource(DataflowReadChannel source) { + assert source + this.source = source + return this + } + + ChainOp withTarget(DataflowWriteChannel target) { + assert target + this.target = target + return this + } + + ChainOp withListener(DataflowEventListener listener) { + assert listener != null + this.listeners = List.of(listener) + return this + } + + ChainOp withListeners(List listeners) { + assert listeners != null + this.listeners = listeners + return this + } + + ChainOp withAction(Closure action) { + this.action = action + return this + } + + DataflowWriteChannel apply() { + assert source + assert target + assert action + + new Op() + .withInput(source) + .withOutput(target) + .withListeners(listeners) + .withCode(new ChainWithClosure(action)) + .apply() + + return target + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/ChannelEx.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/ChannelEx.groovy index ba68ef0390..121620bc10 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/ChannelEx.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/ChannelEx.groovy @@ -16,23 +16,23 @@ package nextflow.extension +import static nextflow.util.LoggerHelper.* + import groovy.transform.CompileDynamic import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import groovyx.gpars.agent.Agent +import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel -import nextflow.Channel -import nextflow.NF import nextflow.dag.NodeMarker import nextflow.exception.ScriptRuntimeException +import nextflow.prov.Tracker import nextflow.script.ChainableDef import nextflow.script.ChannelOut import nextflow.script.ComponentDef import nextflow.script.CompositeDef import nextflow.script.ExecutionStack import org.codehaus.groovy.runtime.InvokerHelper -import static nextflow.util.LoggerHelper.fmtType - /** * Implements dataflow channel extension methods * @@ -207,4 +207,11 @@ class ChannelEx { left.add(right) } + static Object unwrap(DataflowReadChannel self) { + final result = self.getVal() + return result instanceof Tracker.Msg + ? result.value + : result + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/CollateOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/CollateOp.groovy new file mode 100644 index 0000000000..307d8cbf6d --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/CollateOp.groovy @@ -0,0 +1,138 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import groovyx.gpars.dataflow.operator.PoisonPill +import nextflow.Global +import nextflow.Session +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext +import nextflow.extension.op.OpDatum +/** + * Implement "collate" operator + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class CollateOp { + private DataflowReadChannel source + private DataflowWriteChannel target + private int size + private int step + private boolean remainder + private OpContext context = new ContextRunPerThread() + + private static Session getSession() { Global.getSession() as Session } + + CollateOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + CollateOp withTarget( DataflowWriteChannel target ) { + this.target = target + return this + } + + CollateOp withSize(int size) { + this.size = size + return this + } + + CollateOp withStep(int step) { + this.step = step + return this + } + + CollateOp withRemainder(boolean keepRemainder) { + this.remainder = keepRemainder + return this + } + + DataflowWriteChannel apply() { + if( size <= 0 ) { + throw new IllegalArgumentException("Illegal argument 'size' for operator 'collate' -- it must be greater than zero: $size") + } + + if( step <= 0 ) { + throw new IllegalArgumentException("Illegal argument 'step' for operator 'collate' -- it must be greater than zero: $step") + } + + // the result queue + final target = CH.create() + + // the list holding temporary collected elements + List> allBuffers = [] + + // -- intercepts the PoisonPill and sent out the items remaining in the buffer when the 'remainder' flag is true + final listener = new DataflowEventAdapter() { + Object controlMessageArrived(final DataflowProcessor dp, final DataflowReadChannel channel, final int index, final Object message) { + if( message instanceof PoisonPill && remainder && allBuffers.size() ) { + for(List it : allBuffers) { + Op.bindRunValues(target, it, false) + } + } + return message + } + + @Override + boolean onException(DataflowProcessor dp, Throwable e) { + CollateOp.log.error("@unknown", e) + session.abort(e) + return true + } + } + + int index = 0 + new Op() + .withInput(source) + .withOutput(target) + .withContext(context) + .withListener(listener) + .withCode { + + if( index++ % step == 0 ) { + allBuffers.add( [] ) + } + + final run = context.getOperatorRun() + for( List list : allBuffers ) { + list.add(OpDatum.of(it,run)) + } + + final buf = allBuffers.head() + if( buf.size() == size ) { + Op.bindRunValues(target, buf, false) + allBuffers = allBuffers.tail() + } + } + .apply() + + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/CollectFileOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/CollectFileOp.groovy index 6f3dcd2b75..efc1ac5449 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/CollectFileOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/CollectFileOp.groovy @@ -16,20 +16,25 @@ package nextflow.extension +import static nextflow.util.CacheHelper.* +import static nextflow.util.CheckHelper.* + import java.nio.file.Path +import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel import nextflow.Global +import nextflow.extension.op.ContextGrouping +import nextflow.extension.op.Op import nextflow.file.FileCollector import nextflow.file.FileHelper import nextflow.file.SimpleFileCollector import nextflow.file.SortFileCollector import nextflow.util.CacheHelper -import static nextflow.util.CacheHelper.HashMode -import static nextflow.util.CheckHelper.checkParams /** * Implements the body of {@link OperatorImpl#collectFile(groovyx.gpars.dataflow.DataflowReadChannel)} operator * @@ -136,7 +141,7 @@ class CollectFileOp { * each time a value is received, invoke the closure and * append its result value to a file */ - protected processItem( item ) { + protected processItem( DataflowProcessor dp, Object item ) { def value = closure ? closure.call(item) : item // when the value is a list, the first item hold the grouping key @@ -182,13 +187,13 @@ class CollectFileOp { * * @params obj: NOT USED. It needs to be declared because this method is invoked as a closure */ - protected emitItems( obj ) { + protected emitItems(DataflowProcessor dp) { // emit collected files to 'result' channel collector.saveTo(storeDir).each { - result.bind(it) + Op.bind(dp, result, it) } // close the channel - result.bind(Channel.STOP) + Op.bind(dp, result, Channel.STOP) // close the collector collector.safeClose() } @@ -261,9 +266,15 @@ class CollectFileOp { return collector } - + @CompileStatic DataflowWriteChannel apply() { - DataflowHelper.subscribeImpl( channel, [onNext: this.&processItem, onComplete: this.&emitItems] ) + new SubscribeOp() + .withSource(channel) + .withOnNext(this.&processItem) + .withOnComplete(this.&emitItems) + .withContext( new ContextGrouping() ) + .apply() + return result } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/CollectOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/CollectOp.groovy index 9d7b02558f..f2fc3af815 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/CollectOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/CollectOp.groovy @@ -16,13 +16,17 @@ package nextflow.extension -import static nextflow.util.CheckHelper.checkParams +import static nextflow.util.CheckHelper.* import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.ContextGrouping +import nextflow.extension.op.Op import nextflow.util.ArrayBag + /** * Implements {@link OperatorImpl#collect(groovyx.gpars.dataflow.DataflowReadChannel)} operator * @@ -53,11 +57,16 @@ class CollectOp { final result = [] final target = new DataflowVariable() - Map events = [:] - events.onNext = { append(result, it) } - events.onComplete = { target << ( result ? new ArrayBag(normalise(result)) : Channel.STOP ) } + new SubscribeOp() + .withSource(source) + .withContext(new ContextGrouping()) + .withOnNext { append(result, it) } + .withOnComplete { DataflowProcessor dp -> + final msg = result ? new ArrayBag(normalise(result)) : Channel.STOP + Op.bind(dp, target, msg) + } + .apply() - DataflowHelper.subscribeImpl(source, events) return target } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/CombineOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/CombineOp.groovy index 8ef6765faf..2f3358632a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/CombineOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/CombineOp.groovy @@ -24,9 +24,17 @@ import groovy.transform.PackageScope import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel import static nextflow.extension.DataflowHelper.addToList import static nextflow.extension.DataflowHelper.makeKey + +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext +import nextflow.extension.op.OpDatum +import nextflow.prov.OperatorRun + /** * Implements the {@link OperatorImpl#spread(groovyx.gpars.dataflow.DataflowReadChannel, java.lang.Object)} operator * @@ -54,6 +62,8 @@ class CombineOp { private List pivot = NONE + private OpContext context = new ContextRunPerThread() + CombineOp(DataflowReadChannel left, Object right) { leftChannel = left @@ -89,20 +99,20 @@ class CombineOp { private Map handler(int index, DataflowWriteChannel target, AtomicInteger stopCount) { - def opts = new LinkedHashMap(2) + final opts = new LinkedHashMap(2) opts.onNext = { if( pivot ) { - def pair = makeKey(pivot, it) + final pair = makeKey(pivot, it, context.getOperatorRun()) emit(target, index, pair.keys, pair.values) } else { - emit(target, index, NONE, it) + emit(target, index, NONE, OpDatum.of(it, context.getOperatorRun())) } } - opts.onComplete = { - if( stopCount.decrementAndGet()==0) { - target << Channel.STOP + opts.onComplete = { DataflowProcessor dp -> + if( stopCount.decrementAndGet()==0 ) { + Op.bind(dp, target, Channel.STOP) }} return opts @@ -110,7 +120,7 @@ class CombineOp { @PackageScope @CompileDynamic - def tuple( List p, a, b ) { + Object tuple( List p, a, b ) { List result = new LinkedList() result.addAll(p) addToList(result, a) @@ -128,7 +138,7 @@ class CombineOp { if( index == LEFT ) { log.trace "combine >> left >> by=$p; val=$v; right-values: ${rightValues[p]}" for ( Object x : rightValues[p] ) { - target.bind( tuple(p, v, x) ) + bindValues(p, v, x) } leftValues[p].add(v) return @@ -137,7 +147,7 @@ class CombineOp { if( index == RIGHT ) { log.trace "combine >> right >> by=$p; val=$v; right-values: ${leftValues[p]}" for ( Object x : leftValues[p] ) { - target.bind( tuple(p, x, v) ) + bindValues(p, x, v) } rightValues[p].add(v) return @@ -146,19 +156,25 @@ class CombineOp { throw new IllegalArgumentException("Not a valid spread operator index: $index") } - DataflowWriteChannel apply() { + private void bindValues(List p, a, b) { + final i = new ArrayList() + final t = tuple(p, OpDatum.unwrap(a,i), OpDatum.unwrap(b,i)) + final r = new OperatorRun(new LinkedHashSet(i)) + Op.bind(r, target, t) + } + DataflowWriteChannel apply() { target = CH.create() if( rightChannel ) { final stopCount = new AtomicInteger(2) - DataflowHelper.subscribeImpl( leftChannel, handler(LEFT, target, stopCount) ) - DataflowHelper.subscribeImpl( rightChannel, handler(RIGHT, target, stopCount) ) + subscribe0( leftChannel, handler(LEFT, target, stopCount) ) + subscribe0( rightChannel, handler(RIGHT, target, stopCount) ) } else if( rightValues != null ) { final stopCount = new AtomicInteger(1) - DataflowHelper.subscribeImpl( leftChannel, handler(LEFT, target, stopCount) ) + subscribe0( leftChannel, handler(LEFT, target, stopCount) ) } else @@ -166,4 +182,12 @@ class CombineOp { return target } + + private void subscribe0(final DataflowReadChannel source, final Map events) { + new SubscribeOp() + .withSource(source) + .withEvents(events) + .withContext(context) + .apply() + } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/ConcatOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/ConcatOp.groovy index f21a34d76b..a8a1159f57 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/ConcatOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/ConcatOp.groovy @@ -19,7 +19,12 @@ package nextflow.extension import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext + /** * Implements the {@link OperatorImpl#concat} operator * @@ -32,36 +37,34 @@ class ConcatOp { private DataflowReadChannel[] target + private OpContext context = new ContextRunPerThread() + ConcatOp( DataflowReadChannel source, DataflowReadChannel... target ) { assert source != null assert target - this.source = source this.target = target } - DataflowWriteChannel apply() { final result = CH.create() final allChannels = [source] allChannels.addAll(target) - append(result, allChannels, 0) return result } - - private static void append( DataflowWriteChannel result, List channels, int index ) { - def current = channels[index++] - def next = index < channels.size() ? channels[index] : null - - def events = new HashMap(2) - events.onNext = { result.bind(it) } - events.onComplete = { - if(next) append(result, channels, index) - else result.bind(Channel.STOP) - } - - DataflowHelper.subscribeImpl(current, events) + private void append( DataflowWriteChannel result, List channels, int index ) { + final current = channels[index++] + final next = index < channels.size() ? channels[index] : null + new SubscribeOp() + .withSource(current) + .withContext(context) + .withOnNext { DataflowProcessor dp, Object it -> Op.bind(dp, result, it) } + .withOnComplete { DataflowProcessor dp -> + if(next) append(result, channels, index) + else Op.bind(dp, result, Channel.STOP) + } + .apply() } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/DataflowHelper.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/DataflowHelper.groovy index 14c775b04e..5ed5951cdb 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/DataflowHelper.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/DataflowHelper.groovy @@ -21,22 +21,20 @@ import java.lang.reflect.InvocationTargetException import groovy.transform.CompileStatic import groovy.transform.PackageScope import groovy.util.logging.Slf4j -import groovyx.gpars.dataflow.Dataflow import groovyx.gpars.dataflow.DataflowChannel import groovyx.gpars.dataflow.DataflowQueue import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.expression.DataflowExpression -import groovyx.gpars.dataflow.operator.ChainWithClosure import groovyx.gpars.dataflow.operator.DataflowEventAdapter import groovyx.gpars.dataflow.operator.DataflowEventListener import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel import nextflow.Global import nextflow.Session -import nextflow.dag.NodeMarker -import static java.util.Arrays.asList +import nextflow.extension.op.Op +import nextflow.prov.OperatorRun /** * This class provides helper methods to implement nextflow operators * @@ -93,7 +91,7 @@ class DataflowHelper { @PackageScope static DEF_ERROR_LISTENER = new DataflowEventAdapter() { @Override - boolean onException(final DataflowProcessor processor, final Throwable t) { + boolean onException(final DataflowProcessor dp, final Throwable t) { final e = t instanceof InvocationTargetException ? t.cause : t OperatorImpl.log.error("@unknown", e) session?.abort(e) @@ -106,16 +104,16 @@ class DataflowHelper { new DataflowEventAdapter() { @Override - void afterRun(final DataflowProcessor processor, final List messages) { + void afterRun(final DataflowProcessor dp, final List messages) { if( source instanceof DataflowExpression ) { - if( !(target instanceof DataflowExpression) ) - processor.bindOutput( Channel.STOP ) - processor.terminate() + if( target !instanceof DataflowExpression ) + Op.bind(dp, target, Channel.STOP ) + dp.terminate() } } @Override - boolean onException(final DataflowProcessor processor, final Throwable e) { + boolean onException(final DataflowProcessor dp, final Throwable e) { DataflowHelper.log.error("@unknown", e) session.abort(e) return true @@ -141,21 +139,19 @@ class DataflowHelper { * @param params The map holding inputs, outputs channels and other parameters * @param code The closure to be executed by the operator */ + @Deprecated static DataflowProcessor newOperator( Map params, Closure code ) { // -- add a default error listener - if( !params.containsKey('listeners') ) { + if( !params.listeners ) { // add the default error handler params.listeners = [ DEF_ERROR_LISTENER ] } - final op = Dataflow.operator(params, code) - NodeMarker.appendOperator(op) - if( session && session.allOperators != null ) { - session.allOperators.add(op) - } - - return op + return new Op() + .withParams(params) + .withCode(code) + .apply() } /** @@ -167,6 +163,7 @@ class DataflowHelper { * @param outputs The list of list output {@code DataflowWriteChannel}s * @param code The closure to be executed by the operator */ + @Deprecated static DataflowProcessor newOperator( List inputs, List outputs, Closure code ) { newOperator( inputs: inputs, outputs: outputs, code ) } @@ -195,169 +192,33 @@ class DataflowHelper { * @param code The closure to be executed by the operator */ static DataflowProcessor newOperator( DataflowReadChannel input, DataflowWriteChannel output, DataflowEventListener listener, Closure code ) { - if( !listener ) listener = DEF_ERROR_LISTENER - def params = [:] + final params = [:] params.inputs = [input] params.outputs = [output] params.listeners = [listener] - final op = Dataflow.operator(params, code) - NodeMarker.appendOperator(op) - if( session && session.allOperators != null ) { - session.allOperators << op - } - return op + return new Op() + .withParams(params) + .withCode(code) + .apply() } - /* - * the list of valid subscription handlers - */ - static private VALID_HANDLERS = [ 'onNext', 'onComplete', 'onError' ] - - /** - * Verify that the map contains only valid names of subscribe handlers. - * Throws an {@code IllegalArgumentException} when an invalid name is specified - * - * @param handlers The handlers map - */ - @PackageScope - static checkSubscribeHandlers( Map handlers ) { - - if( !handlers ) { - throw new IllegalArgumentException("You must specify at least one of the following events: onNext, onComplete, onError") - } - - handlers.keySet().each { - if( !VALID_HANDLERS.contains(it) ) throw new IllegalArgumentException("Not a valid handler name: $it") - } - - } - - /** - * Subscribe *onNext*, *onError* and *onComplete* - * - * @param source - * @param closure - * @return - */ static final DataflowProcessor subscribeImpl(final DataflowReadChannel source, final Map events ) { - checkSubscribeHandlers(events) - - def error = false - def stopOnFirst = source instanceof DataflowExpression - def listener = new DataflowEventAdapter() { - - @Override - void afterStop(final DataflowProcessor processor) { - if( !events.onComplete || error ) return - try { - events.onComplete.call(processor) - } - catch( Exception e ) { - OperatorImpl.log.error("@unknown", e) - session.abort(e) - } - } - - @Override - boolean onException(final DataflowProcessor processor, final Throwable e) { - error = true - if( !events.onError ) { - log.error("@unknown", e) - session.abort(e) - } - else { - events.onError.call(e) - } - return true - } - } - - - final Map parameters = new HashMap(); - parameters.put("inputs", [source]) - parameters.put("outputs", []) - parameters.put('listeners', [listener]) - - newOperator (parameters) { - if( events.onNext ) { - events.onNext.call(it) - } - if( stopOnFirst ) { - ((DataflowProcessor) getDelegate()).terminate() - } - } - } - - - static DataflowProcessor chainImpl(final DataflowReadChannel source, final DataflowWriteChannel target, final Map params, final Closure closure) { - - final Map parameters = new HashMap(params) - parameters.put("inputs", asList(source)) - parameters.put("outputs", asList(target)) - - newOperator(parameters, new ChainWithClosure(closure)) - } - - /** - * Implements the {@code #reduce} operator - * - * @param channel - * @param seed - * @param closure - * @return - */ - static DataflowProcessor reduceImpl(final DataflowReadChannel channel, final DataflowVariable result, def seed, final Closure closure) { - - // the *accumulator* value - def accum = seed - - // intercepts operator events - def listener = new DataflowEventAdapter() { - /* - * call the passed closure each time - */ - void afterRun(final DataflowProcessor processor, final List messages) { - final item = messages.get(0) - final value = accum == null ? item : closure.call(accum, item) - - if( value == Channel.VOID ) { - // do nothing - } - else if( value == Channel.STOP ) { - processor.terminate() - } - else { - accum = value - } - } - - /* - * when terminates bind the result value - */ - void afterStop(final DataflowProcessor processor) { - result.bind(accum) - } - - boolean onException(final DataflowProcessor processor, final Throwable e) { - log.error("@unknown", e) - session.abort(e) - return true; - } - } - - chainImpl(channel, CH.create(), [listeners: [listener]], {true}) + new SubscribeOp() + .withSource(source) + .withEvents(events) + .apply() } @PackageScope @CompileStatic - static KeyPair makeKey(List pivot, entry) { + static KeyPair makeKey(List pivot, entry, OperatorRun run) { final result = new KeyPair() - if( !(entry instanceof List) ) { + if( entry !instanceof List ) { if( pivot != [0] ) throw new IllegalArgumentException("Not a valid `by` index: $pivot") result.keys = [entry] @@ -373,7 +234,7 @@ class DataflowHelper { if( i in pivot ) result.addKey(list[i]) else - result.addValue(list[i]) + result.addValue(list[i], run) } return result @@ -390,11 +251,4 @@ class DataflowHelper { } } - @CompileStatic - static Map eventsMap(Closure onNext, Closure onComplete) { - def result = new HashMap(2) - result.put('onNext', onNext) - result.put('onComplete', onComplete) - return result - } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/DistinctOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/DistinctOp.groovy new file mode 100644 index 0000000000..00df03a9d1 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/DistinctOp.groovy @@ -0,0 +1,81 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.extension.op.Op +/** + * Implements the "distinct" operator logic + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class DistinctOp { + + private DataflowReadChannel source + private DataflowWriteChannel target + private Closure comparator + + DistinctOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + DistinctOp withTarget(DataflowWriteChannel target) { + assert target!=null + this.target = target + return this + } + + DistinctOp withComparator(Closure comparator) { + assert comparator!=null + this.comparator = comparator + return this + } + + DataflowWriteChannel apply() { + assert source != null + assert comparator != null + + if( !target ) + target = CH.createBy(source) + + def previous = null + final code = { + final dp = getDelegate() as DataflowProcessor + final key = comparator.call(it) + if( key != previous ) { + previous = key + Op.bind(dp, target, it) + } + } + + new Op() + .withInput(source) + .withOutput(target) + .withCode(code) + .apply() + + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/DumpOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/DumpOp.groovy index 2aae01dbab..79d69ccf82 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/DumpOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/DumpOp.groovy @@ -96,7 +96,11 @@ class DumpOp { events.onComplete = { CH.close0(target) } - DataflowHelper.subscribeImpl(source, events) + new SubscribeOp() + .withSource(source) + .withEvents(events) + .apply() + return target } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/FilterOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/FilterOp.groovy new file mode 100644 index 0000000000..df5bec5842 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/FilterOp.groovy @@ -0,0 +1,82 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import static nextflow.extension.DataflowHelper.* + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.extension.op.Op +import org.codehaus.groovy.runtime.callsite.BooleanReturningMethodInvoker +import org.codehaus.groovy.runtime.typehandling.DefaultTypeTransformation +/** + * Implements the "filter" operator logic + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class FilterOp { + + private DataflowReadChannel source + + private Object criteria + + FilterOp withSource(DataflowReadChannel source) { + this.source = source + return this + } + + FilterOp withCriteria(Closure criteria) { + this.criteria = criteria + return this + } + + FilterOp withCriteria(Object criteria) { + this.criteria = criteria + return this + } + + DataflowWriteChannel apply() { + assert source!=null + assert criteria!=null + + final discriminator = criteria !instanceof Closure + ? new BooleanReturningMethodInvoker("isCase") + : null + final target = CH.createBy(source) + final stopOnFirst = source instanceof DataflowExpression + newOperator(source, target, { + final result = criteria instanceof Closure + ? DefaultTypeTransformation.castToBoolean(criteria.call(it)) + : discriminator.invoke(criteria, (Object)it) + final dp = getDelegate() as DataflowProcessor + if( result ) { + Op.bind(dp, target, it) + } + if( stopOnFirst ) { + Op.bind(dp, target, Channel.STOP) + } + }) + + return target + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/FirstOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/FirstOp.groovy new file mode 100644 index 0000000000..22805ad527 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/FirstOp.groovy @@ -0,0 +1,97 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowVariable +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.extension.op.Op +import org.codehaus.groovy.runtime.callsite.BooleanReturningMethodInvoker +/** + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class FirstOp { + + private DataflowReadChannel source + private DataflowVariable target + private Object criteria + + FirstOp() {} + + FirstOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + FirstOp withTarget(DataflowVariable target) { + assert target!=null + this.target = target + return this + } + + FirstOp withCriteria(Object criteria) { + assert criteria!=null + this.criteria = criteria + return this + } + + DataflowWriteChannel apply() { + assert source!=null + assert criteria!=null + + if( target==null ) + target = new DataflowVariable() + + final stopOnFirst = source instanceof DataflowExpression + final discriminator = new BooleanReturningMethodInvoker("isCase"); + + final listener = new DataflowEventAdapter() { + @Override + void afterStop(DataflowProcessor dp) { + if( stopOnFirst && !target.isBound() ) + Op.bind(dp, target, Channel.STOP) + } + } + + final code = { + final dp = getDelegate() as DataflowProcessor + final accept = discriminator.invoke(criteria, it) + if( accept ) + Op.bind(dp, target, it) + if( accept || stopOnFirst ) + dp.terminate() + } + + new Op() + .withInput(source) + .withListener(listener) + .withCode(code) + .apply() + + return target + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/FlatMapOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/FlatMapOp.groovy new file mode 100644 index 0000000000..86959732de --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/FlatMapOp.groovy @@ -0,0 +1,99 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import static nextflow.extension.DataflowHelper.* + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.extension.op.Op +/** + * Implement "flatMap" operator + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class FlatMapOp { + + private DataflowReadChannel source + private DataflowWriteChannel target + private Closure mapper + + FlatMapOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + FlatMapOp withMapper(Closure code) { + this.mapper = code + return this + } + + FlatMapOp setTarget( DataflowWriteChannel target ) { + this.target = target + return this + } + + DataflowWriteChannel apply() { + assert source!=null + + if( target==null ) + target = CH.create() + + new Op() + .withInput(source) + .withOutput(target) + .withListener(stopErrorListener(source,target)) + .withCode { Object item -> + final result = mapper != null ? mapper.call(item) : item + final dp = getDelegate() as DataflowProcessor + + switch( result ) { + case Collection: + result.each { it -> Op.bind(dp, target,it) } + break + + case (Object[]): + result.each { it -> Op.bind(dp, target,it) } + break + + case Map: + result.each { it -> Op.bind(dp, target,it) } + break + + case Map.Entry: + Op.bind(dp, target, (result as Map.Entry).key ) + Op.bind(dp, target, (result as Map.Entry).value ) + break + + case Channel.VOID: + break + + default: + Op.bind(dp, target, result) + } + } + .apply() + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/FlattenOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/FlattenOp.groovy new file mode 100644 index 0000000000..b7d629535b --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/FlattenOp.groovy @@ -0,0 +1,106 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.Global +import nextflow.Session +import nextflow.extension.op.Op +/** + * Implements "flatten" operator + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class FlattenOp { + + private static Session getSession() { Global.getSession() as Session } + + private DataflowReadChannel source + private DataflowWriteChannel target + + FlattenOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + FlattenOp setTarget( DataflowWriteChannel target ) { + this.target = target + return this + } + + DataflowWriteChannel apply() { + final target = CH.create() + final stopOnFirst = source instanceof DataflowExpression + + final listener = new DataflowEventAdapter() { + @Override + void afterRun(final DataflowProcessor dp, final List messages) { + if( stopOnFirst ) + dp.terminate() + } + + @Override + void afterStop(final DataflowProcessor dp) { + Op.bind(dp, target, Channel.STOP) + } + + boolean onException(final DataflowProcessor dp, final Throwable e) { + FlattenOp.log.error("@unknown", e) + session.abort(e) + return true; + } + } + + new Op() + .withInput(source) + .withListener(listener) + .withCode { Object item -> + final dp = getDelegate() as DataflowProcessor + switch( item ) { + case Collection: + ((Collection)item).flatten().each { value -> Op.bind(dp, target, value) } + break + + case (Object[]): + ((Collection)item).flatten().each { value -> Op.bind(dp, target, value) } + break + + case Channel.VOID: + break + + default: + Op.bind(dp, target, item) + } + } + .apply() + + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/GroupTupleOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/GroupTupleOp.groovy index 7c6279eb7f..ce2218764c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/GroupTupleOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/GroupTupleOp.groovy @@ -16,10 +16,17 @@ package nextflow.extension +import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext +import nextflow.extension.op.OpDatum +import nextflow.prov.OperatorRun import nextflow.util.ArrayBag import nextflow.util.CacheHelper import nextflow.util.CheckHelper @@ -29,18 +36,19 @@ import nextflow.util.CheckHelper * @author Paolo Di Tommaso */ @Slf4j +@CompileStatic class GroupTupleOp { - static private Map GROUP_TUPLE_PARAMS = [ by: [Integer, List], sort: [Boolean, 'true','natural','deep','hash',Closure,Comparator], size: Integer, remainder: Boolean ] + static final private Map GROUP_TUPLE_PARAMS = [ by: [Integer, List], sort: [Boolean, 'true','natural','deep','hash',Closure,Comparator], size: Integer, remainder: Boolean ] - static private List GROUP_DEFAULT_INDEX = [0] + static final private List GROUP_DEFAULT_INDEX = [0] /** * Comparator used to sort tuple entries (when required) */ private Comparator comparator - private int size + private Integer size private List indices @@ -54,20 +62,21 @@ class GroupTupleOp { private sort - GroupTupleOp(Map params, DataflowReadChannel source) { + private OpContext context = new ContextRunPerThread() + GroupTupleOp(Map params, DataflowReadChannel source) { CheckHelper.checkParams('groupTuple', params, GROUP_TUPLE_PARAMS) channel = source indices = getGroupTupleIndices(params) - size = params?.size ?: 0 + size = params?.size as Integer ?: 0 remainder = params?.remainder ?: false sort = params?.sort defineComparator() } - GroupTupleOp setTarget(DataflowWriteChannel target) { + GroupTupleOp withTarget(DataflowWriteChannel target) { this.target = target return this } @@ -89,7 +98,7 @@ class GroupTupleOp { /* * Collects received values grouping by key */ - private void collect(List tuple) { + private void collectTuple(List tuple) { final key = tuple[indices] // the actual grouping key final len = tuple.size() @@ -101,6 +110,7 @@ class GroupTupleOp { return result } + final run = context.getOperatorRun() int count=-1 for( int i=0; i bindTuple(items, size ?: sizeBy(keys)) } - target.bind(Channel.STOP) + Op.bind(dp, target, Channel.STOP) } /* * bind collected items to the target channel */ private void bindTuple( List items, int sz ) { - - def tuple = new ArrayList(items) - + final tuple = new ArrayList(items) if( !remainder && sz>0 ) { // verify exist it contains 'size' elements - List list = items.find { it instanceof List } + def list = (List) items.find { it instanceof List } if( list.size() != sz ) { return } } - + // unwrap all "OpData" object and restore original values + final run = unwrapValues(tuple) + // sort the tuple content when a comparator is defined if( comparator ) { sortInnerLists(tuple, comparator) } + // finally bind the resulting tuple + Op.bind(run, target, tuple) + } + + static protected OperatorRun unwrapValues(List tuple) { + final inputs = new ArrayList() + + for( Object it : tuple ) { + if( it instanceof ArrayBag ) { + final bag = it + for( int i=0; i(inputs)) } /** * Define the comparator to be used depending the #sort property */ private void defineComparator( ) { - /* * comparator logic used to sort tuple elements */ @@ -167,21 +191,21 @@ class GroupTupleOp { case true: case 'true': case 'natural': - comparator = { o1,o2 -> o1<=>o2 } as Comparator + comparator = { o1, o2 -> o1<=>o2 } as Comparator break; case 'hash': comparator = { o1, o2 -> - def h1 = CacheHelper.hasher(o1).hash() - def h2 = CacheHelper.hasher(o2).hash() + final h1 = CacheHelper.hasher(o1).hash() + final h2 = CacheHelper.hasher(o2).hash() return h1.asLong() <=> h2.asLong() } as Comparator break case 'deep': comparator = { o1, o2 -> - def h1 = CacheHelper.hasher(o1, CacheHelper.HashMode.DEEP).hash() - def h2 = CacheHelper.hasher(o2, CacheHelper.HashMode.DEEP).hash() + final h1 = CacheHelper.hasher(o1, CacheHelper.HashMode.DEEP).hash() + final h2 = CacheHelper.hasher(o2, CacheHelper.HashMode.DEEP).hash() return h1.asLong() <=> h2.asLong() } as Comparator break @@ -197,8 +221,8 @@ class GroupTupleOp { } else if( closure.getMaximumNumberOfParameters()==1 ) { comparator = { o1, o2 -> - def v1 = closure.call(o1) - def v2 = closure.call(o2) + final v1 = closure.call(o1) as Comparable + final v2 = closure.call(o2) as Comparable return v1 <=> v2 } as Comparator } @@ -209,7 +233,6 @@ class GroupTupleOp { default: throw new IllegalArgumentException("Not a valid sort argument: ${sort}") } - } /** @@ -223,9 +246,14 @@ class GroupTupleOp { target = CH.create() /* - * apply the logic the the source channel + * apply the logic to the source channel */ - DataflowHelper.subscribeImpl(channel, [onNext: this.&collect, onComplete: this.&finalise]) + new SubscribeOp() + .withSource(channel) + .withContext(context) + .withOnNext(this.&collectTuple) + .withOnComplete(this.&finalise) + .apply() /* * return the target channel @@ -234,13 +262,11 @@ class GroupTupleOp { } private static sortInnerLists( List tuple, Comparator c ) { - for( int i=0; i targets ) { assert source assert targets - this.source = source this.outputs = targets } @@ -67,27 +65,6 @@ class IntoOp { this.outputs = targets } - IntoOp( DataflowReadChannel source, Closure holder ) { - assert source - assert holder - - final names = CaptureProperties.capture(holder) - if( !names ) - throw new IllegalArgumentException("Missing target channel names in `into` operator") - if( names.size() == 1 ) - log.warn("The `into` operator should be used to connect two or more target channels -- consider replacing it with `.set { ${names[0]} }`") - - List targets = [] - names.each { identifier -> - def channel = newChannelBy(source) - targets.add(channel) - NF.binding.setVariable(identifier, channel) - } - - this.source = source - this.outputs = targets - } - List getOutputs() { outputs } IntoOp apply() { @@ -107,10 +84,10 @@ class IntoOp { final stopOnFirst = source instanceof DataflowExpression final listener = new DataflowEventAdapter() { @Override - void afterRun(DataflowProcessor processor, List messages) { + void afterRun(DataflowProcessor dp, List messages) { if( !stopOnFirst ) return // -- terminate the process - processor.terminate() + dp.terminate() // -- close the output channels for( def it : outputs ) { if( !(it instanceof DataflowExpression)) @@ -123,7 +100,7 @@ class IntoOp { } @Override - public boolean onException(final DataflowProcessor processor, final Throwable e) { + public boolean onException(final DataflowProcessor dp, final Throwable e) { log.error("@unknown", e) session.abort(e) return true; diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/JoinOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/JoinOp.groovy index 3d9075ba60..396ef3a883 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/JoinOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/JoinOp.groovy @@ -25,9 +25,14 @@ import groovy.transform.PackageScope import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel import nextflow.NF import nextflow.exception.AbortOperationException +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext +import nextflow.extension.op.OpDatum import nextflow.util.CheckHelper /** * Implements {@link OperatorImpl#join} operator logic @@ -58,6 +63,8 @@ class JoinOp { private Set uniqueKeys = new LinkedHashSet() + private OpContext context = new ContextRunPerThread() + JoinOp( DataflowReadChannel source, DataflowReadChannel target, Map params = null ) { CheckHelper.checkParams('join', params, JOIN_PARAMS) this.source = source @@ -81,7 +88,6 @@ class JoinOp { } DataflowWriteChannel apply() { - // the resulting channel final result = CH.create() // the following buffer maintains the state of collected items as a map of maps. @@ -91,12 +97,19 @@ class JoinOp { final count = 2 final stopCount = new AtomicInteger(count) - - DataflowHelper.subscribeImpl( source, handler(state, count, 0, result, stopCount, remainder) ) - DataflowHelper.subscribeImpl( target, handler(state, count, 1, result, stopCount, remainder) ) + subscribe0( source, handler(state, count, 0, result, stopCount, remainder) ) + subscribe0( target, handler(state, count, 1, result, stopCount, remainder) ) return result } + private void subscribe0(DataflowReadChannel source, Map events) { + new SubscribeOp() + .withSource(source) + .withEvents(events) + .withContext(context) + .apply() + } + /** * Returns the methods {@code OnNext} and {@code onComplete} which will implement the join logic * @@ -111,22 +124,22 @@ class JoinOp { final Map result = new HashMap<>(2) - result.onNext = { + result.onNext = { DataflowProcessor dp, Object it -> synchronized (this) { if(!failed) try { - def entries = join0(buffer, size, index, it) + final entries = join0(buffer, size, index, it) if( entries ) { - target.bind( entries.size()==1 ? entries[0] : entries ) + Op.bindRunValues(target, entries, true) } } catch (Exception e) { failed = true - target << Channel.STOP + Op.bind(dp, target, Channel.STOP) throw e } }} - result.onComplete = { + result.onComplete = { DataflowProcessor dp -> if( stopCount.decrementAndGet()==0 && !failed ) { try { if( remainder || failOnDuplicate ) @@ -135,14 +148,15 @@ class JoinOp { checkForMismatch(buffer) } finally { - target << Channel.STOP + Op.bind(dp, target, Channel.STOP) } }} - + return result } + /** * Implements the join operator logic. Basically buffers the values received on each channel by their key . * @@ -171,7 +185,7 @@ class JoinOp { // before a match for it is found on another channel) // get the index key for this object - final item0 = DataflowHelper.makeKey(pivot, data) + final item0 = DataflowHelper.makeKey(pivot, data, context.getOperatorRun()) // check for unique keys checkForDuplicate(item0.keys, item0.values, index, false) @@ -187,11 +201,10 @@ class JoinOp { channels[index] = [] } - def entries = channels[index] - // add the received item to the list // when it is used in the gather op add always as the first item - entries << item0.values + final entries = channels[index] + entries.add(item0.values) setSingleton(index, item0.values.size()==0) // now check if it has received an element matching for each channel @@ -221,7 +234,7 @@ class JoinOp { return result } - private final void checkRemainder(Map> buffers, int count, DataflowWriteChannel target ) { + private final void checkRemainder(Map> buffers, int count, DataflowWriteChannel target) { log.trace "Operator `join` remainder buffer: ${-> buffers}" for( Object key : buffers.keySet() ) { @@ -249,14 +262,13 @@ class JoinOp { } if( fill ) { - final value = singleton() ? result[0] : result // bind value to target channel - if( remainder ) target.bind(value) + if( remainder ) + Op.bindRunValues(target, result, true) } else break } - } } @@ -293,7 +305,10 @@ class JoinOp { private String csv0(value, String sep) { - value instanceof List ? value.join(sep) : value.toString() + final result = value instanceof List + ? value.collect(it->OpDatum.unwrap(it)).join(sep) + : OpDatum.unwrap(value).toString() + return result } private boolean singleton(int i=-1) { diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/KeyPair.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/KeyPair.groovy index e794760b82..7aacde77c9 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/KeyPair.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/KeyPair.groovy @@ -19,6 +19,8 @@ package nextflow.extension import groovy.transform.CompileStatic import groovy.transform.EqualsAndHashCode import groovy.transform.ToString +import nextflow.extension.op.OpDatum +import nextflow.prov.OperatorRun /** * Implements an helper key-value helper object used in dataflow operators @@ -36,8 +38,9 @@ class KeyPair { keys.add(safeStr(el)) } - void addValue(el) { - values.add(el) + void addValue(el, OperatorRun run) { + final v = run ? OpDatum.of(el,run) : el + values.add(v) } static private safeStr(key) { diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/LastOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/LastOp.groovy new file mode 100644 index 0000000000..007cf7c392 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/LastOp.groovy @@ -0,0 +1,64 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowVariable +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.extension.op.Op + +/** + * Implements last operator + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class LastOp { + + private DataflowReadChannel source + private DataflowVariable target + + LastOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + LastOp withTarget(DataflowVariable target) { + assert target!=null + this.target = target + return this + } + + DataflowVariable apply() { + assert source!=null + if( target==null ) + target = new DataflowVariable() + + def last = null + new SubscribeOp() + .withSource(source) + .withOnNext{ last = it } + .withOnComplete{ DataflowProcessor dp -> Op.bind(dp, target, last) } + .apply() + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/MapOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/MapOp.groovy index 78e05a0777..87b0e95306 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/MapOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/MapOp.groovy @@ -16,17 +16,20 @@ package nextflow.extension - +import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.expression.DataflowExpression import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.Op + /** * Implements {@link OperatorImpl#map(groovyx.gpars.dataflow.DataflowReadChannel, groovy.lang.Closure)} operator * * @author Paolo Di Tommaso */ +@CompileStatic class MapOp { private DataflowReadChannel source @@ -35,11 +38,25 @@ class MapOp { private DataflowWriteChannel target + MapOp() {} + MapOp( final DataflowReadChannel source, final Closure mapper ) { this.source = source this.mapper = mapper } + MapOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + MapOp withMapper(Closure code) { + assert code!=null + this.mapper = code + return this + } + MapOp setTarget( DataflowWriteChannel target ) { this.target = target return this @@ -53,16 +70,16 @@ class MapOp { final stopOnFirst = source instanceof DataflowExpression DataflowHelper.newOperator(source, target) { it -> - def result = mapper.call(it) - def proc = (DataflowProcessor) getDelegate() + final result = mapper.call(it) + final proc = getDelegate() as DataflowProcessor // bind the result value if (result != Channel.VOID) - proc.bindOutput(result) + Op.bind(proc, target, result) // when the `map` operator is applied to a dataflow flow variable // terminate the processor after the first emission -- Issue #44 - if( result == Channel.STOP || stopOnFirst ) + if( stopOnFirst ) proc.terminate() } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/MathOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/MathOp.groovy new file mode 100644 index 0000000000..ec7792c59c --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/MathOp.groovy @@ -0,0 +1,140 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + + +import groovy.transform.CompileDynamic +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowVariable +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.extension.op.ContextGrouping +import nextflow.extension.op.Op +/** + * Implements the logic for "sum" and "mean" operators + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class MathOp { + + private DataflowReadChannel source + private DataflowVariable target + private Closure action + private Aggregate aggregate + + private MathOp(Aggregate aggregate) { + this.aggregate = aggregate + } + + static MathOp sum() { + new MathOp(new Aggregate(name: 'sum')) + } + + static MathOp mean() { + new MathOp(new Aggregate(name: 'mean', mean: true)) + } + + MathOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + MathOp withTarget(DataflowVariable target) { + assert target!=null + this.target = target + return this + } + + MathOp withAction(Closure action) { + this.action = action + this.aggregate.withAction(action) + return this + } + + DataflowWriteChannel apply() { + assert source!=null + if( target==null ) + target = new DataflowVariable() + new SubscribeOp() + .withSource(source) + .withContext(new ContextGrouping()) + .withOnNext(aggregate.&process) + .withOnComplete(this.&completion) + .apply() + return target + } + + private void completion(DataflowProcessor dp) { + Op.bind(dp, target, aggregate.result ) + } + + /** + * Implements the logic for sum and mean operators + * + * @author Paolo Di Tommaso + */ + @CompileDynamic + static class Aggregate { + def accum + long count = 0 + boolean mean + Closure action + String name + + def process(it) { + if( it == null || it == Channel.VOID ) + return + + count++ + + def item = action != null ? action.call(it) : it + if( accum == null ) + accum = item + + else if( accum instanceof Number ) + accum += item + + else if( accum instanceof List && item instanceof List) + for( int i=0; i others private DataflowWriteChannel target + private OpContext context = new ContextRunPerThread() MixOp(DataflowReadChannel source, DataflowReadChannel other) { this.source = source @@ -63,15 +66,15 @@ class MixOp { DataflowWriteChannel apply() { if( target == null ) target = CH.create() - def count = new AtomicInteger( others.size()+1 ) - def handlers = [ - onNext: { target << it }, - onComplete: { if(count.decrementAndGet()==0) { target << Channel.STOP } } + final count = new AtomicInteger( others.size()+1 ) + final handlers = [ + onNext: { DataflowProcessor dp, it -> Op.bind(dp, target, it) }, + onComplete: { DataflowProcessor dp -> if(count.decrementAndGet()==0) { Op.bind(dp, target, Channel.STOP) } } ] - subscribeImpl(source, handlers) + subscribe0(source, handlers) for( def it : others ) { - subscribeImpl(it, handlers) + subscribe0(it, handlers) } final allSources = [source] @@ -79,4 +82,12 @@ class MixOp { return target } + private void subscribe0(final DataflowReadChannel source, final Map events ) { + new SubscribeOp() + .withSource(source) + .withContext(context) + .withEvents(events) + .apply() + } + } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/MultiMapOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/MultiMapOp.groovy index 57693b6901..875ca1da63 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/MultiMapOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/MultiMapOp.groovy @@ -20,7 +20,9 @@ import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.Op import nextflow.script.ChannelOut import nextflow.script.TokenMultiMapDef /** @@ -48,29 +50,30 @@ class MultiMapOp { ChannelOut getOutput() { this.output } - protected void doNext(it) { + protected void doNext(DataflowProcessor dp, Object it) { final ret = (Map)forkDef.closure.call(it) for( Map.Entry entry : ret.entrySet() ) { - targets[entry.key].bind(entry.value) + Op.bind(dp, targets[entry.key], entry.value) } } - protected void doComplete(nope) { + protected void doComplete(DataflowProcessor dp) { for( DataflowWriteChannel ch : targets.values() ) { if( ch instanceof DataflowExpression ) { - if( !ch.isBound()) ch.bind(Channel.STOP) + if( !ch.isBound()) Op.bind(dp, ch, Channel.STOP) } else { - ch.bind(Channel.STOP) + Op.bind(dp, ch, Channel.STOP) } } } MultiMapOp apply() { - def events = new HashMap(2) - events.put('onNext', this.&doNext) - events.put('onComplete', this.&doComplete) - DataflowHelper.subscribeImpl(source, events) + new SubscribeOp() + .withSource(source) + .withOnNext(this.&doNext) + .withOnComplete(this.&doComplete) + .apply() return this } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/OperatorImpl.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/OperatorImpl.groovy index 3614de19db..86d074bf67 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/OperatorImpl.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/OperatorImpl.groovy @@ -28,14 +28,13 @@ import groovyx.gpars.dataflow.DataflowVariable import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.expression.DataflowExpression import groovyx.gpars.dataflow.operator.ChainWithClosure -import groovyx.gpars.dataflow.operator.ControlMessage -import groovyx.gpars.dataflow.operator.DataflowEventAdapter import groovyx.gpars.dataflow.operator.DataflowProcessor -import groovyx.gpars.dataflow.operator.PoisonPill import nextflow.Channel import nextflow.Global import nextflow.NF import nextflow.Session +import nextflow.extension.op.ContextRunPerThread +import nextflow.extension.op.Op import nextflow.script.ChannelOut import nextflow.script.TokenBranchDef import nextflow.script.TokenMultiMapDef @@ -44,7 +43,6 @@ import nextflow.splitter.FastqSplitter import nextflow.splitter.JsonSplitter import nextflow.splitter.TextSplitter import org.codehaus.groovy.runtime.callsite.BooleanReturningMethodInvoker -import org.codehaus.groovy.runtime.typehandling.DefaultTypeTransformation /** * A set of operators inspired to RxJava extending the methods available on DataflowChannel * data structure @@ -69,7 +67,10 @@ class OperatorImpl { * @return */ DataflowReadChannel subscribe(final DataflowReadChannel source, final Closure closure) { - subscribeImpl( source, [onNext: closure] ) + new SubscribeOp() + .withSource(source) + .withOnNext(closure) + .apply() return source } @@ -81,36 +82,13 @@ class OperatorImpl { * @return */ DataflowReadChannel subscribe(final DataflowReadChannel source, final Map events ) { - subscribeImpl(source, events) + new SubscribeOp() + .withSource(source) + .withEvents(events) + .apply() return source } - /** - * Chain operator, this is a synonym of {@code DataflowReadChannel.chainWith} - * - * @param source - * @param closure - * @return - */ - DataflowWriteChannel chain(final DataflowReadChannel source, final Closure closure) { - final target = CH.createBy(source) - newOperator(source, target, stopErrorListener(source,target), new ChainWithClosure(closure)) - return target - } - - /** - * Chain operator, this is a synonym of {@code DataflowReadChannel.chainWith} - * - * @param source - * @param closure - * @return - */ - DataflowWriteChannel chain(final DataflowReadChannel source, final Map params, final Closure closure) { - final target = CH.createBy(source) - chainImpl(source, target, params, closure) - return target; - } - /** * Transform the items emitted by a channel by applying a function to each of them * @@ -119,9 +97,10 @@ class OperatorImpl { * @return */ DataflowWriteChannel map(final DataflowReadChannel source, final Closure closure) { - assert source != null - assert closure - return new MapOp(source, closure).apply() + return new MapOp() + .withSource(source) + .withMapper(closure) + .apply() } /** @@ -133,42 +112,10 @@ class OperatorImpl { */ DataflowWriteChannel flatMap(final DataflowReadChannel source, final Closure closure=null) { assert source != null - - final target = CH.create() - final listener = stopErrorListener(source,target) - - newOperator(source, target, listener) { item -> - - def result = closure != null ? closure.call(item) : item - def proc = ((DataflowProcessor) getDelegate()) - - switch( result ) { - case Collection: - result.each { it -> proc.bindOutput(it) } - break - - case (Object[]): - result.each { it -> proc.bindOutput(it) } - break - - case Map: - result.each { it -> proc.bindOutput(it) } - break - - case Map.Entry: - proc.bindOutput( (result as Map.Entry).key ) - proc.bindOutput( (result as Map.Entry).value ) - break - - case Channel.VOID: - break - - default: - proc.bindOutput(result) - } - } - - return target + new FlatMapOp() + .withSource(source) + .withMapper(closure) + .apply() } /** @@ -188,9 +135,10 @@ class OperatorImpl { if( source instanceof DataflowExpression ) throw new IllegalArgumentException('Operator `reduce` cannot be applied to a value channel') - final target = new DataflowVariable() - reduceImpl( source, target, null, closure ) - return target + ReduceOp .create() + .withSource(source) + .withAction(closure) + .apply() } /** @@ -211,9 +159,11 @@ class OperatorImpl { if( source instanceof DataflowExpression ) throw new IllegalArgumentException('Operator `reduce` cannot be applied to a value channel') - final target = new DataflowVariable() - reduceImpl( source, target, seed, closure ) - return target + ReduceOp .create() + .withSource(source) + .withSeed(seed) + .withAction(closure) + .apply() } DataflowWriteChannel collectFile( final DataflowReadChannel source, final Closure closure = null ) { @@ -248,48 +198,23 @@ class OperatorImpl { * @return */ DataflowWriteChannel filter(final DataflowReadChannel source, final Object criteria) { - def discriminator = new BooleanReturningMethodInvoker("isCase"); - - def target = CH.createBy(source) - if( source instanceof DataflowExpression ) { - source.whenBound { - def result = it instanceof ControlMessage ? false : discriminator.invoke(criteria, (Object)it) - target.bind( result ? it : Channel.STOP ) - } - } - else { - newOperator(source, target, { - def result = discriminator.invoke(criteria, (Object)it) - if( result ) target.bind(it) - }) - } - - return target + return new FilterOp() + .withSource(source) + .withCriteria(criteria) + .apply() } DataflowWriteChannel filter(DataflowReadChannel source, final Closure closure) { - def target = CH.createBy(source) - if( source instanceof DataflowExpression ) { - source.whenBound { - def result = it instanceof ControlMessage ? false : DefaultTypeTransformation.castToBoolean(closure.call(it)) - target.bind( result ? it : Channel.STOP ) - } - } - else { - newOperator(source, target, { - def result = DefaultTypeTransformation.castToBoolean(closure.call(it)) - if( result ) target.bind(it) - }) - } - - return target + return new FilterOp() + .withSource(source) + .withCriteria(closure) + .apply() } DataflowWriteChannel until(DataflowReadChannel source, final Closure closure) { return new UntilOp(source,closure).apply() } - /** * Modifies this collection to remove all duplicated items, using the default comparator. * @@ -299,7 +224,7 @@ class OperatorImpl { * @return */ DataflowWriteChannel unique(final DataflowReadChannel source) { - unique(source) { it } + unique(source, Closure.IDENTITY) } /** @@ -311,41 +236,11 @@ class OperatorImpl { * @param comparator * @return */ - DataflowWriteChannel unique(final DataflowReadChannel source, Closure comparator ) { - - final history = [:] - final target = CH.createBy(source) - final stopOnFirst = source instanceof DataflowExpression - - // when the operator stop clear the history map - final events = new DataflowEventAdapter() { - void afterStop(final DataflowProcessor processor) { - history.clear() - } - } - - final filter = { - try { - final key = comparator.call(it) - if( history.containsKey(key) ) { - return Channel.VOID - } - else { - history.put(key,true) - return it - } - } - finally { - if( stopOnFirst ) { - ((DataflowProcessor) getDelegate()).terminate() - } - } - } as Closure - - // filter removing all duplicates - chainImpl(source, target, [listeners: [events]], filter ) - - return target + DataflowWriteChannel unique(final DataflowReadChannel source, final Closure comparator) { + return new UniqueOp() + .withSource(source) + .withComparator(comparator) + .apply() } /** @@ -357,7 +252,7 @@ class OperatorImpl { * @return */ DataflowWriteChannel distinct( final DataflowReadChannel source ) { - distinct(source) {it} + distinct(source, Closure.IDENTITY) } /** @@ -367,23 +262,11 @@ class OperatorImpl { * * @return */ - DataflowWriteChannel distinct( final DataflowReadChannel source, Closure comparator ) { - - def previous = null - final target = CH.createBy(source) - Closure filter = { it -> - - def key = comparator.call(it) - if( key == previous ) { - return Channel.VOID - } - previous = key - return it - } - - chainImpl(source, target, [:], filter) - - return target + DataflowWriteChannel distinct(final DataflowReadChannel source, final Closure comparator) { + new DistinctOp() + .withSource(source) + .withComparator(comparator) + .apply() } /** @@ -404,9 +287,7 @@ class OperatorImpl { log.warn msg } - def target = new DataflowVariable() - source.whenBound { target.bind(it) } - return target + return first(source, { true }) } /** @@ -418,27 +299,11 @@ class OperatorImpl { * @param source * @return */ - DataflowWriteChannel first( final DataflowReadChannel source, Object criteria ) { - - def target = new DataflowVariable() - def discriminator = new BooleanReturningMethodInvoker("isCase"); - - if( source instanceof DataflowExpression ) { - source.whenBound { - def result = it instanceof ControlMessage ? false : discriminator.invoke(criteria, it) - target.bind( result ? it : Channel.STOP ) - } - } - else { - newOperator([source],[]) { - if( discriminator.invoke(criteria, it) ) { - target.bind(it) - ((DataflowProcessor) getDelegate()).terminate() - } - } - } - - return target + DataflowWriteChannel first( final DataflowReadChannel source, final Object criteria ) { + new FirstOp() + .withSource(source) + .withCriteria(criteria) + .apply() } /** @@ -465,11 +330,7 @@ class OperatorImpl { * @return A {@code DataflowVariable} emitting the `last` item in the channel */ DataflowWriteChannel last( final DataflowReadChannel source ) { - - def target = new DataflowVariable() - def last = null - subscribeImpl( source, [onNext: { last = it }, onComplete: { target.bind(last) }] ) - return target + new LastOp().withSource(source).apply() } DataflowWriteChannel collect(final DataflowReadChannel source, Closure action=null) { @@ -477,11 +338,9 @@ class OperatorImpl { } DataflowWriteChannel collect(final DataflowReadChannel source, Map opts, Closure action=null) { - final target = new CollectOp(source,action,opts).apply() - return target + return new CollectOp(source,action,opts).apply() } - /** * Convert a {@code DataflowQueue} alias *channel* to a Java {@code List} * @@ -489,8 +348,7 @@ class OperatorImpl { * @return A list holding all the items send over the channel */ DataflowWriteChannel toList(final DataflowReadChannel source) { - final target = ToListOp.apply(source) - return target + return new ToListOp(source).apply() } /** @@ -500,8 +358,7 @@ class OperatorImpl { * @return A list holding all the items send over the channel */ DataflowWriteChannel toSortedList(final DataflowReadChannel source, Closure closure = null) { - final target = new ToListOp(source, closure ?: true).apply() - return target as DataflowVariable + return new ToListOp(source, closure ?: true).apply() } /** @@ -512,8 +369,7 @@ class OperatorImpl { * @return */ DataflowWriteChannel count(final DataflowReadChannel source ) { - final target = count0(source, null) - return target + return count0(source, null) } /** @@ -524,8 +380,7 @@ class OperatorImpl { * @return */ DataflowWriteChannel count(final DataflowReadChannel source, final Object criteria ) { - final target = count0(source, criteria) - return target + return count0(source, criteria) } private static DataflowVariable count0(DataflowReadChannel source, Object criteria) { @@ -533,17 +388,17 @@ class OperatorImpl { final target = new DataflowVariable() final discriminator = criteria != null ? new BooleanReturningMethodInvoker("isCase") : null - if( source instanceof DataflowExpression) { - source.whenBound { item -> - discriminator == null || discriminator.invoke(criteria, item) ? target.bind(1) : target.bind(0) - } - } - else { - reduceImpl(source, target, 0) { current, item -> - discriminator == null || discriminator.invoke(criteria, item) ? current+1 : current - } + final action = { current, item -> + discriminator == null || discriminator.invoke(criteria, item) ? current+1 : current } + ReduceOp .create() + .withSource(source) + .withTarget(target) + .withSeed(0) + .withAction(action) + .apply() + return target } @@ -554,9 +409,10 @@ class OperatorImpl { * @return A {@code DataflowVariable} returning the minimum value */ DataflowWriteChannel min(final DataflowReadChannel source) { - final target = new DataflowVariable() - reduceImpl(source, target, null) { min, val -> val val comparator.call(item) < comparator.call(min) ? item : min } } @@ -579,9 +435,10 @@ class OperatorImpl { action = (Closure){ a, b -> comparator.call(a,b) < 0 ? a : b } } - final target = new DataflowVariable() - reduceImpl(source, target, null, action) - return target + ReduceOp .create() + .withSource(source) + .withAction(action) + .apply() } /** @@ -592,9 +449,10 @@ class OperatorImpl { * @return A {@code DataflowVariable} returning the minimum value */ DataflowWriteChannel min(final DataflowReadChannel source, Comparator comparator) { - final target = new DataflowVariable() - reduceImpl(source, target, null) { a, b -> comparator.compare(a,b)<0 ? a : b } - return target + ReduceOp .create() + .withSource(source) + .withAction{ a, b -> comparator.compare(a,b)<0 ? a : b } + .apply() } /** @@ -604,9 +462,10 @@ class OperatorImpl { * @return A {@code DataflowVariable} emitting the maximum value */ DataflowWriteChannel max(final DataflowReadChannel source) { - final target = new DataflowVariable() - reduceImpl(source,target, null) { max, val -> val>max ? val : max } - return target + ReduceOp .create() + .withSource(source) + .withAction { max, val -> val>max ? val : max } + .apply() } /** @@ -632,9 +491,10 @@ class OperatorImpl { throw new IllegalArgumentException("Comparator closure can accept at most 2 arguments") } - final target = new DataflowVariable() - reduceImpl(source, target, null, action) - return target + ReduceOp .create() + .withSource(source) + .withAction(action) + .apply() } /** @@ -645,9 +505,10 @@ class OperatorImpl { * @return A {@code DataflowVariable} emitting the maximum value */ DataflowVariable max(final DataflowReadChannel source, Comparator comparator) { - final target = new DataflowVariable() - reduceImpl(source, target, null) { a, b -> comparator.compare(a,b)>0 ? a : b } - return target + ReduceOp .create() + .withSource(source) + .withAction { a, b -> comparator.compare(a,b)>0 ? a : b } + .apply() } /** @@ -658,61 +519,18 @@ class OperatorImpl { * @return A {@code DataflowVariable} emitting the final sum value */ DataflowWriteChannel sum(final DataflowReadChannel source, Closure closure = null) { - - def target = new DataflowVariable() - def aggregate = new Aggregate(name: 'sum', action: closure) - subscribeImpl(source, [onNext: aggregate.&process, onComplete: { target.bind( aggregate.result ) }]) - return target + return MathOp.sum() + .withSource(source) + .withAction(closure) + .apply() } DataflowWriteChannel mean(final DataflowReadChannel source, Closure closure = null) { - - def target = new DataflowVariable() - def aggregate = new Aggregate(name: 'mean', action: closure, mean: true) - subscribeImpl(source, [onNext: aggregate.&process, onComplete: { target.bind( aggregate.result ) }]) - return target - } - - - private static class Aggregate { - - def accum - long count = 0 - boolean mean - Closure action - String name - - def process(it) { - if( it == null || it == Channel.VOID ) - return - - count++ - - def item = action != null ? action.call(it) : it - if( accum == null ) - accum = item - - else if( accum instanceof Number ) - accum += item - - else if( accum instanceof List && item instanceof List) - for( int i=0; i messages) { - if( stopOnFirst ) - processor.terminate() - } - - @Override - void afterStop(final DataflowProcessor processor) { - processor.bindOutput(Channel.STOP) - } - - boolean onException(final DataflowProcessor processor, final Throwable e) { - OperatorImpl.log.error("@unknown", e) - session.abort(e) - return true; - } - } - - newOperator(inputs: [source], outputs: [target], listeners: listeners) { item -> - - def proc = ((DataflowProcessor) getDelegate()) - switch( item ) { - case Collection: - ((Collection)item).flatten().each { value -> proc.bindOutput(value) } - break - - case (Object[]): - ((Collection)item).flatten().each { value -> proc.bindOutput(value) } - break - - case Channel.VOID: - break - - default: - proc.bindOutput(item) - } - } - - return target + new FlattenOp() + .withSource(source) + .apply() } /** @@ -786,109 +562,48 @@ class OperatorImpl { * @return A newly created dataflow queue which emitted the gathered values as bundles */ DataflowWriteChannel buffer( final DataflowReadChannel source, Map params=null, Object closingCriteria ) { - - def target = new BufferOp(source) + return new BufferOp(source) .setParams(params) .setCloseCriteria(closingCriteria) .apply() - return target } DataflowWriteChannel buffer( final DataflowReadChannel source, Object startingCriteria, Object closingCriteria ) { assert startingCriteria != null assert closingCriteria != null - def target = new BufferOp(source) + return new BufferOp(source) .setStartCriteria(startingCriteria) .setCloseCriteria(closingCriteria) .apply() - - return target } DataflowWriteChannel buffer( DataflowReadChannel source, Map params ) { checkParams( 'buffer', params, 'size','skip','remainder' ) - - def target = new BufferOp(source) + return new BufferOp(source) .setParams(params) .apply() - - return target } - DataflowWriteChannel collate( DataflowReadChannel source, int size, boolean keepRemainder = true ) { if( size <= 0 ) { throw new IllegalArgumentException("Illegal argument 'size' for operator 'collate' -- it must be greater than zero: $size") } - def target = new BufferOp(source) - .setParams( size: size, remainder: keepRemainder ) - .apply() - - return target + new BufferOp(source) + .setParams( size: size, remainder: keepRemainder ) + .apply() } DataflowWriteChannel collate( DataflowReadChannel source, int size, int step, boolean keepRemainder = true ) { - if( size <= 0 ) { - throw new IllegalArgumentException("Illegal argument 'size' for operator 'collate' -- it must be greater than zero: $size") - } - - if( step <= 0 ) { - throw new IllegalArgumentException("Illegal argument 'step' for operator 'collate' -- it must be greater than zero: $step") - } - - // the result queue - final target = CH.create() - - // the list holding temporary collected elements - List> allBuffers = [] - - // -- intercepts the PoisonPill and sent out the items remaining in the buffer when the 'remainder' flag is true - def listener = new DataflowEventAdapter() { - - Object controlMessageArrived(final DataflowProcessor processor, final DataflowReadChannel channel, final int index, final Object message) { - if( message instanceof PoisonPill && keepRemainder && allBuffers.size() ) { - allBuffers.each { - target.bind( it ) - } - } - - return message; - } - - @Override - boolean onException(DataflowProcessor processor, Throwable e) { - OperatorImpl.log.error("@unknown", e) - session.abort(e) - return true - } - } - - - int index = 0 - - // -- the operator collecting the elements - newOperator( inputs: [source], outputs: [target], listeners: [listener]) { - - if( index++ % step == 0 ) { - allBuffers.add( [] ) - } - - allBuffers.each { List list -> list.add(it) } - - def buf = allBuffers.head() - if( buf.size() == size ) { - ((DataflowProcessor) getDelegate()).bindOutput(buf) - allBuffers = allBuffers.tail() - } - - } - - return target + new CollateOp() + .withSource(source) + .withSize(size) + .withStep(step) + .withRemainder(keepRemainder) + .apply() } - /** * Similar to https://github.com/Netflix/RxJava/wiki/Combining-Observables#merge * @@ -910,8 +625,7 @@ class OperatorImpl { // therefore the channel need to be added `'manually` to the inputs list // fixes #1346 OpCall.current.get().inputs.add(right) - def target = new JoinOp(left,right) .apply() - return target + return new JoinOp(left,right) .apply() } DataflowWriteChannel join( DataflowReadChannel left, Map opts, right ) { @@ -921,8 +635,7 @@ class OperatorImpl { // therefore the channel need to be added `'manually` to the inputs list // fixes #1346 OpCall.current.get().inputs.add(right) - def target = new JoinOp(left,right,opts) .apply() - return target + return new JoinOp(left,right,opts) .apply() } /** @@ -976,12 +689,9 @@ class OperatorImpl { } DataflowWriteChannel cross( DataflowReadChannel source, DataflowReadChannel other, Closure mapper = null ) { - - def target = new CrossOp(source, other) + return new CrossOp(source, other) .setMapper(mapper) .apply() - - return target } @@ -1018,12 +728,11 @@ class OperatorImpl { * @return The tap resulting dataflow channel */ DataflowWriteChannel tap( final DataflowReadChannel source, final Closure holder ) { - def tap = new TapOp(source, holder).apply() + final tap = new TapOp(source, holder).apply() OpCall.current.get().outputs.addAll( tap.outputs ) return tap.result } - /** * Empty the specified value only if the source channel to which is applied is empty i.e. do not emit * any value. @@ -1037,15 +746,20 @@ class OperatorImpl { boolean empty = true final result = CH.createBy(source) final singleton = result instanceof DataflowExpression - final next = { result.bind(it); empty=false } - final complete = { - if(empty) - result.bind( value instanceof Closure ? value() : value ) - if( !singleton ) - result.bind(Channel.STOP) - } - subscribeImpl(source, [onNext: next, onComplete: complete]) + new SubscribeOp() + .withSource(source) + .withContext(new ContextRunPerThread()) + .withOnNext { DataflowProcessor dp, Object it -> Op.bind(dp,result,it); empty=false } + .withOnComplete { DataflowProcessor dp -> + if(empty) { + final x = value instanceof Closure ? value.call() : value + Op.bind(dp,result,x) + } + if( !singleton ) + result.bind(Channel.STOP) + } + .apply() return result } @@ -1065,17 +779,16 @@ class OperatorImpl { final newLine = opts.newLine != false final target = CH.createBy(source); - final apply = new HashMap(2) - apply.onNext = { - final obj = closure != null ? closure.call(it) : it - session.printConsole(obj?.toString(), newLine) - target.bind(it) - } - - apply. onComplete = { CH.close0(target) } - - subscribeImpl(source,apply) + new SubscribeOp() + .withSource(source) + .withOnNext{ + final obj = closure != null ? closure.call(it) : it + session.printConsole(obj?.toString(), newLine) + target.bind(it) + } + .withOnComplete{ CH.close0(target) } + .apply() return target } @@ -1108,34 +821,43 @@ class OperatorImpl { if( source instanceof DataflowExpression ) throw new IllegalArgumentException("Operator `randomSample` cannot be applied to a value channel") - final result = new RandomSampleOp(source,n, seed).apply() - return result + return new RandomSampleOp(source,n, seed).apply() } DataflowWriteChannel toInteger(final DataflowReadChannel source) { - return chain(source, { it -> it as Integer }) + return new MapOp() + .withSource(source) + .withMapper { it -> it as Integer } + .apply() } DataflowWriteChannel toLong(final DataflowReadChannel source) { - return chain(source, { it -> it as Long }) + return new MapOp() + .withSource(source) + .withMapper { it -> it as Long } + .apply() } DataflowWriteChannel toFloat(final DataflowReadChannel source) { - return chain(source, { it -> it as Float }) + return new MapOp() + .withSource(source) + .withMapper { it -> it as Float } + .apply() } DataflowWriteChannel toDouble(final DataflowReadChannel source) { - return chain(source, { it -> it as Double }) + return new MapOp() + .withSource(source) + .withMapper { it -> it as Double } + .apply() } DataflowWriteChannel transpose( final DataflowReadChannel source, final Map params=null ) { - def result = new TransposeOp(source,params).apply() - return result + return new TransposeOp(source,params).apply() } DataflowWriteChannel splitText(DataflowReadChannel source, Map opts=null) { - final result = new SplitOp( source, 'splitText', opts ).apply() - return result + return new SplitOp( source, 'splitText', opts ).apply() } DataflowWriteChannel splitText(DataflowReadChannel source, Map opts=null, Closure action) { @@ -1148,23 +870,19 @@ class OperatorImpl { } DataflowWriteChannel splitCsv(DataflowReadChannel source, Map opts=null) { - final result = new SplitOp( source, 'splitCsv', opts ).apply() - return result + return new SplitOp( source, 'splitCsv', opts ).apply() } DataflowWriteChannel splitFasta(DataflowReadChannel source, Map opts=null) { - final result = new SplitOp( source, 'splitFasta', opts ).apply() - return result + return new SplitOp( source, 'splitFasta', opts ).apply() } DataflowWriteChannel splitFastq(DataflowReadChannel source, Map opts=null) { - final result = new SplitOp( source, 'splitFastq', opts ).apply() - return result + return new SplitOp( source, 'splitFastq', opts ).apply() } DataflowWriteChannel splitJson(DataflowReadChannel source, Map opts=null) { - final result = new SplitOp( source, 'splitJson', opts ).apply() - return result + return new SplitOp( source, 'splitJson', opts ).apply() } DataflowWriteChannel countLines(DataflowReadChannel source, Map opts=null) { diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/PhaseOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/PhaseOp.groovy index 965a8247eb..dae8ea981d 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/PhaseOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/PhaseOp.groovy @@ -84,27 +84,30 @@ class PhaseOp { * @return A map with {@code OnNext} and {@code onComplete} methods entries */ static private final Map phaseHandler( Map> buffer, int size, int index, DataflowWriteChannel target, Closure mapper, AtomicInteger stopCount, boolean remainder ) { - - DataflowHelper.eventsMap( - { - synchronized (buffer) { - def entries = phaseImpl(buffer, size, index, it, mapper, false) - if( entries ) { - target.bind(entries) - } - }}, - - { - if( stopCount.decrementAndGet()==0) { - if( remainder ) - phaseRemainder(buffer,size, target) - target << Channel.STOP - }} - + eventsMap( + { + synchronized (buffer) { + def entries = phaseImpl(buffer, size, index, it, mapper, false) + if( entries ) { + target.bind(entries) + } + }}, + + { + if( stopCount.decrementAndGet()==0) { + if( remainder ) + phaseRemainder(buffer,size, target) + target << Channel.STOP + }} ) - } + static private Map eventsMap(Closure onNext, Closure onComplete) { + def result = new HashMap(2) + result.put('onNext', onNext) + result.put('onComplete', onComplete) + return result + } /** * Implements the phase operator logic. Basically buffers the values received on each channel by their key . @@ -184,7 +187,6 @@ class PhaseOp { return result } - static private final void phaseRemainder( Map> buffers, int count, DataflowWriteChannel target ) { Collection> slots = buffers.values() diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/PublishOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/PublishOp.groovy index 3f10b7c16c..f1b3a1967a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/PublishOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/PublishOp.groovy @@ -18,7 +18,6 @@ package nextflow.extension import java.nio.file.Path -import groovy.json.JsonOutput import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel @@ -64,10 +63,11 @@ class PublishOp { boolean getComplete() { complete } PublishOp apply() { - final events = new HashMap(2) - events.onNext = this.&onNext - events.onComplete = this.&onComplete - DataflowHelper.subscribeImpl(source, events) + new SubscribeOp() + .withSource(source) + .withOnNext(this.&onNext) + .withOnComplete(this.&onComplete) + .apply() return this } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/RandomSampleOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/RandomSampleOp.groovy index 97c58ee2da..7b4143e0f4 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/RandomSampleOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/RandomSampleOp.groovy @@ -20,10 +20,6 @@ import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel import nextflow.Channel - -import static DataflowHelper.eventsMap -import static DataflowHelper.subscribeImpl - /** * Implements Reservoir sampling of channel content * @@ -80,7 +76,11 @@ class RandomSampleOp { DataflowWriteChannel apply() { result = CH.create() - subscribeImpl(source, eventsMap(this.&sampling, this.&emit)) + new SubscribeOp() + .withSource(source) + .withOnNext(this.&sampling) + .withOnComplete(this.&emit) + .apply() return result } } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/ReduceOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/ReduceOp.groovy new file mode 100644 index 0000000000..ff1441a1e8 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/ReduceOp.groovy @@ -0,0 +1,136 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowVariable +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Channel +import nextflow.Global +import nextflow.Session +import nextflow.extension.op.ContextGrouping +import nextflow.extension.op.Op +/** + * Implements reduce operator logic + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +@CompileStatic +class ReduceOp { + + private DataflowReadChannel source + private DataflowVariable target + private Object seed + private Closure action + private Closure beforeBind + + private ReduceOp() { } + + ReduceOp(DataflowReadChannel source) { + this.source = source + } + + static ReduceOp create() { + new ReduceOp() + } + + ReduceOp withSource(DataflowReadChannel source) { + this.source = source + return this + } + + ReduceOp withTarget(DataflowVariable target) { + this.target = target + return this + } + + ReduceOp withSeed(Object seed) { + this.seed = seed + return this + } + + ReduceOp withAction(Closure action) { + this.action = action + return this + } + + ReduceOp withBeforeBind(Closure beforeBind) { + this.beforeBind = beforeBind + return this + } + + private Session getSession() { + return Global.session as Session + } + + DataflowVariable apply() { + if( source==null ) + throw new IllegalArgumentException("Missing reduce operator source channel") + if( target==null ) + target = new DataflowVariable() + final stopOnFirst = source instanceof DataflowExpression + // the *accumulator* value + def accum = this.seed + + // intercepts operator events + final listener = new DataflowEventAdapter() { + /* + * when terminates bind the result value + */ + void afterStop(final DataflowProcessor dp) { + final result = beforeBind + ? beforeBind.call(accum) + : accum + Op.bind(dp, target, result) + } + + boolean onException(final DataflowProcessor dp, final Throwable e) { + log.error("@unknown", e) + session.abort(e) + return true; + } + } + + final code = { + final value = accum == null ? it : action.call(accum, it) + final proc = getDelegate() as DataflowProcessor + if( value!=Channel.VOID && value!=Channel.STOP ) { + accum = value + } + if( stopOnFirst || value==Channel.STOP ) + proc.terminate() + } + + new Op() + .withInput(source) + .withContext(new ContextGrouping()) + .withListener(listener) + .withCode(code) + .apply() + + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/SplitOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/SplitOp.groovy index db0e58462e..4c6b99e685 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/SplitOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/SplitOp.groovy @@ -186,10 +186,11 @@ class SplitOp { @PackageScope void applySplittingOperator( DataflowReadChannel origin, DataflowWriteChannel output, AbstractSplitter splitter ) { - final events = new HashMap(2) - events.onNext = { entry -> splitter.target(entry).apply() } - events.onComplete = { output << Channel.STOP } - DataflowHelper.subscribeImpl ( origin, events ) + new SubscribeOp() + .withSource(origin) + .withOnNext({ entry -> splitter.target(entry).apply() }) + .withOnComplete({ output << Channel.STOP }) + .apply() } @PackageScope diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/SubscribeOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/SubscribeOp.groovy new file mode 100644 index 0000000000..1a91a76a30 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/SubscribeOp.groovy @@ -0,0 +1,141 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.Global +import nextflow.Session +import nextflow.extension.op.Op +import nextflow.extension.op.OpContext +/** + * Implements the "subscribe" operator + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class SubscribeOp { + + static private List VALID_HANDLERS = [ 'onNext', 'onComplete', 'onError' ] + + private DataflowReadChannel source + private OpContext context + private Closure onNext + private Closure onComplete + private Closure onError + + private static Session getSession() { Global.getSession() as Session } + + SubscribeOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + SubscribeOp withOnNext(Closure event) { + this.onNext = event + return this + } + + SubscribeOp withOnComplete(Closure event) { + this.onComplete = event + return this + } + + SubscribeOp withOnError(Closure event) { + this.onError = event + return this + } + + SubscribeOp withEvents(Map events) { + if( events ) { + events.keySet().each { + if( !VALID_HANDLERS.contains(it) ) throw new IllegalArgumentException("Not a valid handler name: $it") + } + this.onNext = events.onNext as Closure + this.onComplete = events.onComplete as Closure + this.onError = events.onError as Closure + } + return this + } + + SubscribeOp withContext(OpContext context) { + if( context!=null ) + this.context = context + return this + } + + DataflowProcessor apply() { + def error = false + def stopOnFirst = source instanceof DataflowExpression + def listener = new DataflowEventAdapter() { + + @Override + void afterStop(final DataflowProcessor dp) { + if( !onComplete || error ) return + try { + onComplete.call(dp) + } + catch( Exception e ) { + SubscribeOp.log.error("@unknown", e) + session.abort(e) + } + } + + @Override + boolean onException(final DataflowProcessor dp, final Throwable e) { + error = true + if( !onError ) { + log.error("@unknown", e) + session.abort(e) + } + else { + onError.call(e) + } + return true + } + } + + final code = { + final proc = getDelegate() as DataflowProcessor + if( onNext instanceof Closure ) { + final action = (Closure) onNext + final types = action.getParameterTypes() + types.size()==2 && types[0]==DataflowProcessor.class + ? action.call(proc, it) + : action.call(it) + } + if( stopOnFirst ) { + proc.terminate() + } + } + + new Op() + .withInput(source) + .withListener(listener) + .withContext(context) + .withCode(code) + .apply() + + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/TakeOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/TakeOp.groovy index f7bf259d0a..5924c2042f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/TakeOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/TakeOp.groovy @@ -17,21 +17,19 @@ package nextflow.extension -import static nextflow.extension.DataflowHelper.* import groovy.transform.CompileStatic import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel -import groovyx.gpars.dataflow.operator.ChainWithClosure -import groovyx.gpars.dataflow.operator.CopyChannelsClosure import groovyx.gpars.dataflow.operator.DataflowEventAdapter import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel import nextflow.Global import nextflow.Session - +import nextflow.extension.op.Op /** + * Implement "take" operator * * @author Paolo Di Tommaso */ @@ -59,25 +57,29 @@ class TakeOp { final listener = new DataflowEventAdapter() { @Override - void afterRun(final DataflowProcessor processor, final List messages) { + void afterRun(final DataflowProcessor dp, final List messages) { if( ++count >= length ) { - processor.bindOutput( Channel.STOP ) - processor.terminate() + dp.bindOutput( Channel.STOP ) + dp.terminate() } } - boolean onException(final DataflowProcessor processor, final Throwable e) { + boolean onException(final DataflowProcessor dp, final Throwable e) { TakeOp.log.error("@unknown", e) (Global.session as Session).abort(e) return true; } } - newOperator( - inputs: [source], - outputs: [target], - listeners: (length > 0 ? [listener] : []), - new ChainWithClosure(new CopyChannelsClosure())) + new Op() + .withInput(source) + .withOutput(target) + .withListener(length>0 ? listener : null) + .withCode { + final proc = getDelegate() as DataflowProcessor + Op.bind(proc, target, it) + } + .apply() return target } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/ToListOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/ToListOp.groovy index 163415b3f2..5222a8fb40 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/ToListOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/ToListOp.groovy @@ -20,6 +20,8 @@ import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.extension.op.Op /** * Implements {@link OperatorImpl#toList(groovyx.gpars.dataflow.DataflowReadChannel)} operator @@ -50,21 +52,25 @@ class ToListOp { final target = new DataflowVariable() if( source instanceof DataflowExpression ) { final result = new ArrayList(1) - Map events = [:] - events.onNext = { result.add(it) } - events.onComplete = { target.bind(result) } - DataflowHelper.subscribeImpl(source, events ) + new SubscribeOp() + .withSource(source) + .withOnNext({ result.add(it) }) + .withOnComplete({ DataflowProcessor dp -> Op.bind(dp, target, result) }) + .apply() return target } - DataflowHelper.reduceImpl(source, target, []) { List list, item -> list << item } - if( ordering ) { - final sort = { List list -> ordering instanceof Closure ? list.sort((Closure) ordering) : list.sort() } - return (DataflowVariable)target.then(sort) - } - else { - return target - } + Closure beforeBind = ordering + ? { List list -> ordering instanceof Closure ? list.sort((Closure) ordering) : list.sort() } + : null + + ReduceOp .create() + .withSource(source) + .withTarget(target) + .withSeed(new ArrayList()) + .withBeforeBind(beforeBind) + .withAction{ List list, item -> list << item } + .apply() } @Deprecated diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/TransposeOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/TransposeOp.groovy index e9c26a61f8..9d3a71812c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/TransposeOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/TransposeOp.groovy @@ -64,7 +64,11 @@ class TransposeOp { } DataflowWriteChannel apply() { - DataflowHelper.subscribeImpl(source, DataflowHelper.eventsMap(this.&transpose, this.&done)) + new SubscribeOp() + .withSource(source) + .withOnNext(this.&transpose) + .withOnComplete(this.&done) + .apply() return target } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/UniqueOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/UniqueOp.groovy new file mode 100644 index 0000000000..d5d632e59f --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/UniqueOp.groovy @@ -0,0 +1,103 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension + +import groovy.transform.CompileStatic +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.expression.DataflowExpression +import groovyx.gpars.dataflow.operator.DataflowEventAdapter +import groovyx.gpars.dataflow.operator.DataflowProcessor +import nextflow.extension.op.Op + +/** + * Implements the "unique" operator logic + * + * @author Paolo Di Tommaso + */ +@CompileStatic +class UniqueOp { + + private DataflowReadChannel source + private DataflowWriteChannel target + private Closure comparator + + UniqueOp() {} + + UniqueOp withSource(DataflowReadChannel source) { + assert source!=null + this.source = source + return this + } + + UniqueOp withTarget(DataflowWriteChannel target) { + assert target!=null + this.target = target + return this + } + + UniqueOp withComparator(Closure comparator) { + assert comparator!=null + this.comparator = comparator + return this + } + + DataflowWriteChannel apply() { + assert source != null + assert comparator != null + + if( !target ) + target = CH.createBy(source) + + final history = new LinkedHashMap() + final stopOnFirst = source instanceof DataflowExpression + + // when the operator stop clear the history map + final listener = new DataflowEventAdapter() { + void afterStop(final DataflowProcessor dp) { + history.clear() + } + } + + final code = { + final proc = getDelegate() as DataflowProcessor + try { + final key = comparator.call(it) + if( !history.containsKey(key) ) { + history.put(key,true) + Op.bind(proc, target, it) + } + } + finally { + if( stopOnFirst ) { + proc.terminate() + } + } + } + + new Op() + .withInput(source) + .withOutput(target) + .withListener(listener) + .withCode(code) + .apply() + + return target + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/UntilManyOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/UntilManyOp.groovy index 7ec5e49dd2..42f05d8ca6 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/UntilManyOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/UntilManyOp.groovy @@ -103,7 +103,7 @@ class UntilManyOp { @Override Object call(final Object arguments) { - throw new UnsupportedOperationException() + call(InvokerHelper.asArray(arguments)) } @Override diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/UntilOp.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/UntilOp.groovy index cb9ebefc6b..071131f635 100644 --- a/modules/nextflow/src/main/groovy/nextflow/extension/UntilOp.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/extension/UntilOp.groovy @@ -17,18 +17,19 @@ package nextflow.extension -import static nextflow.extension.DataflowHelper.* - +import groovy.transform.CompileStatic import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowWriteChannel import groovyx.gpars.dataflow.operator.DataflowProcessor import nextflow.Channel +import nextflow.extension.op.Op import org.codehaus.groovy.runtime.typehandling.DefaultTypeTransformation /** * Implements Nextflow `until` operator * * @author Paolo Di Tommaso */ +@CompileStatic class UntilOp { private DataflowReadChannel source @@ -41,19 +42,22 @@ class UntilOp { DataflowWriteChannel apply() { final target = CH.createBy(source) - - newOperator(source, target, { - final result = DefaultTypeTransformation.castToBoolean(closure.call(it)) - final proc = ((DataflowProcessor) getDelegate()) - - if( result ) { - proc.bindOutput(Channel.STOP) - proc.terminate() - } - else { - proc.bindOutput(it) + + new Op() + .withInput(source) + .withOutput(target) + .withCode { + final result = DefaultTypeTransformation.castToBoolean(closure.call(it)) + final dp = getDelegate() as DataflowProcessor + if( result ) { + Op.bind(dp, target, Channel.STOP) + dp.terminate() + } + else { + Op.bind(dp, target, it) + } } - }) + .apply() return target } diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextGrouping.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextGrouping.groovy new file mode 100644 index 0000000000..fa65aae4b8 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextGrouping.groovy @@ -0,0 +1,55 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import java.util.concurrent.ConcurrentHashMap + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.prov.OperatorRun + +/** + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class ContextGrouping implements OpContext { + + private final Map holder = new ConcurrentHashMap<>(1); + + ContextGrouping(){ + holder.put('run', new OperatorRun()) + } + + @Override + OperatorRun allocateRun() { + final result = holder.get('run') + log.trace "+ AllocateRun=$result" + return result + } + + @Override + OperatorRun getOperatorRun() { + final result = holder.get('run') + log.trace "+ GetOperatorRun=$result" + holder.put('run', new OperatorRun()) + return result + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextRunPerThread.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextRunPerThread.groovy new file mode 100644 index 0000000000..ea23d14ea1 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextRunPerThread.groovy @@ -0,0 +1,49 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.prov.OperatorRun +/** + * Implements an operator context that binds a new run to the current thread + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class ContextRunPerThread implements OpContext { + + private final ThreadLocal runs = ThreadLocal.withInitial(()->new OperatorRun()) + + @Override + synchronized OperatorRun allocateRun() { + final run = runs.get() + log.trace "+ AllocateRun run=$run" + return run + } + + @Override + synchronized OperatorRun getOperatorRun() { + final run = runs.get() + log.trace "+ GetOperatorRun run=$run" + runs.remove() + return run + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextSequential.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextSequential.groovy new file mode 100644 index 0000000000..343bc1441a --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/ContextSequential.groovy @@ -0,0 +1,46 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import java.util.concurrent.ConcurrentHashMap + +import nextflow.prov.OperatorRun + +/** + * Implements an operator context that expected a new run is allocated + * "sequentially" after the previous execution + * + * @author Paolo Di Tommaso + */ +class ContextSequential implements OpContext { + + private final Map holder = new ConcurrentHashMap<>(1) + + @Override + OperatorRun allocateRun() { + final result = new OperatorRun() + holder.put('run', result) + return result + } + + @Override + OperatorRun getOperatorRun() { + final result = holder.get('run') + return result + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/Op.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/Op.groovy new file mode 100644 index 0000000000..a4bed06c08 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/Op.groovy @@ -0,0 +1,310 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import java.util.concurrent.ConcurrentHashMap + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.Dataflow +import groovyx.gpars.dataflow.DataflowReadChannel +import groovyx.gpars.dataflow.DataflowWriteChannel +import groovyx.gpars.dataflow.operator.DataflowEventListener +import groovyx.gpars.dataflow.operator.DataflowOperator +import groovyx.gpars.dataflow.operator.DataflowProcessor +import groovyx.gpars.dataflow.operator.PoisonPill +import nextflow.Global +import nextflow.Session +import nextflow.dag.NodeMarker +import nextflow.prov.OperatorRun +import nextflow.prov.Prov +import nextflow.prov.Tracker +/** + * Model a dataflow operator + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class Op { + + /** + * Associate a {@link DataflowProcessor} with the corresponding {@link OpContext} object + * tracking running operator context + */ + static final public ConcurrentHashMap allContexts = new ConcurrentHashMap<>(); + + static List unwrap(List messages) { + final ArrayList result = new ArrayList<>(); + for( Object it : messages ) { + result.add(it instanceof Tracker.Msg ? ((Tracker.Msg)it).value : it); + } + return result; + } + + static Object unwrap(Object it) { + return it instanceof Tracker.Msg ? it.value : it + } + + static Tracker.Msg wrap(Object obj) { + obj instanceof Tracker.Msg ? obj : Tracker.Msg.of(obj) + } + + static void bind(DataflowProcessor dp, DataflowWriteChannel channel, Object msg) { + try { + if( msg instanceof PoisonPill ) { + channel.bind(msg) + allContexts.remove(dp) + } + else { + final ctx = allContexts.get(dp) + if( !ctx ) + throw new IllegalStateException("Cannot find any context for operator=$dp") + final run = ctx.getOperatorRun() + Prov.getTracker().bindOutput(run, channel, msg) + } + } + catch (Throwable t) { + log.error("Unexpected resolving execution provenance: ${t.message}", t) + (Global.session as Session).abort(t) + } + } + + /** + * This method gets a list of one or more {@link OpDatum} objects, and binds + * the corresponding values to the specified channel using a new {@link OperatorRun} + * instance using as {@link OperatorRun#inputIds} the set of IDs obtained by the + * provide "entries" lists. + * + * @param target + * The dataflow channel where outputs are going to be bound. + * @param entries + * A list of one or more {@link OpDatum} holding the value to be bound. + * @param singleton + * When {@code true} and the entries list is provided, the value object is emitted + * in place of a list with a sole object, when {@code false} the list of values + * is emitted in any case. + */ + static void bindRunValues(DataflowWriteChannel target, List entries, boolean singleton) { + final inputs = new ArrayList(entries.size()) + final values = new ArrayList(entries.size()) + for( Object it : entries ) { + if( it instanceof OpDatum ) { + inputs.addAll(it.run.inputIds) + values.add(it.value) + } + else + values.add(it) + } + final run = new OperatorRun(new LinkedHashSet(inputs)) + final out = singleton && values.size()==1 ? values[0] : values + Op.bind(run, target, out) + } + + static void bind(OperatorRun run, DataflowWriteChannel channel, Object msg) { + Prov.getTracker().bindOutput(run, channel, msg) + } + + private static Session getSession() { Global.getSession() as Session } + + private List inputs + private List outputs + private List listeners + private OpContext context = new ContextSequential() + private Closure code + + List getInputs() { inputs } + List getOutputs() { outputs } + List getListeners() { listeners } + OpContext getContext() { context } + Closure getCode() { code } + + /** + * Set the operator input channel + * + * @param channel + * The channel from which the operator is receiving the input data. + * @return + * The operator object itself. + */ + Op withInput(DataflowReadChannel channel) { + assert channel != null + this.inputs = List.of(channel) + return this + } + + /** + * Set the operator input channels + * + * @param channels + * The channels from which the operator is receiving the input data + * @return + * The operator object itself. + */ + Op withInputs(List channels) { + assert channels != null + this.inputs = channels + return this + } + + /** + * Set the operator output channel + * + * @param channel + * The channel to which the operator is binding output data. + * @return + * The operator object itself. + */ + Op withOutput(DataflowWriteChannel channel) { + assert channel != null + this.outputs = List.of(channel) + return this + } + + /** + * Set the operator output channels + * + * @param channels + * The channels to which the operator is binding output data. + * @return + * The operator object itself. + */ + Op withOutputs(List channels) { + assert channels != null + this.outputs = channels + return this + } + + /** + * Set a {@link DataflowEventListener} associated with the operator. + * + * @param listener + * The {@link DataflowEventListener} listener object. + * @return + * The operator object itself. + */ + Op withListener(DataflowEventListener listener) { + if( listener ) + this.listeners = List.of(listener) + return this + } + + /** + * Set one or more {@link DataflowEventListener} objects associated with the operator. + * + * @param listeners + * The list of {@link DataflowEventListener} listener objects. + * @return + * The operator object itself. + */ + Op withListeners(List listeners) { + if( listeners ) + this.listeners = listeners + return this + } + + /** + * Set the operator inputs, outputs and listeners parameters using a map object. + * + * @param params + * A {@link Map} object holding the {@code inputs} and {@code outputs} channels + * and {@code listeners} objects. + * @return + * The operator object itself. + */ + Op withParams(Map params) { + if( params.inputs ) + this.inputs = params.inputs as List + if( params.outputs ) + this.outputs = params.outputs as List + if( params.listeners ) + this.listeners = params.listeners as List + return this + } + + /** + * Set the context object associated associated with the operator run. + * + * @param context + * The {@link OpContext} object associated with the operator execution. + * @return + * The operator object itself. + */ + Op withContext(OpContext context) { + if( context!=null ) + this.context = context + return this + } + + /** + * Set the operator action as a closure + * + * @param code + * The action that should be performed when the operator run. + * @return + * The operator object itself. + * + */ + Op withCode(Closure code) { + this.code = code + return this + } + + /** + * Map the operator inputs, outputs and listeners to a Java {@Link Map} + * for compatibility with {@link DataflowProcessor} API + * + * @return + * A map object holding the operator {@code inputs}, {@code outputs} channels + * and the {@code listeners} object. + */ + protected Map toMap() { + final ret = new HashMap() + ret.inputs = inputs ?: List.of() + ret.outputs = outputs ?: List.of() + ret.listeners = listeners ?: List.of() + return ret + } + + /** + * Creates the {@link DataflowOperator} object and start it + * + * @return + * The corresponding {@link DataflowOperator} object + */ + DataflowProcessor apply() { + assert inputs + assert code + assert context + + // Encapsulate the target "code" closure with a "OpClosure" object + // to grab input data and track the execution provenance + final closure = new OpClosure(code, context) + final group = Dataflow.retrieveCurrentDFPGroup() + final operator = new DataflowOperator(group, toMap(), closure) + allContexts.put(operator, context) + operator.start() + // track the operator as dag node + NodeMarker.appendOperator(operator) + if( session && session.allOperators != null ) { + session.allOperators << operator + } + return operator + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/OpClosure.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpClosure.groovy new file mode 100644 index 0000000000..77446ec6b3 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpClosure.groovy @@ -0,0 +1,107 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.prov.OperatorRun +import nextflow.prov.Prov +import org.codehaus.groovy.runtime.InvokerHelper +/** + * A closure that wraps the execution of an operator target code + * associating the inputs and outputs to the corresponding operator run. + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class OpClosure extends Closure { + + private final Closure target + private final OpContext context + + OpClosure(Closure code, OpContext context) { + super(code.getOwner(), code.getThisObject()) + this.target = code + this.target.setDelegate(code.getDelegate()) + this.target.setResolveStrategy(code.getResolveStrategy()) + this.context = context + } + + @Override + Class[] getParameterTypes() { + return target.getParameterTypes() + } + + @Override + int getMaximumNumberOfParameters() { + return target.getMaximumNumberOfParameters() + } + + @Override + Object getDelegate() { + return target.getDelegate() + } + + @Override + int getDirective() { + return target.getDirective() + } + + @Override + void setDelegate(Object delegate) { + target.setDelegate(delegate) + } + + @Override + void setDirective(int directive) { + target.setDirective(directive) + } + + @Override + void setResolveStrategy(int resolveStrategy) { + target.setResolveStrategy(resolveStrategy) + } + + @Override + void setProperty(String propertyName, Object newValue) { + target.setProperty(propertyName, newValue) + } + + @Override + Object call(final Object... args) { + // when the accumulator flag true, re-use the previous run object + final OperatorRun run = context.allocateRun() + // map the inputs + final List inputs = Prov.getTracker().receiveInputs(run, Arrays.asList(args)) + final Object result = InvokerHelper.invokeMethod(target, "call", inputs.toArray()) + // return the operation result + return result + } + + @Override + Object call(Object args) { + return call(InvokerHelper.asArray(args)) + } + + @Override + Object call() { + return call(InvokerHelper.EMPTY_ARGS) + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/OpContext.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpContext.groovy new file mode 100644 index 0000000000..92445064e4 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpContext.groovy @@ -0,0 +1,30 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import nextflow.prov.OperatorRun + +/** + * Model an operator run context + * + * @author Paolo Di Tommaso + */ +interface OpContext { + OperatorRun allocateRun() + OperatorRun getOperatorRun() +} diff --git a/modules/nextflow/src/main/groovy/nextflow/extension/op/OpDatum.groovy b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpDatum.groovy new file mode 100644 index 0000000000..b022ba27db --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/extension/op/OpDatum.groovy @@ -0,0 +1,50 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import groovy.transform.Canonical +import nextflow.prov.OperatorRun + +/** + * Associated a data value acquired by an operator with the corresponding + * {@link OperatorRun} instance. + * + * @author Paolo Di Tommaso + */ +@Canonical +class OpDatum { + Object value + OperatorRun run + + static OpDatum of(Object value, OperatorRun run) { + new OpDatum(value,run) + } + + static Object unwrap(Object obj, List inputs=null) { + if( obj instanceof Collection ) { + return obj.collect(it-> unwrap(it,inputs)) + } + if( obj instanceof OpDatum ) { + if(inputs!=null) + inputs.addAll(obj.run.inputIds) + return obj.value + } + else + return obj + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy index 5d4175aeff..9c5698d611 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskBean.groovy @@ -36,6 +36,10 @@ class TaskBean implements Serializable, Cloneable { String name + TaskId taskId + + Set upstreamTasks + def input def scratch @@ -122,6 +126,8 @@ class TaskBean implements Serializable, Cloneable { TaskBean(TaskRun task) { this.name = task.name + this.taskId = task.id + this.upstreamTasks = task.upstreamTasks // set the input (when available) this.input = task.stdin diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskId.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskId.groovy index 3576b9fde7..bbb467ca5f 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskId.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskId.groovy @@ -19,6 +19,8 @@ package nextflow.processor import java.util.concurrent.atomic.AtomicInteger import groovy.transform.CompileStatic +import nextflow.util.TestOnly + /** * TaskRun unique identifier * @@ -32,12 +34,16 @@ class TaskId extends Number implements Comparable, Serializable, Cloneable { */ static final private AtomicInteger allCount = new AtomicInteger() + @TestOnly static void clear() { allCount.set(0) } + static TaskId next() { new TaskId(allCount.incrementAndGet()) } private final int value + int getValue() { value } + static TaskId of( value ) { if( value instanceof Integer ) return new TaskId(value) diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy index f17e905f4e..a530e24f89 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskProcessor.groovy @@ -15,8 +15,6 @@ */ package nextflow.processor -import nextflow.trace.TraceRecord - import static nextflow.processor.ErrorStrategy.* import java.lang.reflect.InvocationTargetException @@ -35,6 +33,7 @@ import java.util.regex.Pattern import ch.artecat.grengine.Grengine import com.google.common.hash.HashCode import groovy.json.JsonOutput +import groovy.transform.Canonical import groovy.transform.CompileStatic import groovy.transform.Memoized import groovy.transform.PackageScope @@ -82,6 +81,7 @@ import nextflow.file.FilePatternSplitter import nextflow.file.FilePorter import nextflow.plugin.Plugins import nextflow.processor.tip.TaskTipProvider +import nextflow.prov.Prov import nextflow.script.BaseScript import nextflow.script.BodyDef import nextflow.script.ProcessConfig @@ -106,6 +106,7 @@ import nextflow.script.params.TupleInParam import nextflow.script.params.TupleOutParam import nextflow.script.params.ValueInParam import nextflow.script.params.ValueOutParam +import nextflow.trace.TraceRecord import nextflow.util.ArrayBag import nextflow.util.BlankSeparatedList import nextflow.util.CacheHelper @@ -133,6 +134,11 @@ class TaskProcessor { RunType(String str) { message=str }; } + @Canonical + static class FairEntry { + Map emissions + TaskRun task + } static final public String TASK_CONTEXT_PROPERTY_NAME = 'task' final private static Pattern ENV_VAR_NAME = ~/[a-zA-Z_]+[a-zA-Z0-9_]*/ @@ -143,6 +149,8 @@ class TaskProcessor { @TestOnly static TaskProcessor currentProcessor() { currentProcessor0 } + @TestOnly static Map allTasks = new HashMap<>() + /** * Keeps track of the task instance executed by the current thread */ @@ -251,9 +259,9 @@ class TaskProcessor { private static LockManager lockManager = new LockManager() - private List> fairBuffers = new ArrayList<>() + private List fairBuffers = new ArrayList<>() - private int currentEmission + private volatile int currentEmission private Boolean isFair0 @@ -553,7 +561,9 @@ class TaskProcessor { // the channel forwarding the data from the *iterator* process to the target task final linkingChannels = new ArrayList(size) - size.times { linkingChannels[it] = new DataflowQueue() } + for( int i=0; i $name with params=$params; values=$values" + log.trace "Invoking task > $name with params=$params; values=${inputs}" // -- create the task run instance final task = createTaskRun(params) // -- set the task instance as the current in this thread currentTask.set(task) + allTasks.put(task.id, task) + // track the task provenance for the given inputs + final values = Prov.tracker.receiveInputs(task, inputs) // -- validate input lengths validateInputTuples(values) @@ -1465,7 +1477,7 @@ class TaskProcessor { fairBindOutputs0(tuples, task) } else { - bindOutputs0(tuples) + bindOutputs0(tuples, task) } // -- finally prints out the task output when 'debug' is true @@ -1478,25 +1490,26 @@ class TaskProcessor { synchronized (isFair0) { // decrement -1 because tasks are 1-based final index = task.index-1 + FairEntry entry = new FairEntry(emissions, task) // store the task emission values in a buffer - fairBuffers[index-currentEmission] = emissions + fairBuffers[index-currentEmission] = entry // check if the current task index matches the expected next emission index if( currentEmission == index ) { - while( emissions!=null ) { + while( entry!=null ) { // bind the emission values - bindOutputs0(emissions) + bindOutputs0(entry.emissions, entry.task) // remove the head and try with the following fairBuffers.remove(0) // increase the index of the next emission currentEmission++ // take the next emissions - emissions = fairBuffers[0] + entry = fairBuffers[0] } } } } - protected void bindOutputs0(Map tuples) { + protected void bindOutputs0(Map tuples, TaskRun task) { // -- bind out the collected values for( OutParam param : config.getOutputs() ) { final outValue = tuples[param.index] @@ -1509,11 +1522,11 @@ class TaskProcessor { } log.trace "Process $name > Binding out param: ${param} = ${outValue}" - bindOutParam(param, outValue) + bindOutParam(param, outValue, task) } } - protected void bindOutParam( OutParam param, List values ) { + protected void bindOutParam( OutParam param, List values, TaskRun task ) { log.trace "<$name> Binding param $param with $values" final x = values.size() == 1 ? values[0] : values final ch = param.getOutChannel() @@ -1523,7 +1536,7 @@ class TaskProcessor { // and result in a potential error. See https://github.com/nextflow-io/nextflow/issues/3768 final copy = x instanceof List && x instanceof Cloneable ? x.clone() : x // emit the final value - ch.bind(copy) + Prov.tracker.bindOutput(task, ch, copy) } } @@ -2373,7 +2386,7 @@ class TaskProcessor { * @param task The {@code TaskRun} instance to finalize */ @PackageScope - final finalizeTask( TaskHandler handler) { + final finalizeTask(TaskHandler handler) { def task = handler.task log.trace "finalizing process > ${safeTaskName(task)} -- $task" diff --git a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy index 38b4ba4782..922d67a13a 100644 --- a/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/processor/TaskRun.groovy @@ -38,6 +38,7 @@ import nextflow.exception.ProcessTemplateException import nextflow.exception.ProcessUnrecoverableException import nextflow.file.FileHelper import nextflow.file.FileHolder +import nextflow.prov.TrailRun import nextflow.script.BodyDef import nextflow.script.ScriptType import nextflow.script.TaskClosure @@ -59,7 +60,7 @@ import nextflow.spack.SpackCache */ @Slf4j -class TaskRun implements Cloneable { +class TaskRun implements Cloneable, TrailRun { final private ConcurrentHashMap cache0 = new ConcurrentHashMap() @@ -98,6 +99,11 @@ class TaskRun implements Cloneable { */ Map outputs = [:] + /** + * Holds the IDs of the upstream tasks that contributed to trigger + * the execution of this task run + */ + Set upstreamTasks void setInput( InParam param, Object value = null ) { assert param @@ -573,8 +579,8 @@ class TaskRun implements Cloneable { static final public String CMD_ENV = '.command.env' - String toString( ) { - "id: $id; name: $name; type: $type; exit: ${exitStatus==Integer.MAX_VALUE ? '-' : exitStatus}; error: $error; workDir: $workDir" + String toString() { + "TaskRun[id: $id; name: $name; type: $type; upstreams: ${upstreamTasks} exit: ${exitStatus==Integer.MAX_VALUE ? '-' : exitStatus}; error: $error; workDir: $workDir]" } diff --git a/modules/nextflow/src/main/groovy/nextflow/prov/OperatorRun.groovy b/modules/nextflow/src/main/groovy/nextflow/prov/OperatorRun.groovy new file mode 100644 index 0000000000..e3016d05c9 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/prov/OperatorRun.groovy @@ -0,0 +1,40 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.prov + +import groovy.transform.Canonical +import groovy.transform.CompileStatic + +/** + * Model an operator run + * + * @author Paolo Di Tommaso + */ +@Canonical +@CompileStatic +class OperatorRun implements TrailRun { + /** + * The list of (object) ids that was received as input by a operator run + */ + Set inputIds = new LinkedHashSet<>(10) + + @Override + String toString() { + "OperatorRun[id=${System.identityHashCode(this)}; inputs=${inputIds}]" + } +} diff --git a/modules/nextflow/src/main/groovy/nextflow/prov/Prov.groovy b/modules/nextflow/src/main/groovy/nextflow/prov/Prov.groovy new file mode 100644 index 0000000000..ddf0a02f88 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/prov/Prov.groovy @@ -0,0 +1,46 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.prov + + +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import nextflow.util.TestOnly +/** + * Provenance tracker facade class + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class Prov { + + static private volatile Tracker tracker0 + + static Tracker getTracker() { + if( tracker0==null ) + tracker0 = new Tracker() + return tracker0 + } + + @TestOnly + static void clear() { + tracker0 = null + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/prov/Tracker.groovy b/modules/nextflow/src/main/groovy/nextflow/prov/Tracker.groovy new file mode 100644 index 0000000000..db21d2bbb7 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/prov/Tracker.groovy @@ -0,0 +1,165 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.prov + +import java.util.concurrent.ConcurrentHashMap + +import groovy.transform.Canonical +import groovy.transform.CompileStatic +import groovy.util.logging.Slf4j +import groovyx.gpars.dataflow.DataflowWriteChannel +import nextflow.extension.op.Op +import nextflow.processor.TaskId +import nextflow.processor.TaskRun +/** + * Implement provenance tracking logic for tasks and operators + * + * @author Paolo Di Tommaso + */ +@Slf4j +@CompileStatic +class Tracker { + + static @Canonical class Msg { + final Object value + + String toString() { + "Msg[id=${System.identityHashCode(this)}; value=${value}]" + } + + static Msg of(Object o) { + new Msg(o) + } + } + + /** + * Associate an output value with the corresponding task run that emitted it + */ + private Map messages = new ConcurrentHashMap<>() + + List receiveInputs(TaskRun task, List inputs) { + // find the upstream tasks id + findUpstreamTasks(task, inputs) + // log for debugging purposes + logInputs(task, inputs) + // the second entry of messages list represent the run inputs list + // apply the de-normalization before returning it + return Op.unwrap(inputs) + } + + private logInputs(TaskRun task, List inputs) { + if( log.isTraceEnabled() ) { + def msg = "Task input" + msg += "\n - id : ${task.id} " + msg += "\n - name : '${task.name}'" + msg += "\n - upstream: ${task.upstreamTasks*.value.join(',')}" + for( Object it : inputs ) { + msg += "\n<= ${it}" + } + log.trace(msg) + } + } + + private logInputs(OperatorRun run, List inputs) { + if( log.isTraceEnabled() ) { + def msg = "Operator input" + msg += "\n - id: ${System.identityHashCode(run)} " + for( Object it : inputs ) { + msg += "\n<= ${it}" + } + log.trace(msg) + } + } + + List receiveInputs(OperatorRun run, List inputs) { + // find the upstream tasks id + run.inputIds.addAll(inputs.collect(msg-> System.identityHashCode(msg))) + // log for debugging purposes + logInputs(run, inputs) + // the second entry of messages list represent the task inputs list + // apply the de-normalization before returning it + return Op.unwrap(inputs) + } + + protected void findUpstreamTasks(TaskRun task, List messages) { + // find upstream tasks and restore nulls + final result = new HashSet() + for( Object msg : messages ) { + if( msg==null ) + throw new IllegalArgumentException("Message cannot be a null object") + if( msg !instanceof Msg ) + continue + final msgId = System.identityHashCode(msg) + result.addAll(findUpstreamTasks0(msgId,result)) + } + // finally bind the result to the task record + task.upstreamTasks = result + } + + protected Set findUpstreamTasks0(final int msgId, Set upstream) { + final run = messages.get(msgId) + if( run instanceof TaskRun ) { + upstream.add(run.id) + return upstream + } + if( run instanceof OperatorRun ) { + for( Integer it : run.inputIds ) { + if( it!=msgId ) { + findUpstreamTasks0(it, upstream) + } + else { + log.trace "Skip duplicate provenance message id=${msgId}" + } + } + } + return upstream + } + + Msg bindOutput(TrailRun run, DataflowWriteChannel channel, Object out) { + assert run!=null, "Argument 'run' cannot be null" + assert channel!=null, "Argument 'channel' cannot be null" + + final msg = Op.wrap(out) + logOutput(run, msg) + // map the message with the run where it has been output + messages.put(System.identityHashCode(msg), run) + // now emit the value + channel.bind(msg) + return msg + } + + private void logOutput(TrailRun run, Msg msg) { + if( log.isTraceEnabled() ) { + String str + if( run instanceof OperatorRun ) { + str = "Operator output" + str += "\n - id : ${System.identityHashCode(run)}" + } + else if( run instanceof TaskRun ) { + str = "Task output" + str += "\n - id : ${run.id}" + str += "\n - name: '${run.name}'" + } + else + throw new IllegalArgumentException("Unknown run type: ${run}") + str += "\n=> ${msg}" + log.trace(str) + } + } + +} diff --git a/modules/nextflow/src/main/groovy/nextflow/prov/TrailRun.groovy b/modules/nextflow/src/main/groovy/nextflow/prov/TrailRun.groovy new file mode 100644 index 0000000000..32e1eb4b95 --- /dev/null +++ b/modules/nextflow/src/main/groovy/nextflow/prov/TrailRun.groovy @@ -0,0 +1,26 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.prov + +/** + * Marker interface to identity a {@link nextflow.processor.TaskRun} or {@link OperatorRun} + * + * @author Paolo Di Tommaso + */ +interface TrailRun { +} diff --git a/modules/nextflow/src/main/groovy/nextflow/script/params/EachInParam.groovy b/modules/nextflow/src/main/groovy/nextflow/script/params/EachInParam.groovy index 5313a16e94..e81dc9124c 100644 --- a/modules/nextflow/src/main/groovy/nextflow/script/params/EachInParam.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/script/params/EachInParam.groovy @@ -24,6 +24,7 @@ import groovyx.gpars.dataflow.DataflowVariable import groovyx.gpars.dataflow.expression.DataflowExpression import nextflow.extension.CH import nextflow.extension.ToListOp +import nextflow.extension.op.Op import nextflow.script.TokenFileCall import nextflow.script.TokenPathCall @@ -97,7 +98,12 @@ class EachInParam extends BaseInParam { result.bind(value) } - return result.chainWith { it instanceof Collection || it == null ? it : [it] } + return result.chainWith(this.&normalizeValue) + } + + private Object normalizeValue(Object obj) { + final value = Op.unwrap(obj) + return value instanceof Collection || value == null ? value : [value] } } diff --git a/modules/nextflow/src/main/groovy/nextflow/splitter/SplitterFactory.groovy b/modules/nextflow/src/main/groovy/nextflow/splitter/SplitterFactory.groovy index e7d1d13168..f1bb8731e3 100644 --- a/modules/nextflow/src/main/groovy/nextflow/splitter/SplitterFactory.groovy +++ b/modules/nextflow/src/main/groovy/nextflow/splitter/SplitterFactory.groovy @@ -21,7 +21,7 @@ import groovy.util.logging.Slf4j import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable import groovyx.gpars.dataflow.DataflowWriteChannel -import nextflow.extension.DataflowHelper +import nextflow.extension.SubscribeOp /** * Factory class for splitter objects * @@ -95,11 +95,11 @@ class SplitterFactory { opt.each = { count++ } strategy.options(opt) - def events = new HashMap(2) - events.onNext = { entry -> strategy.target(entry).apply() } - events.onComplete = { result.bind(count) } - - DataflowHelper.subscribeImpl ( source, events ) + new SubscribeOp() + .withSource(source) + .withOnNext({ entry -> strategy.target(entry).apply() }) + .withOnComplete({ result.bind(count) }) + .apply() // return the resulting channel return result diff --git a/modules/nextflow/src/test/groovy/nextflow/ChannelTest.groovy b/modules/nextflow/src/test/groovy/nextflow/ChannelTest.groovy index c7b45a6395..90cee0b717 100644 --- a/modules/nextflow/src/test/groovy/nextflow/ChannelTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/ChannelTest.groovy @@ -47,59 +47,59 @@ class ChannelTest extends Specification { when: result = Channel.of('a') then: - result.val == 'a' - result.val == Channel.STOP + result.unwrap() == 'a' + result.unwrap() == Channel.STOP when: result = Channel.of('a','b','c') then: - result.val == 'a' - result.val == 'b' - result.val == 'c' - result.val == Channel.STOP + result.unwrap() == 'a' + result.unwrap() == 'b' + result.unwrap() == 'c' + result.unwrap() == Channel.STOP when: result = Channel.of([1,2,3]) then: - result.val == [1,2,3] - result.val == Channel.STOP + result.unwrap() == [1,2,3] + result.unwrap() == Channel.STOP when: result = Channel.of([1,2], [3,4]) then: - result.val == [1,2] - result.val == [3,4] - result.val == Channel.STOP + result.unwrap() == [1,2] + result.unwrap() == [3,4] + result.unwrap() == Channel.STOP when: result = Channel.of([]) then: - result.val == [] - result.val == Channel.STOP + result.unwrap() == [] + result.unwrap() == Channel.STOP when: result = Channel.of() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.of([1,2,3].toArray()) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.of([].toArray()) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.of(null) then: - result.val == null - result.val == Channel.STOP + result.unwrap() == null + result.unwrap() == Channel.STOP } @@ -111,30 +111,30 @@ class ChannelTest extends Specification { when: result = Channel.of(1..3) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.of(1..3,'X','Y') then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == 'X' - result.val == 'Y' - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 'X' + result.unwrap() == 'Y' + result.unwrap() == Channel.STOP when: result = Channel.of(1..3,'X'..'Y') then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == 'X' - result.val == 'Y' - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 'X' + result.unwrap() == 'Y' + result.unwrap() == Channel.STOP } def 'should create channel from a list'() { @@ -145,26 +145,26 @@ class ChannelTest extends Specification { when: result = Channel.fromList(['alpha','delta']) then: - result.val == 'alpha' - result.val == 'delta' - result.val == Channel.STOP + result.unwrap() == 'alpha' + result.unwrap() == 'delta' + result.unwrap() == Channel.STOP when: result = Channel.fromList([]) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.fromList(null) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.fromList([1..3, 'X'..'Y']) then: - result.val == 1..3 - result.val == 'X'..'Y' - result.val == Channel.STOP + result.unwrap() == 1..3 + result.unwrap() == 'X'..'Y' + result.unwrap() == Channel.STOP } def testFrom() { @@ -174,27 +174,27 @@ class ChannelTest extends Specification { when: result = Channel.from('hola') then: - result.val == 'hola' - result.val == Channel.STOP + result.unwrap() == 'hola' + result.unwrap() == Channel.STOP when: result = Channel.from('alpha','delta') then: - result.val == 'alpha' - result.val == 'delta' - result.val == Channel.STOP + result.unwrap() == 'alpha' + result.unwrap() == 'delta' + result.unwrap() == Channel.STOP when: result = Channel.from(['alpha','delta']) then: - result.val == 'alpha' - result.val == 'delta' - result.val == Channel.STOP + result.unwrap() == 'alpha' + result.unwrap() == 'delta' + result.unwrap() == Channel.STOP when: result = Channel.from([]) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } def testSingleFile() { @@ -202,12 +202,12 @@ class ChannelTest extends Specification { when: def channel = Channel.fromPath('/some/file.txt') then: - channel.val == Paths.get('/some/file.txt') + channel.unwrap() == Paths.get('/some/file.txt') when: channel = Channel.fromPath('/some/f{i}le.txt') then: - channel.val == Paths.get('/some/f{i}le.txt') + channel.unwrap() == Paths.get('/some/f{i}le.txt') } @@ -228,42 +228,42 @@ class ChannelTest extends Specification { when: def result = Channel .fromPath("$folder/{alpha,gamma}.txt") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == [ 'alpha.txt', 'gamma.txt' ] when: result = Channel .fromPath("$folder/file?.txt") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == [ 'file4.txt', 'file5.txt' ] when: result = Channel .fromPath("$folder/file*.txt") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == [ 'file4.txt', 'file5.txt', 'file66.txt' ] when: result = Channel .fromPath("$folder/{alpha,}.txt") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == ['alpha.txt'] when: result = Channel .fromPath("$folder/{,beta}.txt") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == ['beta.txt'] when: result = Channel .fromPath("$folder/alpha.txt{,}") - .toSortedList().getVal().collect { it -> it.name } + .toSortedList().unwrap().collect { it -> it.name } then: result == ['alpha.txt'] } @@ -276,21 +276,21 @@ class ChannelTest extends Specification { when: def result = Channel .fromPath([relative:false], 'alpha.txt') - .toSortedList().getVal() + .toSortedList().unwrap() then: result*.toString() == [ file1.absolutePath ] when: result = Channel .fromPath([relative:true], 'alpha.txt') - .toSortedList().getVal() + .toSortedList().unwrap() then: result*.toString() == [ 'alpha.txt' ] when: result = Channel .fromPath([:], 'alpha.txt') //no relative option set - .toSortedList().getVal() + .toSortedList().unwrap() then: result*.toString() == [ file1.absolutePath ] @@ -308,17 +308,17 @@ class ChannelTest extends Specification { def file4 = Files.createFile(folder.resolve('gamma.txt')) when: - def result = Channel.fromPath("$folder/*").toSortedList().getVal() + def result = Channel.fromPath("$folder/*").toSortedList().unwrap() then: result == [file3, file4] when: - result = Channel.fromPath("$folder/.*").toSortedList().getVal() + result = Channel.fromPath("$folder/.*").toSortedList().unwrap() then: result == [file1, file2] when: - result = Channel.fromPath("$folder/{.*,*}", hidden: true).toSortedList().getVal() + result = Channel.fromPath("$folder/{.*,*}", hidden: true).toSortedList().unwrap() then: result == [file1, file2, file3, file4] @@ -337,17 +337,17 @@ class ChannelTest extends Specification { def file6 = Files.createFile(sub1.resolve('file6.txt')) when: - def result = Channel.fromPath("$folder/*.txt").toSortedList().getVal() + def result = Channel.fromPath("$folder/*.txt").toSortedList().unwrap() then: result == [file1, file2, file3] when: - def result2 = Channel.fromPath("$folder/**.txt").toSortedList().getVal() + def result2 = Channel.fromPath("$folder/**.txt").toSortedList().unwrap() then: result2 == [file1, file2, file3, file6] when: - def result3 = Channel.fromPath("$folder/sub1/**.log").toSortedList().getVal() + def result3 = Channel.fromPath("$folder/sub1/**.log").toSortedList().unwrap() then: result3 == [file5] @@ -365,17 +365,17 @@ class ChannelTest extends Specification { def file6 = Files.createFile(sub1.resolve('file6.txt')) when: - def result = Channel.fromPath("$folder/file\\*.txt").toSortedList().getVal() + def result = Channel.fromPath("$folder/file\\*.txt").toSortedList().unwrap() then: result == [file2] when: - result = Channel.fromPath("$folder/file*.txt", glob: false).toSortedList().getVal() + result = Channel.fromPath("$folder/file*.txt", glob: false).toSortedList().unwrap() then: result == [file2] when: - result = Channel.fromPath("$folder/sub\\[a-b\\]/file*").toSortedList().getVal() + result = Channel.fromPath("$folder/sub\\[a-b\\]/file*").toSortedList().unwrap() then: result == [file5,file6] @@ -395,56 +395,56 @@ class ChannelTest extends Specification { when: List result = Channel .fromPath( folder.toAbsolutePath().toString() + '/*.txt' ) - .toSortedList().getVal().collect { it.name } + .toSortedList().unwrap().collect { it.name } then: result == [ 'file1.txt', 'file2.txt' ] when: result = Channel .fromPath( folder.toAbsolutePath().toString() + '/*' ) - .toSortedList().getVal().collect { it.name } + .toSortedList().unwrap().collect { it.name } then: result == [ 'file1.txt', 'file2.txt', 'file3.log' ] when: result = Channel .fromPath( folder.toAbsolutePath().toString() + '/*', type: 'file' ) - .toSortedList().getVal().collect { it.name } + .toSortedList().unwrap().collect { it.name } then: result == [ 'file1.txt', 'file2.txt', 'file3.log' ] when: result = Channel .fromPath( folder.toAbsolutePath().toString() + '/*', type: 'dir' ) - .toSortedList().getVal().collect { it.name } + .toSortedList().unwrap().collect { it.name } then: result == ['sub1'] when: result = Channel .fromPath( folder.toAbsolutePath().toString() + '/*', type: 'any' ) - .toSortedList().getVal().collect { it.name } + .toSortedList().unwrap().collect { it.name } then: result == [ 'file1.txt', 'file2.txt', 'file3.log', 'sub1' ] when: result = Channel .fromPath( folder.toAbsolutePath().toString() + '/**', type: 'file' ) - .toSortedList() .getVal() .collect { it.name } + .toSortedList() .unwrap() .collect { it.name } then: result == [ 'file1.txt', 'file2.txt', 'file3.log', 'file5.log' ] when: def result2 = Channel .fromPath( folder.toAbsolutePath().toString() + '/**', type: 'file', maxDepth: 0 ) - .toSortedList() .getVal() .collect { it.name } + .toSortedList() .unwrap() .collect { it.name } then: result2 == ['file1.txt', 'file2.txt', 'file3.log' ] when: def result3 = Channel .fromPath (folder.toAbsolutePath().toString() + '/{file1.txt,sub1/file5.log}') - .toSortedList() .val .collect { it.name } + .toSortedList() .unwrap() .collect { it.name } then: result3 == ['file1.txt','file5.log'] @@ -472,14 +472,14 @@ class ChannelTest extends Specification { when: List result = Channel .fromPath( '*.txt' ) - .toSortedList().getVal().collect { it.toString() } + .toSortedList().unwrap().collect { it.toString() } then: result == [ file1.toString(), file2.toString() ] when: result = Channel .fromPath( '*.txt', relative: true ) - .toSortedList().getVal().collect { it.toString() } + .toSortedList().unwrap().collect { it.toString() } then: result == [ file1.name, file2.name ] @@ -499,14 +499,14 @@ class ChannelTest extends Specification { when: List result = Channel .fromPath( 'file3.log' ) - .toSortedList().getVal().collect { it.toString() } + .toSortedList().unwrap().collect { it.toString() } then: result == [ file3.toString() ] when: result = Channel .fromPath( 'file3.log', relative: true ) - .toSortedList().getVal().collect { it.toString() } + .toSortedList().unwrap().collect { it.toString() } then: result == [ file3.name ] @@ -531,13 +531,13 @@ class ChannelTest extends Specification { // -- by default traverse symlinks when: - def result = Channel.fromPath( folder.toAbsolutePath().toString() + '/**/*.txt' ).toSortedList({it.name}).getVal().collect { it.getName() } + def result = Channel.fromPath( folder.toAbsolutePath().toString() + '/**/*.txt' ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == ['file3.txt','file3.txt','file4.txt','file4.txt'] // -- switch off symlinks traversing when: - def result2 = Channel.fromPath( folder.toAbsolutePath().toString() + '/**/*.txt', followLinks: false ).toSortedList({it.name}).getVal().collect { it.getName() } + def result2 = Channel.fromPath( folder.toAbsolutePath().toString() + '/**/*.txt', followLinks: false ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result2 == ['file3.txt','file4.txt'] @@ -554,17 +554,17 @@ class ChannelTest extends Specification { // -- by default no hidden when: - def result = Channel.fromPath( folder.toAbsolutePath().toString() + '/*.txt' ).toSortedList({it.name}).getVal().collect { it.getName() } + def result = Channel.fromPath( folder.toAbsolutePath().toString() + '/*.txt' ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == ['file1.txt','file2.txt'] when: - result = Channel.fromPath( folder.toAbsolutePath().toString() + '/.*.txt' ).toSortedList({it.name}).getVal().collect { it.getName() } + result = Channel.fromPath( folder.toAbsolutePath().toString() + '/.*.txt' ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == ['.file_hidden.txt'] when: - result = Channel.fromPath( folder.toAbsolutePath().toString() + '/*.txt', hidden: true ).toSortedList({it.name}).getVal().collect { it.getName() } + result = Channel.fromPath( folder.toAbsolutePath().toString() + '/*.txt', hidden: true ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == ['.file_hidden.txt', 'file1.txt','file2.txt'] @@ -580,17 +580,17 @@ class ChannelTest extends Specification { folder.resolve('file3.fq').text = 'Ciao' when: - def result = Channel.fromPath( ["$folder/*.txt", "$folder/*.fq"] ).toSortedList({it.name}).getVal().collect { it.getName() } + def result = Channel.fromPath( ["$folder/*.txt", "$folder/*.fq"] ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == [ 'file1.txt', 'file2.txt', 'file3.fq' ] when: - result = Channel.fromPath( ["$folder/file1.txt", "$folder/file2.txt", "$folder/file3.fq"] ).toSortedList({it.name}).getVal().collect { it.getName() } + result = Channel.fromPath( ["$folder/file1.txt", "$folder/file2.txt", "$folder/file3.fq"] ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == [ 'file1.txt', 'file2.txt', 'file3.fq' ] when: - result = Channel.fromPath( ["$folder/*"] ).toSortedList({it.name}).getVal().collect { it.getName() } + result = Channel.fromPath( ["$folder/*"] ).toSortedList({it.name}).unwrap().collect { it.getName() } then: result == [ 'file1.txt', 'file2.txt', 'file3.fq' ] } @@ -611,21 +611,21 @@ class ChannelTest extends Specification { when: def ch = Channel.fromPath(file1.toString()) then: - ch.getVal() == file1 - ch.getVal() == Channel.STOP + ch.unwrap() == file1 + ch.unwrap() == Channel.STOP when: ch = Channel.fromPath([file1.toString(), file2.toString()]) then: - ch.getVal() == file1 - ch.getVal() == file2 - ch.getVal() == Channel.STOP + ch.unwrap() == file1 + ch.unwrap() == file2 + ch.unwrap() == Channel.STOP when: ch = Channel.fromPath(file1.toString(), checkIfExists: true) then: - ch.getVal() == file1 - ch.getVal() == Channel.STOP + ch.unwrap() == file1 + ch.unwrap() == Channel.STOP when: def session = new Session() @@ -680,7 +680,7 @@ class ChannelTest extends Specification { when: def result = Channel.fromPath("$folder/*.txt", checkIfExists: true) then: - result.getVal() instanceof Path + result.unwrap() instanceof Path !session.terminated when: @@ -743,18 +743,18 @@ class ChannelTest extends Specification { when: def pairs = Channel.fromFilePairs(folder.resolve("*_{1,2}.*")) then: - pairs.val == ['alpha', [a1, a2]] - pairs.val == ['beta', [b1, b2]] - pairs.val == ['delta', [d1, d2]] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', [a1, a2]] + pairs.unwrap() == ['beta', [b1, b2]] + pairs.unwrap() == ['delta', [d1, d2]] + pairs.unwrap() == Channel.STOP when: pairs = Channel.fromFilePairs(folder.resolve("*_{1,2}.fa") , flat: true) then: - pairs.val == ['alpha', a1, a2] - pairs.val == ['beta', b1, b2] - pairs.val == ['delta', d1, d2] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', a1, a2] + pairs.unwrap() == ['beta', b1, b2] + pairs.unwrap() == ['delta', d1, d2] + pairs.unwrap() == Channel.STOP } def 'should group files with the same prefix and root path' () { @@ -771,7 +771,7 @@ class ChannelTest extends Specification { SysEnv.push(NXF_FILE_ROOT: folder.toString()) when: - def pairs = Channel .fromFilePairs("*_{1,2}.*") .toList(). getVal() .sort { it[0] } + def pairs = Channel .fromFilePairs("*_{1,2}.*") .toList() .unwrap() .sort { it[0] } then: pairs == [ ['aa', [a1, a2]], @@ -797,9 +797,9 @@ class ChannelTest extends Specification { def grouping = { Path file -> file.name.substring(0,1) } def pairs = Channel.fromFilePairs(folder.resolve("*_{1,2}.*"), grouping, size:-1) then: - pairs.val == ['c', [c1, c2]] - pairs.val == ['h', [a1, a2, b1, b2]] - pairs.val == Channel.STOP + pairs.unwrap() == ['c', [c1, c2]] + pairs.unwrap() == ['h', [a1, a2, b1, b2]] + pairs.unwrap() == Channel.STOP } @@ -824,26 +824,26 @@ class ChannelTest extends Specification { // default size == 2 def pairs = Channel.fromFilePairs(folder.resolve("*_{1,2,3}.fa")) then: - pairs.val == ['alpha', [a1, a2]] - pairs.val == ['beta', [b1, b2]] - pairs.val == ['delta', [d1, d2]] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', [a1, a2]] + pairs.unwrap() == ['beta', [b1, b2]] + pairs.unwrap() == ['delta', [d1, d2]] + pairs.unwrap() == Channel.STOP when: pairs = Channel.fromFilePairs(folder.resolve("*_{1,2,3,4}.fa"), size: 3) then: - pairs.val == ['alpha', [a1, a2, a3]] - pairs.val == ['beta', [b1, b2, b3]] - pairs.val == ['delta', [d1, d2, d3]] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', [a1, a2, a3]] + pairs.unwrap() == ['beta', [b1, b2, b3]] + pairs.unwrap() == ['delta', [d1, d2, d3]] + pairs.unwrap() == Channel.STOP when: pairs = Channel.fromFilePairs(folder.resolve("*_{1,2,3,4}.fa"), size: -1) then: - pairs.val == ['alpha', [a1, a2, a3]] - pairs.val == ['beta', [b1, b2, b3]] - pairs.val == ['delta', [d1, d2, d3, d4]] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', [a1, a2, a3]] + pairs.unwrap() == ['beta', [b1, b2, b3]] + pairs.unwrap() == ['delta', [d1, d2, d3, d4]] + pairs.unwrap() == Channel.STOP } @@ -864,11 +864,11 @@ class ChannelTest extends Specification { when: def pairs = Channel.fromFilePairs([folder.resolve("*_{1,2}.fa"), folder.resolve("$folder/*_{1,2}.fq")]) then: - pairs.val == ['alpha', [a1, a2]] - pairs.val == ['beta', [b1, b2]] - pairs.val == ['delta', [d1, d2]] - pairs.val == ['gamma', [g1, g2]] - pairs.val == Channel.STOP + pairs.unwrap() == ['alpha', [a1, a2]] + pairs.unwrap() == ['beta', [b1, b2]] + pairs.unwrap() == ['delta', [d1, d2]] + pairs.unwrap() == ['gamma', [g1, g2]] + pairs.unwrap() == Channel.STOP } @@ -881,9 +881,9 @@ class ChannelTest extends Specification { when: def files = Channel.fromFilePairs(folder.resolve('*.fa'), size:1) then: - files.val == ['alpha_1', [a1]] - files.val == ['alpha_2', [a2]] - files.val == Channel.STOP + files.unwrap() == ['alpha_1', [a1]] + files.unwrap() == ['alpha_2', [a2]] + files.unwrap() == Channel.STOP } def 'should return singleton' () { @@ -894,8 +894,8 @@ class ChannelTest extends Specification { when: def files = Channel.fromFilePairs(a1, size:1) then: - files.val == ['alpha_1', [a1]] - files.val == Channel.STOP + files.unwrap() == ['alpha_1', [a1]] + files.unwrap() == Channel.STOP } def 'should use size one by default' () { @@ -906,8 +906,8 @@ class ChannelTest extends Specification { when: def files = Channel.fromFilePairs(a1) then: - files.val == ['alpha_1', [a1]] - files.val == Channel.STOP + files.unwrap() == ['alpha_1', [a1]] + files.unwrap() == Channel.STOP } def 'should watch and emit a file' () { @@ -919,14 +919,14 @@ class ChannelTest extends Specification { sleep 500 Files.createFile(folder.resolve('hello.txt')) then: - result.val == folder.resolve('hello.txt') + result.unwrap() == folder.resolve('hello.txt') when: result = Channel.watchPath(folder.toString()) sleep 500 Files.createFile(folder.resolve('ciao.txt')) then: - result.val == folder.resolve('ciao.txt') + result.unwrap() == folder.resolve('ciao.txt') cleanup: folder?.deleteDir() @@ -938,7 +938,7 @@ class ChannelTest extends Specification { when: def result = Channel.watchPath("$folder/foo/*") then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP cleanup: folder?.deleteDir() } @@ -954,10 +954,10 @@ class ChannelTest extends Specification { when: def result = Channel.fromFilePairs(files) then: - result.val == ['SRR389222_sub1', [Paths.get('/data/SRR389222_sub1.fastq.gz')]] - result.val == ['SRR389222_sub2', [Paths.get('/data/SRR389222_sub2.fastq.gz')]] - result.val == ['SRR389222_sub3', [Paths.get('/data/SRR389222_sub3.fastq.gz')]] - result.val == Channel.STOP + result.unwrap() == ['SRR389222_sub1', [Paths.get('/data/SRR389222_sub1.fastq.gz')]] + result.unwrap() == ['SRR389222_sub2', [Paths.get('/data/SRR389222_sub2.fastq.gz')]] + result.unwrap() == ['SRR389222_sub3', [Paths.get('/data/SRR389222_sub3.fastq.gz')]] + result.unwrap() == Channel.STOP } def 'should check file pair exists' () { @@ -984,9 +984,9 @@ class ChannelTest extends Specification { when: def result = Channel.fromFilePairs([ftp1, ftp2]) then: - result.val == ['SRR389222_sub1', [ftp1]] - result.val == ['SRR389222_sub2', [ftp2]] - result.val == Channel.STOP + result.unwrap() == ['SRR389222_sub1', [ftp1]] + result.unwrap() == ['SRR389222_sub2', [ftp2]] + result.unwrap() == Channel.STOP } diff --git a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy index 2a405ef956..ac6adcddd2 100644 --- a/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/executor/BashWrapperBuilderTest.groovy @@ -27,6 +27,7 @@ import nextflow.container.ContainerConfig import nextflow.container.DockerBuilder import nextflow.container.SingularityBuilder import nextflow.processor.TaskBean +import nextflow.processor.TaskId import nextflow.util.MustacheTemplateEngine import org.yaml.snakeyaml.Yaml import spock.lang.Specification @@ -262,6 +263,7 @@ class BashWrapperBuilderTest extends Specification { */ when: def wrapper = newBashWrapperBuilder( + taskId: TaskId.of(100), name: 'Hello 1', workDir: folder, headerScript: '#BSUB -x 1\n#BSUB -y 2', @@ -286,6 +288,7 @@ class BashWrapperBuilderTest extends Specification { */ when: def wrapper = newBashWrapperBuilder( + taskId: TaskId.of(200), name: 'Hello 2', workDir: folder, statsEnabled: true ) .buildNew0() @@ -342,13 +345,15 @@ class BashWrapperBuilderTest extends Specification { def 'should create task metadata string' () { given: def builder = newBashWrapperBuilder( + taskId: TaskId.of(100), name: 'foo', arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, arrayWorkDirs: [ Path.of('/work/01'), Path.of('/work/02'), Path.of('/work/03') ], containerConfig: [enabled: true], containerImage: 'quay.io/nextflow:bash', - outputFiles: ['foo.txt', '*.bar', '**/baz'] + outputFiles: ['foo.txt', '*.bar', '**/baz'], + upstreamTasks: new LinkedHashSet<>([TaskId.of(10), TaskId.of(20)]) ) when: @@ -356,6 +361,7 @@ class BashWrapperBuilderTest extends Specification { then: meta == '''\ ### --- + ### id: '100' ### name: 'foo' ### array: ### index-name: SLURM_ARRAY_TASK_ID @@ -369,6 +375,9 @@ class BashWrapperBuilderTest extends Specification { ### - 'foo.txt' ### - '*.bar' ### - '**/baz' + ### upstream-tasks: + ### - '10' + ### - '20' ### ... '''.stripIndent() @@ -376,6 +385,7 @@ class BashWrapperBuilderTest extends Specification { def yaml = meta.readLines().collect(it-> it.substring(4)).join('\n') def obj = new Yaml().load(yaml) as Map then: + obj.id == '100' obj.name == 'foo' obj.array == [ 'index-name':'SLURM_ARRAY_TASK_ID', @@ -388,17 +398,22 @@ class BashWrapperBuilderTest extends Specification { def 'should add task metadata' () { when: - def bash = newBashWrapperBuilder([name:'task1']) + def bash = newBashWrapperBuilder( + taskId: TaskId.of(123), + name:'task1' + ) then: bash.makeBinding().containsKey('task_metadata') bash.makeBinding().task_metadata == '''\ ### --- + ### id: '123' ### name: 'task1' ### ... '''.stripIndent() when: bash = newBashWrapperBuilder( + taskId: TaskId.of(321), name: 'task2', arrayIndexName: 'SLURM_ARRAY_TASK_ID', arrayIndexStart: 0, @@ -410,6 +425,7 @@ class BashWrapperBuilderTest extends Specification { then: bash.makeBinding().task_metadata == '''\ ### --- + ### id: '321' ### name: 'task2' ### array: ### index-name: SLURM_ARRAY_TASK_ID diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/BranchOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/BranchOpTest.groovy index 13d3c9f49f..29269e8240 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/BranchOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/BranchOpTest.groovy @@ -48,14 +48,14 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 3 and: - result[0].val == 0 - result[0].val == Channel.STOP + result[0].unwrap() == 0 + result[0].unwrap() == Channel.STOP and: - result[1].val == 1 - result[1].val == Channel.STOP + result[1].unwrap() == 1 + result[1].unwrap() == Channel.STOP and: - result[2].val == 2 - result[2].val == Channel.STOP + result[2].unwrap() == 2 + result[2].unwrap() == Channel.STOP } def 'should branch and capture default' () { @@ -71,12 +71,12 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 2 and: - result[0].val == 10 - result[0].val == Channel.STOP + result[0].unwrap() == 10 + result[0].unwrap() == Channel.STOP and: - result[1].val == 20 - result[1].val == 30 - result[1].val == Channel.STOP + result[1].unwrap() == 20 + result[1].unwrap() == 30 + result[1].unwrap() == Channel.STOP } @@ -93,12 +93,12 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 2 and: - result[0].val == 1 - result[0].val == 2 - result[0].val == 3 - result[0].val == Channel.STOP + result[0].unwrap() == 1 + result[0].unwrap() == 2 + result[0].unwrap() == 3 + result[0].unwrap() == Channel.STOP and: - result[1].val == Channel.STOP + result[1].unwrap() == Channel.STOP } @@ -164,14 +164,14 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 3 and: - result[0].val == 0 - result[0].val == Channel.STOP + result[0].unwrap() == 0 + result[0].unwrap() == Channel.STOP and: - result[1].val == 10 - result[1].val == Channel.STOP + result[1].unwrap() == 10 + result[1].unwrap() == Channel.STOP and: - result[2].val == 20 - result[2].val == Channel.STOP + result[2].unwrap() == 20 + result[2].unwrap() == Channel.STOP } def 'should handle complex nested return statement' () { @@ -187,10 +187,10 @@ class BranchOpTest extends Dsl2Spec { } ''') then: - result.val == 'less than zero' - result.val == 'zero' - result.val == 'great than zero' - result.val == Channel.STOP + result.unwrap() == 'less than zero' + result.unwrap() == 'zero' + result.unwrap() == 'great than zero' + result.unwrap() == Channel.STOP } @Ignore // this is not supported and require explicit use of `return` @@ -207,10 +207,10 @@ class BranchOpTest extends Dsl2Spec { } ''') then: - result.val == 'less than zero' - result.val == 'zero' - result.val == 'great than zero' - result.val == Channel.STOP + result.unwrap() == 'less than zero' + result.unwrap() == 'zero' + result.unwrap() == 'great than zero' + result.unwrap() == Channel.STOP } @@ -230,14 +230,14 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 3 and: - result[0].val == 0 - result[0].val == Channel.STOP + result[0].unwrap() == 0 + result[0].unwrap() == Channel.STOP and: - result[1].val == 3 - result[1].val == Channel.STOP + result[1].unwrap() == 3 + result[1].unwrap() == Channel.STOP and: - result[2].val == 6 - result[2].val == Channel.STOP + result[2].unwrap() == 6 + result[2].unwrap() == Channel.STOP } def 'should branch on pair argument' () { @@ -254,11 +254,11 @@ class BranchOpTest extends Dsl2Spec { then: result.size() == 2 and: - result[0].val == 1 - result[0].val == Channel.STOP + result[0].unwrap() == 1 + result[0].unwrap() == Channel.STOP and: - result[1].val == ['b', 2] - result[1].val == Channel.STOP + result[1].unwrap() == ['b', 2] + result[1].unwrap() == Channel.STOP } def 'should pass criteria as argument' () { @@ -362,10 +362,10 @@ class BranchOpTest extends Dsl2Spec { result.size() == 2 and: result[0] instanceof DataflowVariable - result[0].val == Channel.STOP + result[0].unwrap() == Channel.STOP and: result[1] instanceof DataflowVariable - result[1].val == 10 + result[1].unwrap() == 10 } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/BufferOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/BufferOpTest.groovy index 3d8494277a..469fa08093 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/BufferOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/BufferOpTest.groovy @@ -27,7 +27,7 @@ import nextflow.Session * * @author Paolo Di Tommaso */ -@Timeout(10) +@Timeout(5) class BufferOpTest extends Specification { def setup() { @@ -35,120 +35,107 @@ class BufferOpTest extends Specification { } def testBufferClose() { - when: def r1 = Channel.of(1,2,3,1,2,3).buffer({ it == 2 }) then: - r1.val == [1,2] - r1.val == [3,1,2] - r1.val == Channel.STOP + r1.unwrap() == [1,2] + r1.unwrap() == [3,1,2] + r1.unwrap() == Channel.STOP when: def r2 = Channel.of('a','b','c','a','b','z').buffer(~/b/) then: - r2.val == ['a','b'] - r2.val == ['c','a','b'] - r2.val == Channel.STOP - + r2.unwrap() == ['a','b'] + r2.unwrap() == ['c','a','b'] + r2.unwrap() == Channel.STOP } def testBufferWithCount() { - when: def r1 = Channel.of(1,2,3,1,2,3,1).buffer( size:2 ) then: - r1.val == [1,2] - r1.val == [3,1] - r1.val == [2,3] - r1.val == Channel.STOP + r1.unwrap() == [1,2] + r1.unwrap() == [3,1] + r1.unwrap() == [2,3] + r1.unwrap() == Channel.STOP when: r1 = Channel.of(1,2,3,1,2,3,1).buffer( size:2, remainder: true ) then: - r1.val == [1,2] - r1.val == [3,1] - r1.val == [2,3] - r1.val == [1] - r1.val == Channel.STOP - + r1.unwrap() == [1,2] + r1.unwrap() == [3,1] + r1.unwrap() == [2,3] + r1.unwrap() == [1] + r1.unwrap() == Channel.STOP when: def r2 = Channel.of(1,2,3,4,5,1,2,3,4,5,1,2,9).buffer( size:3, skip:2 ) then: - r2.val == [3,4,5] - r2.val == [3,4,5] - r2.val == Channel.STOP + r2.unwrap() == [3,4,5] + r2.unwrap() == [3,4,5] + r2.unwrap() == Channel.STOP when: r2 = Channel.of(1,2,3,4,5,1,2,3,4,5,1,2,9).buffer( size:3, skip:2, remainder: true ) then: - r2.val == [3,4,5] - r2.val == [3,4,5] - r2.val == [9] - r2.val == Channel.STOP - + r2.unwrap() == [3,4,5] + r2.unwrap() == [3,4,5] + r2.unwrap() == [9] + r2.unwrap() == Channel.STOP } def testBufferInvalidArg() { - when: Channel.create().buffer( xxx: true ) then: IllegalArgumentException e = thrown() - } def testBufferOpenClose() { - when: def r1 = Channel.of(1,2,3,4,5,1,2,3,4,5,1,2).buffer( 2, 4 ) then: - r1.val == [2,3,4] - r1.val == [2,3,4] - r1.val == Channel.STOP + r1.unwrap() == [2,3,4] + r1.unwrap() == [2,3,4] + r1.unwrap() == Channel.STOP when: def r2 = Channel.of('a','b','c','a','b','z').buffer(~/a/,~/b/) then: - r2.val == ['a','b'] - r2.val == ['a','b'] - r2.val == Channel.STOP - + r2.unwrap() == ['a','b'] + r2.unwrap() == ['a','b'] + r2.unwrap() == Channel.STOP } def testBufferCloseWithOptions() { - when: def sum = 0 def r1 = Channel.of(1,2,3,1,2,3).buffer(remainder: true, { sum+=it; sum==7 }) then: - r1.val == [1,2,3,1] - r1.val == [2,3] - r1.val == Channel.STOP - + r1.unwrap() == [1,2,3,1] + r1.unwrap() == [2,3] + r1.unwrap() == Channel.STOP } def testBufferWithValueChannel() { - when: def result = Channel.value(1).buffer(size: 1) then: - result.val == [1] - result.val == Channel.STOP + result.unwrap() == [1] + result.unwrap() == Channel.STOP when: result = Channel.value(1).buffer(size: 10) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.value(1).buffer(size: 10,remainder: true) - result.val == [1] + result.unwrap() == [1] then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } - } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/CollectFileOperatorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/CollectFileOperatorTest.groovy index e0a6a39ad5..9b14839ac3 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/CollectFileOperatorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/CollectFileOperatorTest.groovy @@ -46,14 +46,13 @@ class CollectFileOperatorTest extends Specification { } def testCollectFileString() { - when: def result = Channel .from('alpha','beta','gamma') .collectFile { it == 'beta' ? ['file2', it.reverse() ] : ['file1',it] } .toSortedList { it.name } - List list = result.val + List list = result.unwrap() then: list[0].name == 'file1' @@ -61,12 +60,9 @@ class CollectFileOperatorTest extends Specification { list[1].name == 'file2' list[1].text == 'ateb' - } - def testCollectFileWithFiles() { - given: def file1 = Files.createTempDirectory('temp').resolve('A') file1.deleteOnExit() @@ -85,7 +81,7 @@ class CollectFileOperatorTest extends Specification { .from(file1,file2,file3) .collectFile(sort:'index') .toSortedList { it.name } - .getVal() as List + .unwrap() as List then: list[0].name == 'A' @@ -100,7 +96,7 @@ class CollectFileOperatorTest extends Specification { .from(file1,file2,file3) .collectFile(sort:'index', newLine:true) .toSortedList { it.name } - .getVal() as List + .unwrap() as List then: list[0].name == 'A' @@ -108,19 +104,15 @@ class CollectFileOperatorTest extends Specification { list[1].name == 'B' list[1].text == 'Hello\nworld\n' - - } def testCollectManyFiles() { - - when: def list = Channel .from('Hola', 'Ciao', 'Hello', 'Bonjour', 'Halo') .collectFile(sort:'index') { item -> [ "${item[0]}.txt", item + '\n' ] } .toList() - .getVal() + .unwrap() .sort { it.name } then: @@ -130,73 +122,65 @@ class CollectFileOperatorTest extends Specification { list[1].name == 'C.txt' list[2].name == 'H.txt' list[2].text == 'Hola\nHello\nHalo\n' - } - def testCollectFileWithStrings() { - when: def result = Channel .from('alpha', 'beta', 'gamma') .collectFile(name: 'hello.txt', newLine: true, sort:'index') - def file = result.val + def file = result.unwrap() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP file.name == 'hello.txt' file.text == 'alpha\nbeta\ngamma\n' } def testCollectFileWithDefaultName() { - when: def result = Channel .from('alpha', 'beta', 'gamma') .collectFile(newLine: true, sort:'index') - def file = result.val + def file = result.unwrap() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP file.name.startsWith('collect') file.text == 'alpha\nbeta\ngamma\n' } def testCollectFileAndSortWithClosure() { - when: def result = Channel .from('delta', 'beta', 'gamma','alpha') .collectFile(newLine: true, sort:{ it -> it }) - def file = result.val + def file = result.unwrap() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP file.name.startsWith('collect') file.text == 'alpha\nbeta\ndelta\ngamma\n' } def testCollectFileAndSortWithComparator() { - when: def result = Channel .from('delta', 'beta', 'gamma','alpha') .collectFile(newLine: true, sort:{ a,b -> b<=>a } as Comparator) - def file = result.val + def file = result.unwrap() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP file.name.startsWith('collect') file.text == 'gamma\ndelta\nbeta\nalpha\n' } - def 'should collect file and skip header line' () { - given: def file1 = Files.createTempDirectory('temp').resolve('A') file1.deleteOnExit() @@ -210,13 +194,12 @@ class CollectFileOperatorTest extends Specification { file3.deleteOnExit() file3.text = 'HEADER\nxxx\nyyy\nzzz\n' - when: def files = Channel .from(file1,file2,file3) .collectFile(skip:1, sort: 'index') .toList() - .getVal() + .unwrap() def result = [:]; files.each{ result[it.name]=it } then: @@ -226,13 +209,12 @@ class CollectFileOperatorTest extends Specification { result.B.name == 'B' result.B.text == 'Hello\nworld\n' - when: files = Channel .from(file1,file2,file3) .collectFile(skip:2, sort: 'index') .toList() - .getVal() + .unwrap() result = [:]; files.each{ result[it.name]=it } then: @@ -242,13 +224,12 @@ class CollectFileOperatorTest extends Specification { result.B.name == 'B' result.B.text == 'world\n' - when: files = Channel .from(file1,file2,file3) .collectFile(skip:3, sort: 'index') .toList() - .getVal() + .unwrap() result = [:]; files.each{ result[it.name]=it } then: @@ -258,13 +239,12 @@ class CollectFileOperatorTest extends Specification { result.B.name == 'B' result.B.text == '' - when: files = Channel .from(file1,file2,file3) .collectFile(skip:10, sort: 'index') .toList() - .getVal() + .unwrap() result = [:]; files.each{ result[it.name]=it } then: @@ -273,11 +253,9 @@ class CollectFileOperatorTest extends Specification { result.B.name == 'B' result.B.text == '' - } def 'should collect file and keep header line' () { - given: def file1 = Files.createTempDirectory('temp').resolve('A') file1.deleteOnExit() @@ -291,13 +269,12 @@ class CollectFileOperatorTest extends Specification { file3.deleteOnExit() file3.text = 'HEADER\nxxx\nyyy\nzzz\n' - when: def files = Channel .from(file1,file2,file3) .collectFile(keepHeader:true, sort: 'index') .toList() - .getVal() + .unwrap() def result = [:]; files.each{ result[it.name]=it } then: @@ -306,7 +283,6 @@ class CollectFileOperatorTest extends Specification { result.B.name == 'B' result.B.text == '## HEAD ##\nHello\nworld\n' - } def 'check invalid options' () { @@ -339,6 +315,5 @@ class CollectFileOperatorTest extends Specification { new CollectFileOp(Mock(DataflowReadChannel), [seed: 'foo', keepHeader: true]) then: thrown(IllegalArgumentException) - } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/CollectOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/CollectOpTest.groovy index 84ca73d2be..2192f32ea8 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/CollectOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/CollectOpTest.groovy @@ -38,14 +38,14 @@ class CollectOpTest extends Specification { def source = Channel.of(1,2,3) def result = source.collect() then: - result.val == [1,2,3] - result.val instanceof ArrayBag + result.unwrap() == [1,2,3] + result.unwrap() instanceof ArrayBag when: source = Channel.empty() result = source.collect() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } @@ -56,64 +56,59 @@ class CollectOpTest extends Specification { when: def result = source.channel().collect { it.length() } then: - result.val == [5, 4, 7] - result.val instanceof ArrayBag + result.unwrap() == [5, 4, 7] + result.unwrap() instanceof ArrayBag } @Timeout(1) def 'should collect and flatten items'() { - given: def source = [[1,['a','b']], [3,['c','d']], [5,['p','q']]] when: def result = source.channel().collect() then: - result.val == [1,['a','b'],3,['c','d'],5,['p','q']] - result.val instanceof ArrayBag + result.unwrap() == [1,['a','b'],3,['c','d'],5,['p','q']] + result.unwrap() instanceof ArrayBag when: result = source.channel().collect(flat: true) then: - result.val == [1,['a','b'],3,['c','d'],5,['p','q']] - result.val instanceof ArrayBag + result.unwrap() == [1,['a','b'],3,['c','d'],5,['p','q']] + result.unwrap() instanceof ArrayBag when: result = source.channel().collect(flat: false) then: - result.val == [[1,['a','b']], [3,['c','d']], [5,['p','q']]] - result.val instanceof ArrayBag + result.unwrap() == [[1,['a','b']], [3,['c','d']], [5,['p','q']]] + result.unwrap() instanceof ArrayBag when: result = source.channel().collect { it.flatten() } then: - result.val == [1,'a','b',3,'c','d',5,'p','q'] - result.val instanceof ArrayBag - + result.unwrap() == [1,'a','b',3,'c','d',5,'p','q'] + result.unwrap() instanceof ArrayBag } @Timeout(1) def 'should collect items into a sorted list '() { - when: def result = [3,1,4,2].channel().collect(sort: true) then: - result.val == [1,2,3,4] - result.val instanceof ArrayBag + result.unwrap() == [1,2,3,4] + result.unwrap() instanceof ArrayBag when: result = ['aaa','bb', 'c'].channel().collect(sort: {it->it.size()} as Closure) then: - result.val == ['c','bb','aaa'] - result.val instanceof ArrayBag + result.unwrap() == ['c','bb','aaa'] + result.unwrap() instanceof ArrayBag when: result = ['aaa','bb', 'c'].channel().collect(sort: {a,b -> a.size()<=>b.size()} as Comparator) then: - result.val == ['c','bb','aaa'] - result.val instanceof ArrayBag - + result.unwrap() == ['c','bb','aaa'] + result.unwrap() instanceof ArrayBag } - } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/CombineOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/CombineOpTest.groovy index 2efe65faed..2fa27357ab 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/CombineOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/CombineOpTest.groovy @@ -31,13 +31,12 @@ class CombineOpTest extends Specification { new Session() } - def 'should make a tuple' () { given: def op = new CombineOp(Mock(DataflowQueue), Mock(DataflowQueue)) expect: - op.tuple(pivot, left,right) == result + op.tuple(pivot, left, right) == result where: pivot | left | right | result @@ -63,7 +62,7 @@ class CombineOpTest extends Specification { when: def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 9 ['a', 1] in all @@ -86,14 +85,13 @@ class CombineOpTest extends Specification { def op = new CombineOp(left,right) op.setPivot([0]) def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() println all then: true } def 'should combine a channel with a list' () { - given: def left = Channel.of('a','b') def right = [1,2,3,4] @@ -101,7 +99,7 @@ class CombineOpTest extends Specification { when: def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 8 ['a', 1] in all @@ -115,7 +113,6 @@ class CombineOpTest extends Specification { } def 'should combine a value with a list' () { - given: def left = Channel.value('x') def right = [1,2,3,4] @@ -123,7 +120,7 @@ class CombineOpTest extends Specification { when: def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 4 ['x', 1] in all @@ -133,7 +130,6 @@ class CombineOpTest extends Specification { } def 'should combine two values' () { - given: def left = Channel.value('x') def right = Channel.value('z') @@ -141,7 +137,7 @@ class CombineOpTest extends Specification { when: def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 1 ['x', 'z'] in all @@ -155,11 +151,10 @@ class CombineOpTest extends Specification { when: def result = op.apply() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } def 'should chain combine ops flat default' () { - given: def ch2 = Channel.of('a','b','c') def ch3 = Channel.of('x','y') @@ -167,7 +162,7 @@ class CombineOpTest extends Specification { when: def result = new CombineOp(new CombineOp(ch1, ch2).apply(), ch3).apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 12 @@ -184,11 +179,9 @@ class CombineOpTest extends Specification { [2,'b','y'] in all [2,'c','x'] in all [2,'c','y'] in all - } def 'should chain combine ops flat true' () { - given: def ch1 = Channel.of(1,2) def ch2 = Channel.of('a','b','c') @@ -196,7 +189,7 @@ class CombineOpTest extends Specification { when: def result = new CombineOp(new CombineOp(ch1, ch2).apply(), ch3).apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 12 @@ -216,13 +209,12 @@ class CombineOpTest extends Specification { } def 'should combine with tuples' () { - when: def left = Channel.of([1, 'x'], [2,'y'], [3, 'z']) def right = ['alpha','beta','gamma'] def result = new CombineOp(left, right).apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 9 @@ -238,7 +230,6 @@ class CombineOpTest extends Specification { [3, 'z', 'alpha'] in all [3, 'z', 'beta'] in all [3, 'z', 'gamma'] in all - } def 'should combine with map' () { @@ -247,7 +238,7 @@ class CombineOpTest extends Specification { def left = Channel.of([id:1, val:'x'], [id:2,val:'y'], [id:3, val:'z']) def right = ['alpha','beta','gamma'] def result = left.combine(right) - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 9 @@ -262,17 +253,13 @@ class CombineOpTest extends Specification { [[id:3, val:'z'], 'alpha'] in all [[id:3, val:'z'], 'beta'] in all [[id:3, val:'z'], 'gamma'] in all - } - - def 'should combine items'() { - when: def left = Channel.of(1,2,3) def right = ['a','b'] - def result = left.combine(right).toSortedList().val.iterator() + def result = left.combine(right).toSortedList().unwrap().iterator() then: result.next() == [1, 'a'] result.next() == [1, 'b'] @@ -284,7 +271,7 @@ class CombineOpTest extends Specification { when: left = Channel.of(1,2) right = Channel.of('a','b','c') - result = left.combine(right).toSortedList().val.iterator() + result = left.combine(right).toSortedList().unwrap().iterator() then: result.next() == [1, 'a'] result.next() == [1, 'b'] @@ -292,15 +279,13 @@ class CombineOpTest extends Specification { result.next() == [2, 'a'] result.next() == [2, 'b'] result.next() == [2, 'c'] - } def 'should chain combine'() { - when: def str1 = Channel.of('a','b','c') def str2 = Channel.of('x','y') - def result = Channel.of(1,2).combine(str1).combine(str2).toSortedList().val.iterator() + def result = Channel.of(1,2).combine(str1).combine(str2).toSortedList().unwrap().iterator() then: result.next() == [1,'a','x'] result.next() == [1,'a','y'] @@ -318,7 +303,7 @@ class CombineOpTest extends Specification { when: str1 = Channel.of('a','b','c') str2 = Channel.of('x','y') - result = Channel.of(1,2).combine(str1).combine(str2,flat:false).toSortedList().val.iterator() + result = Channel.of(1,2).combine(str1).combine(str2,flat:false).toSortedList().unwrap().iterator() then: result.next() == [1,'a','x'] result.next() == [1,'a','y'] @@ -335,7 +320,6 @@ class CombineOpTest extends Specification { } def 'should combine by first element' () { - given: def left = Channel.of( ['A',1], ['A',2], ['B',1], ['B',2] ) def right = Channel.of( ['A',1], ['A',2], ['B',1], ['B',2] ) @@ -344,7 +328,7 @@ class CombineOpTest extends Specification { def op = new CombineOp(left, right) op.pivot = 0 def result = op.apply() - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 8 @@ -367,7 +351,7 @@ class CombineOpTest extends Specification { when: def result = left.channel().combine(right.channel()) - def all = (List) ToListOp.apply(result).val + def all = (List) ToListOp.apply(result).unwrap() then: all.size() == 9 [1, 'a', 2, 'p'] in all @@ -384,7 +368,7 @@ class CombineOpTest extends Specification { when: result = left.channel().combine(right.channel(), by: 0) - all = (List) ToListOp.apply(result).val + all = (List) ToListOp.apply(result).unwrap() then: all.size() == 4 [1, 'a', 'r'] in all @@ -394,7 +378,7 @@ class CombineOpTest extends Specification { when: result = left.channel().combine(right.channel(), by: [0]) - all = (List) ToListOp.apply(result).val + all = (List) ToListOp.apply(result).unwrap() then: all.size() == 4 [1, 'a', 'r'] in all diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/ConcatOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/ConcatOpTest.groovy index 5a40b18927..1941932bb8 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/ConcatOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/ConcatOpTest.groovy @@ -25,7 +25,7 @@ import test.Dsl2Spec * @author Paolo Di Tommaso */ @Timeout(5) -class ConcatOp2Test extends Dsl2Spec { +class ConcatOpTest extends Dsl2Spec { def 'should concat two channel'() { @@ -36,13 +36,13 @@ class ConcatOp2Test extends Dsl2Spec { c1.concat(c2) ''') then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == 'a' - result.val == 'b' - result.val == 'c' - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 'a' + result.unwrap() == 'b' + result.unwrap() == 'c' + result.unwrap() == Channel.STOP } def 'should concat value with channel'() { @@ -53,10 +53,10 @@ class ConcatOp2Test extends Dsl2Spec { ch1.concat(ch2) ''') then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } def 'should concat two value channels'() { @@ -67,9 +67,9 @@ class ConcatOp2Test extends Dsl2Spec { ch1.concat(ch2) ''') then: - result.val == 1 - result.val == 2 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == Channel.STOP } def 'should concat with empty'() { @@ -80,8 +80,8 @@ class ConcatOp2Test extends Dsl2Spec { ch1.concat(ch2) ''') then: - result.val == 1 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == Channel.STOP when: result = dsl_eval(''' @@ -90,7 +90,7 @@ class ConcatOp2Test extends Dsl2Spec { ch1.concat(ch2) ''') then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/CountFastaOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/CountFastaOpTest.groovy index 5efe023050..2f6c9fcfd9 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/CountFastaOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/CountFastaOpTest.groovy @@ -28,7 +28,6 @@ import test.TestHelper class CountFastaOpTest extends Specification { def 'should count fasta channel' () { - given: def str = ''' >1aboA @@ -64,8 +63,7 @@ class CountFastaOpTest extends Specification { when: def result = Channel.of( str, str2 ).countFasta() then: - result.val == 8 - + result.unwrap() == 8 } def 'should count fasta records from files' () { @@ -113,7 +111,6 @@ class CountFastaOpTest extends Specification { when: def result = Channel.of( file1, file2 ).countFasta() then: - result.val == 10 - + result.unwrap() == 10 } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowHelperTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowHelperTest.groovy index 44c934d288..d63b1d499a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowHelperTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowHelperTest.groovy @@ -16,48 +16,24 @@ package nextflow.extension + import nextflow.Session import spock.lang.Specification import spock.lang.Unroll - /** * * @author Paolo Di Tommaso */ class DataflowHelperTest extends Specification { - def setupSpec() { new Session() } - def 'should subscribe handlers'() { - - when: - DataflowHelper.checkSubscribeHandlers( [:] ) - then: - thrown(IllegalArgumentException) - - when: - DataflowHelper.checkSubscribeHandlers( [ onNext:{}] ) - then: - true - - when: - DataflowHelper.checkSubscribeHandlers( [ onNext:{}, xxx:{}] ) - then: - thrown(IllegalArgumentException) - - when: - DataflowHelper.checkSubscribeHandlers( [ xxx:{}] ) - then: - thrown(IllegalArgumentException) - } - @Unroll def 'should split entry' () { when: - def pair = DataflowHelper.makeKey(pivot, entry) + def pair = DataflowHelper.makeKey(pivot, entry, null) then: pair.keys == keys pair.values == values @@ -70,4 +46,5 @@ class DataflowHelperTest extends Specification { [0,1,4] | ['A','B','C','D','E','F'] | ['A','B','E'] | ['C','D','F'] [0] | 'A' | ['A'] | [] } + } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMathExtensionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMathExtensionTest.groovy index afff091b8a..3e076c568e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMathExtensionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMathExtensionTest.groovy @@ -34,52 +34,47 @@ class DataflowMathExtensionTest extends Specification { Comparator makeComparator(Closure c) { c as Comparator } def 'should return the min value'() { - expect: - Channel.of(4,1,7,5).min().val == 1 - Channel.of("hello","hi","hey").min { it.size() } .val == "hi" - Channel.of("hello","hi","hey").min { a,b -> a.size()<=>b.size() } .val == "hi" - Channel.of("hello","hi","hey").min { a,b -> a.size()<=>b.size() } .val == "hi" - Channel.of("hello","hi","hey").min ( makeComparator({ a,b -> a.size()<=>b.size() }) ) .val == "hi" - + Channel.of(4,1,7,5).min().unwrap() == 1 + Channel.of("hello","hi","hey").min { it.size() } .unwrap() == "hi" + Channel.of("hello","hi","hey").min { a,b -> a.size()<=>b.size() } .unwrap() == "hi" + Channel.of("hello","hi","hey").min { a,b -> a.size()<=>b.size() } .unwrap() == "hi" + Channel.of("hello","hi","hey").min ( makeComparator({ a,b -> a.size()<=>b.size() }) ) .unwrap() == "hi" } def 'should return the max value'() { expect: - Channel.of(4,1,7,5).max().val == 7 - Channel.of("hello","hi","hey").max { it.size() } .val == "hello" - Channel.of("hello","hi","hey").max { a,b -> a.size()<=>b.size() } .val == "hello" - Channel.of("hello","hi","hey").max { a,b -> a.size()<=>b.size() } .val == "hello" + Channel.of(4,1,7,5).max().unwrap() == 7 + Channel.of("hello","hi","hey").max { it.size() } .unwrap() == "hello" + Channel.of("hello","hi","hey").max { a,b -> a.size()<=>b.size() } .unwrap() == "hello" + Channel.of("hello","hi","hey").max { a,b -> a.size()<=>b.size() } .unwrap() == "hello" // this may fail randomly - the cause should be investigated - Channel.of("hello","hi","hey").max (makeComparator({ a,b -> a.size()<=>b.size() })) .val == "hello" - + Channel.of("hello","hi","hey").max (makeComparator({ a,b -> a.size()<=>b.size() })) .unwrap() == "hello" } def 'should return the sum'() { expect: - Channel.of(4,1,7,5).sum().val == 17 - Channel.of(4,1,7,5).sum { it * 2 } .val == 34 - Channel.of( [1,1,1], [0,1,2], [10,20,30] ). sum() .val == [ 11, 22, 33 ] + Channel.of(4,1,7,5).sum().unwrap() == 17 + Channel.of(4,1,7,5).sum { it * 2 } .unwrap() == 34 + Channel.of( [1,1,1], [0,1,2], [10,20,30] ). sum() .unwrap() == [ 11, 22, 33 ] } - def 'should return the mean'() { expect: - Channel.of(10,20,30).mean().val == 20 - Channel.of(10,20,30).mean { it * 2 }.val == 40 - Channel.of( [10,20,30], [10, 10, 10 ], [10, 30, 50]).mean().val == [10, 20, 30] + Channel.of(10,20,30).mean().unwrap() == 20 + Channel.of(10,20,30).mean { it * 2 }.unwrap() == 40 + Channel.of( [10,20,30], [10, 10, 10 ], [10, 30, 50]).mean().unwrap() == [10, 20, 30] } def 'should convert string to integers' () { - expect: - Channel.value('11').toInteger().val == 11 + Channel.value('11').toInteger().unwrap() == 11 when: def list = Channel.of('1', '4\n', ' 7 ', '100' ) .toInteger() .toList() - .getVal() + .unwrap() then: list.size() == 4 @@ -93,17 +88,15 @@ class DataflowMathExtensionTest extends Specification { list[3] instanceof Integer } - def 'should convert string to long' () { - expect: - Channel.value('33').toLong().val == 33L + Channel.value('33').toLong().unwrap() == 33L when: def list = Channel.of('1', '4\n', ' 7 ', '100' ) .toLong() .toList() - .getVal() + .unwrap() then: list.size() == 4 @@ -118,16 +111,14 @@ class DataflowMathExtensionTest extends Specification { } def 'should convert string to float' () { - - expect: - Channel.value('99.1').toFloat().val == 99.1f + Channel.value('99.1').toFloat().unwrap() == 99.1f when: def list = Channel.of('1', '4\n', ' 7.5 ', '100.1' ) .toFloat() .toList() - .getVal() + .unwrap() then: list.size() == 4 @@ -142,15 +133,14 @@ class DataflowMathExtensionTest extends Specification { } def 'should convert string to double' () { - expect: - Channel.value('99.1').toDouble().val == 99.1d + Channel.value('99.1').toDouble().unwrap() == 99.1d when: def list = Channel.of('1', '4\n', ' 7.5 ', '100.1' ) .toDouble() .toList() - .getVal() + .unwrap() then: list.size() == 4 @@ -162,17 +152,15 @@ class DataflowMathExtensionTest extends Specification { list[1] instanceof Double list[2] instanceof Double list[3] instanceof Double - } @Retry def 'should return a random sample' () { - when: def result = Channel .of(0,1,2,3,4,5,6,7,8,9) .randomSample(5) - .toList().val as List + .toList().unwrap() as List then: result.size() == 5 @@ -183,7 +171,6 @@ class DataflowMathExtensionTest extends Specification { result[2] in 0..9 result[3] in 0..9 result[4] in 0..9 - } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMergeExtensionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMergeExtensionTest.groovy index 34940f48b8..b8d9db4a56 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMergeExtensionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowMergeExtensionTest.groovy @@ -48,10 +48,10 @@ class DataflowMergeExtensionTest extends Specification { def result = alpha.merge( beta, delta ) { a,b,c -> [c,b,a] } then: result instanceof DataflowQueue - result.val == [7,2,1] - result.val == [8,4,3] - result.val == [1,6,5] - result.val == Channel.STOP + result.unwrap() == [7,2,1] + result.unwrap() == [8,4,3] + result.unwrap() == [1,6,5] + result.unwrap() == Channel.STOP } def 'should merge with open array' () { @@ -62,10 +62,10 @@ class DataflowMergeExtensionTest extends Specification { def result = alpha.merge( beta, delta ) then: result instanceof DataflowQueue - result.val == [1,2,7] - result.val == [3,4,8] - result.val == [5,6,1] - result.val == Channel.STOP + result.unwrap() == [1,2,7] + result.unwrap() == [3,4,8] + result.unwrap() == [5,6,1] + result.unwrap() == Channel.STOP } def 'should merge with with default'() { @@ -76,10 +76,10 @@ class DataflowMergeExtensionTest extends Specification { def result = left.merge(right) then: result instanceof DataflowQueue - result.val == [1,2] - result.val == [3,4] - result.val == [5,6] - result.val == Channel.STOP + result.unwrap() == [1,2] + result.unwrap() == [3,4] + result.unwrap() == [5,6] + result.unwrap() == Channel.STOP when: left = Channel.of(1, 2, 3) @@ -87,10 +87,10 @@ class DataflowMergeExtensionTest extends Specification { result = left.merge(right) then: result instanceof DataflowQueue - result.val == [1, 'a','b'] - result.val == [2, 'p','q'] - result.val == [3, 'x','z'] - result.val == Channel.STOP + result.unwrap() == [1, 'a','b'] + result.unwrap() == [2, 'p','q'] + result.unwrap() == [3, 'x','z'] + result.unwrap() == Channel.STOP when: left = Channel.of('A','B','C') @@ -98,10 +98,10 @@ class DataflowMergeExtensionTest extends Specification { result = left.merge(right) then: result instanceof DataflowQueue - result.val == ['A', 'a', [1,2,3]] - result.val == ['B', 'b', [3,4,5]] - result.val == ['C', 'c', [6,7,8]] - result.val == Channel.STOP + result.unwrap() == ['A', 'a', [1,2,3]] + result.unwrap() == ['B', 'b', [3,4,5]] + result.unwrap() == ['C', 'c', [6,7,8]] + result.unwrap() == Channel.STOP } @@ -114,10 +114,10 @@ class DataflowMergeExtensionTest extends Specification { def result = alpha.merge( [beta, delta] ) { a,b,c -> [c,b,a] } then: result instanceof DataflowQueue - result.val == [7,2,1] - result.val == [8,4,3] - result.val == [1,6,5] - result.val == Channel.STOP + result.unwrap() == [7,2,1] + result.unwrap() == [8,4,3] + result.unwrap() == [1,6,5] + result.unwrap() == Channel.STOP } @@ -131,10 +131,10 @@ class DataflowMergeExtensionTest extends Specification { then: result instanceof DataflowQueue - result.val == [1,3] - result.val == [3,5] - result.val == [5,7] - result.val == Channel.STOP + result.unwrap() == [1,3] + result.unwrap() == [3,5] + result.unwrap() == [5,7] + result.unwrap() == Channel.STOP } def 'should merge with variables with custom closure'() { @@ -145,8 +145,8 @@ class DataflowMergeExtensionTest extends Specification { def result = alpha.merge(beta) { a,b -> [b, a] } then: result instanceof DataflowVariable - result.val == ['World', 'Hello'] - result.val == ['World', 'Hello'] + result.unwrap() == ['World', 'Hello'] + result.unwrap() == ['World', 'Hello'] } def 'should merge variables' () { @@ -156,8 +156,8 @@ class DataflowMergeExtensionTest extends Specification { def result = alpha.merge(beta) then: result instanceof DataflowVariable - result.val == ['Hello','World'] - result.val == ['Hello','World'] + result.unwrap() == ['Hello','World'] + result.unwrap() == ['Hello','World'] } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowTapExtensionTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowTapExtensionTest.groovy index fbd484aa3f..1459ab9486 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/DataflowTapExtensionTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/DataflowTapExtensionTest.groovy @@ -16,13 +16,11 @@ package nextflow.extension -import groovyx.gpars.dataflow.DataflowQueue -import groovyx.gpars.dataflow.DataflowVariable + import nextflow.Channel import nextflow.Session import spock.lang.Shared import spock.lang.Specification - /** * * @author Paolo Di Tommaso @@ -42,15 +40,15 @@ class DataflowTapExtensionTest extends Specification { when: def result = Channel.of( 4,7,9 ) .tap { first }.map { it+1 } then: - session.binding.first.val == 4 - session.binding.first.val == 7 - session.binding.first.val == 9 - session.binding.first.val == Channel.STOP + session.binding.first.unwrap() == 4 + session.binding.first.unwrap() == 7 + session.binding.first.unwrap() == 9 + session.binding.first.unwrap() == Channel.STOP - result.val == 5 - result.val == 8 - result.val == 10 - result.val == Channel.STOP + result.unwrap() == 5 + result.unwrap() == 8 + result.unwrap() == 10 + result.unwrap() == Channel.STOP !session.dag.isEmpty() @@ -61,19 +59,19 @@ class DataflowTapExtensionTest extends Specification { when: def result = Channel.of( 4,7,9 ) .tap { foo; bar }.map { it+1 } then: - session.binding.foo.val == 4 - session.binding.foo.val == 7 - session.binding.foo.val == 9 - session.binding.foo.val == Channel.STOP - session.binding.bar.val == 4 - session.binding.bar.val == 7 - session.binding.bar.val == 9 - session.binding.bar.val == Channel.STOP - - result.val == 5 - result.val == 8 - result.val == 10 - result.val == Channel.STOP + session.binding.foo.unwrap() == 4 + session.binding.foo.unwrap() == 7 + session.binding.foo.unwrap() == 9 + session.binding.foo.unwrap() == Channel.STOP + session.binding.bar.unwrap() == 4 + session.binding.bar.unwrap() == 7 + session.binding.bar.unwrap() == 9 + session.binding.bar.unwrap() == Channel.STOP + + result.unwrap() == 5 + result.unwrap() == 8 + result.unwrap() == 10 + result.unwrap() == Channel.STOP !session.dag.isEmpty() diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/GroupTupleOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/GroupTupleOpTest.groovy index 6e1910e1ef..b1f0633ec5 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/GroupTupleOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/GroupTupleOpTest.groovy @@ -17,11 +17,11 @@ package nextflow.extension import nextflow.Channel - -import spock.lang.Specification - import nextflow.Session - +import nextflow.extension.op.OpDatum +import nextflow.prov.OperatorRun +import nextflow.util.ArrayBag +import spock.lang.Specification /** * * @author Paolo Di Tommaso @@ -33,7 +33,6 @@ class GroupTupleOpTest extends Specification { } def 'should reuse the same key' () { - given: def key1 = ['a', 'b', 'c'] def key2 = ['a', 'b', 'c'] @@ -63,7 +62,6 @@ class GroupTupleOpTest extends Specification { } def 'should reuse same key with GroupSize' () { - given: def key1 = [new GroupKey('A',1) ] def key2 = [new GroupKey('A',1) ] @@ -80,7 +78,6 @@ class GroupTupleOpTest extends Specification { map.get(key2) == 1 map.get(key3) == null map.getOrCreate(key2) { return 10 } == 1 - } def 'should fetch groupsize' () { @@ -100,6 +97,33 @@ class GroupTupleOpTest extends Specification { 0 | [] } + def 'should unwrap tuple values' () { + given: + def r1 = new OperatorRun(new HashSet([1,2])) + def r2 = new OperatorRun(new HashSet([2,3])) + def d1 = OpDatum.of('A', r1) + def d2 = OpDatum.of('B', r1) + def d3 = OpDatum.of('C', r2) + def d4 = OpDatum.of('D', r2) + and: + def bag1 = new ArrayBag(d1, d2) + def bag2 = new ArrayBag(d3, d4) + def key = Mock(GroupKey) + and: + def tuple = [key, bag1, bag2] + + when: + def result = GroupTupleOp.unwrapValues(tuple) + then: + tuple[0] == key + and: + tuple[1] == new ArrayBag<>('A','B') + and: + tuple[2] == new ArrayBag<>('C', 'D') + and: + result.inputIds == [1,2,3] as Set + + } def 'should group items using dyn group size' () { given: @@ -115,29 +139,29 @@ class GroupTupleOpTest extends Specification { // here the size is defined as operator argument def result = tuples.channel().groupTuple(size: 2) then: - result.val == [k1, ['a', 'b'] ] - result.val == [k1, ['d', 'c'] ] - result.val == [k2, ['x', 'y'] ] - result.val == Channel.STOP + result.unwrap() == [k1, ['a', 'b'] ] + result.unwrap() == [k1, ['d', 'c'] ] + result.unwrap() == [k2, ['x', 'y'] ] + result.unwrap() == Channel.STOP when: // here the size is inferred by the key itself result = tuples.channel().groupTuple() then: - result.val == [k1, ['a', 'b'] ] - result.val == [k1, ['d', 'c'] ] - result.val == [k2, ['x', 'y', 'z'] ] - result.val == Channel.STOP + result.unwrap() == [k1, ['a', 'b'] ] + result.unwrap() == [k1, ['d', 'c'] ] + result.unwrap() == [k2, ['x', 'y', 'z'] ] + result.unwrap() == Channel.STOP when: result = tuples.channel().groupTuple(remainder: true) then: - result.val == [k1, ['a', 'b'] ] - result.val == [k1, ['d', 'c'] ] - result.val == [k2, ['x', 'y', 'z'] ] - result.val == [k3, ['q']] - result.val == [k1, ['f']] - result.val == Channel.STOP + result.unwrap() == [k1, ['a', 'b'] ] + result.unwrap() == [k1, ['d', 'c'] ] + result.unwrap() == [k2, ['x', 'y', 'z'] ] + result.unwrap() == [k3, ['q']] + result.unwrap() == [k1, ['f']] + result.unwrap() == Channel.STOP } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/JoinOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/JoinOpTest.groovy index 673686fdf9..34c4640231 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/JoinOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/JoinOpTest.groovy @@ -41,7 +41,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 3 result.contains( ['X', 1, 4] ) @@ -50,7 +50,6 @@ class JoinOpTest extends Specification { } - def 'should join entries by index' () { given: def ch1 = Channel.of([1, 'X'], [2, 'Y'], [3, 'Z'], [7, 'P']) @@ -58,7 +57,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [by:1]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 3 result.contains( ['X', 1, 4] ) @@ -73,7 +72,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [by:[1,2]]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 3 result.contains( ['a','b', 1, ['foo'], 4, [444]] ) @@ -90,7 +89,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [remainder: true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 5 result.contains( ['X', 1, 4] ) @@ -108,7 +107,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 3 result == [1,2,3] @@ -122,7 +121,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [remainder: true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 8 result == [1, 2, 3, 0, 0, 7, 8, 9] @@ -135,28 +134,25 @@ class JoinOpTest extends Specification { def right = Channel.empty() def result = left.join(right, remainder: true) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP - + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } def 'should join empty channel with pairs and remainder' () { - when: def left = Channel.of(['X', 1], ['Y', 2], ['Z', 3]) def right = Channel.empty() def result = left.join(right, remainder: true) then: - result.val == ['X', 1, null] - result.val == ['Y', 2, null] - result.val == ['Z', 3, null] - result.val == Channel.STOP + result.unwrap() == ['X', 1, null] + result.unwrap() == ['Y', 2, null] + result.unwrap() == ['Z', 3, null] + result.unwrap() == Channel.STOP } def 'should join a singleton value' () { - when: given: def ch1 = Channel.of( 1,2,3 ) @@ -164,35 +160,32 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result == [1] } - def 'should join pair with singleton and remainder' () { - when: def left = Channel.of(['P', 0], ['X', 1], ['Y', 2], ['Z', 3]) def right = Channel.of('X', 'Y', 'Z', 'Q') def result = left.join(right) then: - result.val == ['X', 1] - result.val == ['Y', 2] - result.val == ['Z', 3] - result.val == Channel.STOP + result.unwrap() == ['X', 1] + result.unwrap() == ['Y', 2] + result.unwrap() == ['Z', 3] + result.unwrap() == Channel.STOP when: left = Channel.of(['P', 0], ['X', 1], ['Y', 2], ['Z', 3]) right = Channel.of('X', 'Y', 'Z', 'Q') - result = left.join(right, remainder: true).toList().val.sort { it -> it[0] } + result = left.join(right, remainder: true).toList().unwrap().sort { it -> it[0] } then: result[2] == ['X', 1] result[3] == ['Y', 2] result[4] == ['Z', 3] result[0] == ['P', 0] result[1] == ['Q', null] - } def 'should match gstrings' () { @@ -201,12 +194,11 @@ class JoinOpTest extends Specification { def left = Channel.of(['A', 'hola'], ['B', 'hello'], ['C', 'ciao']) def right = Channel.of(["$A", 'mundo'], ["$B", 'world'], ["$C", 'mondo'] ) when: - def result = left.join(right).toList().val.sort { it[0] } + def result = left.join(right).toList().unwrap().sort { it[0] } then: result[0] == ['A','hola','mundo'] result[1] == ['B','hello','world'] result[2] == ['C','ciao','mondo'] - } def 'should be able to use identical ArrayBags join key' () { @@ -218,7 +210,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1 as DataflowReadChannel, ch2 as DataflowReadChannel) - List result = op.apply().toList().getVal() + List result = op.apply().toList().unwrap() then: !result.isEmpty() @@ -233,7 +225,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1 as DataflowReadChannel, ch2 as DataflowReadChannel) - List result = op.apply().toList().getVal() + List result = op.apply().toList().unwrap() then: result.isEmpty() @@ -246,7 +238,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [failOnMismatch:true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 2 result.contains( ['X', 1, 6] ) @@ -262,7 +254,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [failOnMismatch:true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() and: await(sess) then: @@ -305,7 +297,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [:]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: result.size() == 2 result.contains( ['X', 1, 2] ) @@ -321,7 +313,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [failOnDuplicate:true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() println "result=$result" and: await(sess) @@ -340,7 +332,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [failOnDuplicate:true, remainder: true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() and: await(sess) then: @@ -357,7 +349,7 @@ class JoinOpTest extends Specification { when: def op = new JoinOp(ch1, ch2, [failOnDuplicate:true]) - def result = op.apply().toList().getVal() + def result = op.apply().toList().unwrap() then: await(sess) then: @@ -365,7 +357,6 @@ class JoinOpTest extends Specification { sess.getError().message == 'Detected join operation duplicate emission on left channel -- offending element: key=X; value=3' } - protected void await(Session session) { def begin = System.currentTimeMillis() while( !session.isAborted() && System.currentTimeMillis()-begin<5_000 ) diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/MixOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/MixOpTest.groovy index b91765d6da..58593044d0 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/MixOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/MixOpTest.groovy @@ -34,7 +34,7 @@ class MixOpTest extends Dsl2Spec { c3 = Channel.value( 'z' ) c1.mix(c2,c3) - ''') .toList().val + ''') .toList().unwrap() then: 1 in result @@ -44,7 +44,6 @@ class MixOpTest extends Dsl2Spec { 'b' in result 'z' in result !('c' in result) - } def 'should mix with value channels'() { @@ -53,7 +52,7 @@ class MixOpTest extends Dsl2Spec { Channel.value(1).mix( Channel.fromList([2,3]) ) ''') then: - result.toList().val.sort() == [1,2,3] + result.toList().unwrap().sort() == [1,2,3] } def 'should mix with two singleton'() { @@ -62,7 +61,7 @@ class MixOpTest extends Dsl2Spec { Channel.value(1).mix( Channel.value(2) ) ''') then: - result.toList().val.sort() == [1,2] + result.toList().unwrap().sort() == [1,2] } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/MultiMapOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/MultiMapOpTest.groovy index 75dbb611f2..1b931886a1 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/MultiMapOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/MultiMapOpTest.groovy @@ -16,6 +16,7 @@ package nextflow.extension +import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable import org.junit.Rule @@ -43,24 +44,24 @@ class MultiMapOpTest extends Dsl2Spec { bar: it*it+2 baz: 3 } - ''') + ''') as List then: result.size() == 3 and: - result[0].val == 1 - result[0].val == 2 - result[0].val == 3 - result[0].val == Channel.STOP + result[0].unwrap() == 1 + result[0].unwrap() == 2 + result[0].unwrap() == 3 + result[0].unwrap() == Channel.STOP and: - result[1].val == 2 - result[1].val == 3 - result[1].val == 6 - result[1].val == Channel.STOP + result[1].unwrap() == 2 + result[1].unwrap() == 3 + result[1].unwrap() == 6 + result[1].unwrap() == Channel.STOP and: - result[2].val == 3 - result[2].val == 3 - result[2].val == 3 - result[2].val == Channel.STOP + result[2].unwrap() == 3 + result[2].unwrap() == 3 + result[2].unwrap() == 3 + result[2].unwrap() == Channel.STOP } @@ -75,25 +76,24 @@ class MultiMapOpTest extends Dsl2Spec { bar: p*p+2 baz: p-1 } - ''') + ''') as List then: result.size() == 3 and: - result[0].val == 1 - result[0].val == 2 - result[0].val == 3 - result[0].val == Channel.STOP + result[0].unwrap() == 1 + result[0].unwrap() == 2 + result[0].unwrap() == 3 + result[0].unwrap() == Channel.STOP and: - result[1].val == 2 - result[1].val == 3 - result[1].val == 6 - result[1].val == Channel.STOP + result[1].unwrap() == 2 + result[1].unwrap() == 3 + result[1].unwrap() == 6 + result[1].unwrap() == Channel.STOP and: - result[2].val == -1 - result[2].val == 0 - result[2].val == 1 - result[2].val == Channel.STOP - + result[2].unwrap() == -1 + result[2].unwrap() == 0 + result[2].unwrap() == 1 + result[2].unwrap() == Channel.STOP } def 'should pass criteria as argument' () { @@ -131,16 +131,15 @@ class MultiMapOpTest extends Dsl2Spec { foo: p.toUpperCase() bar: p.reverse() } - ''') + ''') as List then: result.size() == 2 and: result[0] instanceof DataflowVariable - result[0].val == 'HELLO' + result[0].unwrap() == 'HELLO' and: result[1] instanceof DataflowVariable - result[1].val == 'olleh' - + result[1].unwrap() == 'olleh' } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/OperatorDsl2Test.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/OperatorDsl2Test.groovy index 47caf5abe6..82af39dca7 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/OperatorDsl2Test.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/OperatorDsl2Test.groovy @@ -17,6 +17,7 @@ package nextflow.extension +import groovyx.gpars.dataflow.DataflowReadChannel import nextflow.Channel import spock.lang.Timeout import test.Dsl2Spec @@ -30,35 +31,35 @@ class OperatorDsl2Test extends Dsl2Spec { def 'should test unique' () { when: - def channel = dsl_eval(""" + def result = dsl_eval(""" Channel.of(1,2,3).unique() - """) + """) as DataflowReadChannel then: - channel.val == 1 - channel.val == 2 - channel.val == 3 - channel.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } def 'should test unique with value' () { when: - def channel = dsl_eval(""" + def result = dsl_eval(""" Channel.value(1).unique() - """) + """) as DataflowReadChannel then: - channel.val == 1 + result.unwrap() == 1 } def 'should test unique with collect' () { when: - def ch = dsl_eval(""" + def result = dsl_eval(""" Channel.of( 'a', 'b', 'c') .collect() .unique() .view() - """) + """) as DataflowReadChannel then: - ch.val == ['a','b','c'] + result.unwrap() == ['a','b','c'] } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/OperatorImplTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/OperatorImplTest.groovy index 10a96ba95a..29b8904088 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/OperatorImplTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/OperatorImplTest.groovy @@ -19,6 +19,7 @@ package nextflow.extension import java.nio.file.Paths import groovyx.gpars.dataflow.DataflowQueue +import groovyx.gpars.dataflow.DataflowReadChannel import groovyx.gpars.dataflow.DataflowVariable import nextflow.Channel import nextflow.Session @@ -36,64 +37,61 @@ class OperatorImplTest extends Specification { } def testFilter() { - when: - def c1 = Channel.of(1,2,3,4,5).filter { it > 3 } + DataflowReadChannel c1 = Channel.of(1,2,3,4,5).filter { it > 3 } then: - c1.val == 4 - c1.val == 5 - c1.val == Channel.STOP + c1.unwrap() == 4 + c1.unwrap() == 5 + c1.unwrap() == Channel.STOP when: def c2 = Channel.of('hola','hello','cioa','miao').filter { it =~ /^h.*/ } then: - c2.val == 'hola' - c2.val == 'hello' - c2.val == Channel.STOP + c2.unwrap() == 'hola' + c2.unwrap() == 'hello' + c2.unwrap() == Channel.STOP when: def c3 = Channel.of('hola','hello','cioa','miao').filter { it ==~ /^h.*/ } then: - c3.val == 'hola' - c3.val == 'hello' - c3.val == Channel.STOP + c3.unwrap() == 'hola' + c3.unwrap() == 'hello' + c3.unwrap() == Channel.STOP when: def c4 = Channel.of('hola','hello','cioa','miao').filter( ~/^h.*/ ) then: - c4.val == 'hola' - c4.val == 'hello' - c4.val == Channel.STOP + c4.unwrap() == 'hola' + c4.unwrap() == 'hello' + c4.unwrap() == Channel.STOP when: def c5 = Channel.of('hola',1,'cioa',2,3).filter( Number ) then: - c5.val == 1 - c5.val == 2 - c5.val == 3 - c5.val == Channel.STOP + c5.unwrap() == 1 + c5.unwrap() == 2 + c5.unwrap() == 3 + c5.unwrap() == Channel.STOP expect: - Channel.of(1,2,4,2,4,5,6,7,4).filter(1) .count().val == 1 - Channel.of(1,2,4,2,4,5,6,7,4).filter(2) .count().val == 2 - Channel.of(1,2,4,2,4,5,6,7,4).filter(4) .count().val == 3 - + Channel.of(1,2,4,2,4,5,6,7,4).filter(1) .count() .unwrap() == 1 + Channel.of(1,2,4,2,4,5,6,7,4).filter(2) .count() .unwrap() == 2 + Channel.of(1,2,4,2,4,5,6,7,4).filter(4) .count() .unwrap() == 3 } def testFilterWithValue() { expect: - Channel.value(3).filter { it>1 }.val == 3 - Channel.value(0).filter { it>1 }.val == Channel.STOP - Channel.value(Channel.STOP).filter { it>1 }.val == Channel.STOP + Channel.value(3).filter { it>1 }.unwrap() == 3 + Channel.value(0).filter { it>1 }.unwrap() == Channel.STOP + Channel.value(Channel.STOP).filter { it>1 }.unwrap() == Channel.STOP } def testSubscribe() { - when: def channel = Channel.create() int count = 0 channel.subscribe { count++; } << 1 << 2 << 3 - sleep(100) + sleep(200) then: count == 3 @@ -104,12 +102,9 @@ class OperatorImplTest extends Specification { sleep(100) then: count == 4 - - } def testSubscribe1() { - when: def count = 0 def done = false @@ -118,11 +113,9 @@ class OperatorImplTest extends Specification { then: done count == 3 - } def testSubscribe2() { - when: def count = 0 def done = false @@ -131,11 +124,9 @@ class OperatorImplTest extends Specification { then: done count == 1 - } def testSubscribeError() { - when: int next=0 int error=0 @@ -153,18 +144,16 @@ class OperatorImplTest extends Specification { error == 1 // complete never complete == 0 - } - def testMap() { when: def result = Channel.of(1,2,3).map { "Hello $it" } then: - result.val == 'Hello 1' - result.val == 'Hello 2' - result.val == 'Hello 3' - result.val == Channel.STOP + result.unwrap() == 'Hello 1' + result.unwrap() == 'Hello 2' + result.unwrap() == 'Hello 3' + result.unwrap() == Channel.STOP } def testMapWithVariable() { @@ -173,448 +162,406 @@ class OperatorImplTest extends Specification { when: def result = variable.map { it.reverse() } then: - result.val == 'olleH' - result.val == 'olleH' - result.val == 'olleH' + result.unwrap() == 'olleH' + result.unwrap() == 'olleH' + result.unwrap() == 'olleH' } def testMapParamExpanding () { - when: def result = Channel.of(1,2,3).map { [it, it] }.map { x, y -> x+y } then: - result.val == 2 - result.val == 4 - result.val == 6 - result.val == Channel.STOP + result.unwrap() == 2 + result.unwrap() == 4 + result.unwrap() == 6 + result.unwrap() == Channel.STOP } def testSkip() { - when: def result = Channel.of(1,2,3).map { it == 2 ? Channel.VOID : "Hello $it" } then: - result.val == 'Hello 1' - result.val == 'Hello 3' - result.val == Channel.STOP - + result.unwrap() == 'Hello 1' + result.unwrap() == 'Hello 3' + result.unwrap() == Channel.STOP } - def testMapMany () { - when: def result = Channel.of(1,2,3).flatMap { it -> [it, it*2] } then: - result.val == 1 - result.val == 2 - result.val == 2 - result.val == 4 - result.val == 3 - result.val == 6 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 2 + result.unwrap() == 4 + result.unwrap() == 3 + result.unwrap() == 6 + result.unwrap() == Channel.STOP } def testMapManyWithSingleton() { - when: def result = Channel.value([1,2,3]).flatMap() then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.empty().flatMap() then: - result.val == Channel.STOP - + result.unwrap() == Channel.STOP } def testMapManyWithTuples () { - when: def result = Channel.of( [1,2], ['a','b'] ).flatMap { it -> [it, it.reverse()] } then: - result.val == [1,2] - result.val == [2,1] - result.val == ['a','b'] - result.val == ['b','a'] - result.val == Channel.STOP + result.unwrap() == [1, 2] + result.unwrap() == [2, 1] + result.unwrap() == ['a', 'b'] + result.unwrap() == ['b', 'a'] + result.unwrap() == Channel.STOP } def testMapManyDefault () { - when: def result = Channel.of( [1,2], ['a',['b','c']] ).flatMap() then: - result.val == 1 - result.val == 2 - result.val == 'a' - result.val == ['b','c'] // <-- nested list are preserved - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 'a' + result.unwrap() == ['b', 'c'] // <-- nested list are preserved + result.unwrap() == Channel.STOP } def testMapManyWithHashArray () { - when: def result = Channel.of(1,2,3).flatMap { it -> [ k: it, v: it*2] } then: - result.val == new MapEntry('k',1) - result.val == new MapEntry('v',2) - result.val == new MapEntry('k',2) - result.val == new MapEntry('v',4) - result.val == new MapEntry('k',3) - result.val == new MapEntry('v',6) - result.val == Channel.STOP - + result.unwrap() == new MapEntry('k',1) + result.unwrap() == new MapEntry('v',2) + result.unwrap() == new MapEntry('k',2) + result.unwrap() == new MapEntry('v',4) + result.unwrap() == new MapEntry('k',3) + result.unwrap() == new MapEntry('v',6) + result.unwrap() == Channel.STOP } - - def testReduce() { - when: def channel = Channel.create() def result = channel.reduce { a, e -> a += e } channel << 1 << 2 << 3 << 4 << 5 << Channel.STOP then: - result.getVal() == 15 - + result.unwrap() == 15 when: channel = Channel.of(1,2,3,4,5) result = channel.reduce { a, e -> a += e } then: - result.getVal() == 15 + result.unwrap() == 15 when: channel = Channel.create() result = channel.reduce { a, e -> a += e } channel << 99 << Channel.STOP then: - result.getVal() == 99 + result.unwrap() == 99 when: channel = Channel.create() result = channel.reduce { a, e -> a += e } channel << Channel.STOP then: - result.getVal() == null + result.unwrap() == null when: result = Channel.of(6,5,4,3,2,1).reduce { a, e -> Channel.STOP } then: - result.val == 6 - + result.unwrap() == 6 } - def testReduceWithSeed() { - when: def channel = Channel.create() def result = channel.reduce (1) { a, e -> a += e } channel << 1 << 2 << 3 << 4 << 5 << Channel.STOP then: - result.getVal() == 16 + result.unwrap() == 16 when: channel = Channel.create() result = channel.reduce (10) { a, e -> a += e } channel << Channel.STOP then: - result.getVal() == 10 + result.unwrap() == 10 when: result = Channel.of(6,5,4,3,2,1).reduce(0) { a, e -> a < 3 ? a+1 : Channel.STOP } then: - result.val == 3 - + result.unwrap() == 3 } def testFirst() { - expect: - Channel.of(3,6,4,5,4,3,4).first().val == 3 + Channel.of(3,6,4,5,4,3,4).first().unwrap() == 3 } def testFirstWithCriteria() { expect: - Channel.of(3,6,4,5,4,3,4).first{ it>4 } .val == 6 + Channel.of(3,6,4,5,4,3,4).first{ it>4 }.unwrap() == 6 } def testFirstWithValue() { - expect: - Channel.value(3).first().val == 3 - Channel.value(3).first{ it>1 }.val == 3 - Channel.value(3).first{ it>3 }.val == Channel.STOP - Channel.value(Channel.STOP).first { it>3 }.val == Channel.STOP + Channel.value(3).first().unwrap() == 3 + Channel.value(3).first{ it>1 }.unwrap() == 3 + Channel.value(3).first{ it>3 }.unwrap() == Channel.STOP + Channel.value(Channel.STOP).first { it>3 }.unwrap() == Channel.STOP } - def testFirstWithCondition() { - expect: - Channel.of(3,6,4,5,4,3,4).first { it % 2 == 0 } .val == 6 - Channel.of( 'a', 'b', 'c', 1, 2 ).first( Number ) .val == 1 - Channel.of( 'a', 'b', 1, 2, 'aaa', 'bbb' ).first( ~/aa.*/ ) .val == 'aaa' - Channel.of( 'a', 'b', 1, 2, 'aaa', 'bbb' ).first( 1 ) .val == 1 - + Channel.of(3,6,4,5,4,3,4).first { it % 2 == 0 }.unwrap() == 6 + Channel.of( 'a', 'b', 'c', 1, 2 ).first( Number ).unwrap() == 1 + Channel.of( 'a', 'b', 1, 2, 'aaa', 'bbb' ).first( ~/aa.*/ ).unwrap() == 'aaa' + Channel.of( 'a', 'b', 1, 2, 'aaa', 'bbb' ).first( 1 ).unwrap() == 1 } - def testTake() { - when: def result = Channel.of(1,2,3,4,5,6).take(3) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.of(1).take(3) then: - result.val == 1 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == Channel.STOP when: result = Channel.of(1,2,3).take(0) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP when: result = Channel.of(1,2,3).take(-1) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.of(1,2,3).take(3) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP - + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } def testLast() { - expect: - Channel.of(3,6,4,5,4,3,9).last().val == 9 - Channel.value('x').last().val == 'x' + Channel.of(3,6,4,5,4,3,9).last().unwrap() == 9 + Channel.value('x').last().unwrap() == 'x' } - - - def testCount() { expect: - Channel.of(4,1,7,5).count().val == 4 - Channel.of(4,1,7,1,1).count(1).val == 3 - Channel.of('a','c','c','q','b').count ( ~/c/ ) .val == 2 - Channel.value(5).count().val == 1 - Channel.value(5).count(5).val == 1 - Channel.value(5).count(6).val == 0 + Channel.of(4,1,7,5).count().unwrap() == 4 + Channel.of(4,1,7,1,1).count(1).unwrap() == 3 + Channel.of('a','c','c','q','b').count ( ~/c/ ).unwrap() == 2 + Channel.value(5).count().unwrap() == 1 + Channel.value(5).count(5).unwrap() == 1 + Channel.value(5).count(6).unwrap() == 0 } def testToList() { - when: def channel = Channel.of(1,2,3) then: - channel.toList().val == [1,2,3] + channel.toList().unwrap() == [1, 2, 3] when: channel = Channel.create() channel << Channel.STOP then: - channel.toList().val == [] + channel.toList().unwrap() == [] when: channel = Channel.value(1) then: - channel.toList().val == [1] + channel.toList().unwrap() == [1] when: channel = Channel.empty() then: - channel.toList().val == [] + channel.toList().unwrap() == [] } def testToSortedList() { - when: def channel = Channel.of(3,1,4,2) then: - channel.toSortedList().val == [1,2,3,4] + channel.toSortedList().unwrap() == [1, 2, 3, 4] when: - channel = Channel.create() - channel << Channel.STOP + channel = Channel.empty() then: - channel.toSortedList().val == [] + channel.toSortedList().unwrap() == [] when: channel = Channel.of([1,'zeta'], [2,'gamma'], [3,'alpaha'], [4,'delta']) then: - channel.toSortedList { it[1] } .val == [[3,'alpaha'], [4,'delta'], [2,'gamma'], [1,'zeta'] ] + channel.toSortedList { it[1] }.unwrap() == [[3, 'alpaha'], [4, 'delta'], [2, 'gamma'], [1, 'zeta'] ] when: channel = Channel.value(1) then: - channel.toSortedList().val == [1] + channel.toSortedList().unwrap() == [1] when: channel = Channel.empty() then: - channel.toSortedList().val == [] - + channel.toSortedList().unwrap() == [] } - def testUnique() { expect: - Channel.of(1,1,1,5,7,7,7,3,3).unique().toList().val == [1,5,7,3] - Channel.of(1,3,4,5).unique { it%2 } .toList().val == [1,4] + Channel.of(1,1,1,5,7,7,7,3,3).unique().toList().unwrap() == [1, 5, 7, 3] + Channel.of(1,3,4,5).unique { it%2 } .toList().unwrap() == [1, 4] and: - Channel.of(1).unique().val == 1 - Channel.value(1).unique().val == 1 + Channel.of(1).unique().unwrap() == 1 + Channel.value(1).unique().unwrap() == 1 } def testDistinct() { expect: - Channel.of(1,1,2,2,2,3,1,1,2,2,3).distinct().toList().val == [1,2,3,1,2,3] - Channel.of(1,1,2,2,2,3,1,1,2,4,6).distinct { it%2 } .toList().val == [1,2,3,2] + Channel.of(1,1,2,2,2,3,1,1,2,2,3).distinct().toList().unwrap() == [1, 2, 3, 1, 2, 3] + Channel.of(1,1,2,2,2,3,1,1,2,4,6).distinct { it%2 } .toList().unwrap() == [1, 2, 3, 2] } - def testFlatten() { - when: def r1 = Channel.of(1,2,3).flatten() then: - r1.val == 1 - r1.val == 2 - r1.val == 3 - r1.val == Channel.STOP + r1.unwrap() == 1 + r1.unwrap() == 2 + r1.unwrap() == 3 + r1.unwrap() == Channel.STOP when: def r2 = Channel.of([1,'a'], [2,'b']).flatten() then: - r2.val == 1 - r2.val == 'a' - r2.val == 2 - r2.val == 'b' - r2.val == Channel.STOP + r2.unwrap() == 1 + r2.unwrap() == 'a' + r2.unwrap() == 2 + r2.unwrap() == 'b' + r2.unwrap() == Channel.STOP when: def r3 = Channel.of( [1,2] as Integer[], [3,4] as Integer[] ).flatten() then: - r3.val == 1 - r3.val == 2 - r3.val == 3 - r3.val == 4 - r3.val == Channel.STOP + r3.unwrap() == 1 + r3.unwrap() == 2 + r3.unwrap() == 3 + r3.unwrap() == 4 + r3.unwrap() == Channel.STOP when: def r4 = Channel.of( [1,[2,3]], 4, [5,[6]] ).flatten() then: - r4.val == 1 - r4.val == 2 - r4.val == 3 - r4.val == 4 - r4.val == 5 - r4.val == 6 - r4.val == Channel.STOP + r4.unwrap() == 1 + r4.unwrap() == 2 + r4.unwrap() == 3 + r4.unwrap() == 4 + r4.unwrap() == 5 + r4.unwrap() == 6 + r4.unwrap() == Channel.STOP } def testFlattenWithSingleton() { when: def result = Channel.value([3,2,1]).flatten() then: - result.val == 3 - result.val == 2 - result.val == 1 - result.val == Channel.STOP + result.unwrap() == 3 + result.unwrap() == 2 + result.unwrap() == 1 + result.unwrap() == Channel.STOP when: result = Channel.empty().flatten() then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } def testCollate() { - when: def r1 = Channel.of(1,2,3,1,2,3,1).collate( 2, false ) then: - r1.val == [1,2] - r1.val == [3,1] - r1.val == [2,3] - r1.val == Channel.STOP + r1.unwrap() == [1, 2] + r1.unwrap() == [3, 1] + r1.unwrap() == [2, 3] + r1.unwrap() == Channel.STOP when: def r2 = Channel.of(1,2,3,1,2,3,1).collate( 3 ) then: - r2.val == [1,2,3] - r2.val == [1,2,3] - r2.val == [1] - r2.val == Channel.STOP - + r2.unwrap() == [1, 2, 3] + r2.unwrap() == [1, 2, 3] + r2.unwrap() == [1] + r2.unwrap() == Channel.STOP } def testCollateWithStep() { - when: def r1 = Channel.of(1,2,3,4).collate( 3, 1, false ) then: - r1.val == [1,2,3] - r1.val == [2,3,4] - r1.val == Channel.STOP + r1.unwrap() == [1, 2, 3] + r1.unwrap() == [2, 3, 4] + r1.unwrap() == Channel.STOP when: def r2 = Channel.of(1,2,3,4).collate( 3, 1, true ) then: - r2.val == [1,2,3] - r2.val == [2,3,4] - r2.val == [3,4] - r2.val == [4] - r2.val == Channel.STOP + r2.unwrap() == [1, 2, 3] + r2.unwrap() == [2, 3, 4] + r2.unwrap() == [3, 4] + r2.unwrap() == [4] + r2.unwrap() == Channel.STOP when: - def r3 = Channel.of(1,2,3,4).collate( 3, 1 ) + def r3 = Channel.of(1,2,3,4).collate( 3, 1 ) then: - r3.val == [1,2,3] - r3.val == [2,3,4] - r3.val == [3,4] - r3.val == [4] - r3.val == Channel.STOP + r3.unwrap() == [1, 2, 3] + r3.unwrap() == [2, 3, 4] + r3.unwrap() == [3, 4] + r3.unwrap() == [4] + r3.unwrap() == Channel.STOP when: def r4 = Channel.of(1,2,3,4).collate( 4,4 ) then: - r4.val == [1,2,3,4] - r4.val == Channel.STOP + r4.unwrap() == [1, 2, 3, 4] + r4.unwrap() == Channel.STOP when: def r5 = Channel.of(1,2,3,4).collate( 6,6 ) then: - r5.val == [1,2,3,4] - r5.val == Channel.STOP + r5.unwrap() == [1, 2, 3, 4] + r5.unwrap() == Channel.STOP when: def r6 = Channel.of(1,2,3,4).collate( 6,6,false ) then: - r6.val == Channel.STOP - + r6.unwrap() == Channel.STOP } def testCollateIllegalArgs() { @@ -637,32 +584,31 @@ class OperatorImplTest extends Specification { Channel.create().collate(1,0) then: thrown(IllegalArgumentException) - } def testCollateWithValueChannel() { when: def result = Channel.value(1).collate(1) then: - result.val == [1] - result.val == Channel.STOP + result.unwrap() == [1] + result.unwrap() == Channel.STOP when: result = Channel.value(1).collate(10) then: - result.val == [1] - result.val == Channel.STOP + result.unwrap() == [1] + result.unwrap() == Channel.STOP when: result = Channel.value(1).collate(10, true) then: - result.val == [1] - result.val == Channel.STOP + result.unwrap() == [1] + result.unwrap() == Channel.STOP when: result = Channel.value(1).collate(10, false) then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP } def testMix() { @@ -670,7 +616,7 @@ class OperatorImplTest extends Specification { def c1 = Channel.of( 1,2,3 ) def c2 = Channel.of( 'a','b' ) def c3 = Channel.value( 'z' ) - def result = c1.mix(c2,c3).toList().val + def result = c1.mix(c2,c3).toList().unwrap() then: 1 in result @@ -680,20 +626,16 @@ class OperatorImplTest extends Specification { 'b' in result 'z' in result !('c' in result) - } def testMixWithSingleton() { when: def result = Channel.value(1).mix( Channel.of(2,3) ) then: - result.toList().val.sort() == [1,2,3] + result.toList().unwrap().sort() == [1, 2, 3] } - - def testDefaultMappingClosure() { - expect: OperatorImpl.DEFAULT_MAPPING_CLOSURE.call( [7, 8, 9] ) == 7 OperatorImpl.DEFAULT_MAPPING_CLOSURE.call( [7, 8, 9], 2 ) == 9 @@ -727,12 +669,10 @@ class OperatorImplTest extends Specification { OperatorImpl.DEFAULT_MAPPING_CLOSURE.call( 99 ) == 99 OperatorImpl.DEFAULT_MAPPING_CLOSURE.call( 99, 2 ) == null - } def testCross() { - setup: def ch1 = Channel.of( [1, 'x'], [2,'y'], [3,'z'] ) def ch2 = Channel.of( [1,11], [1,13], [2,21],[2,22], [2,23], [4,1], [4,2] ) @@ -741,17 +681,15 @@ class OperatorImplTest extends Specification { def result = ch1.cross(ch2) then: - result.val == [ [1, 'x'], [1,11] ] - result.val == [ [1, 'x'], [1,13] ] - result.val == [ [2, 'y'], [2,21] ] - result.val == [ [2, 'y'], [2,22] ] - result.val == [ [2, 'y'], [2,23] ] - result.val == Channel.STOP - + result.unwrap() == [[1, 'x'], [1, 11] ] + result.unwrap() == [[1, 'x'], [1, 13] ] + result.unwrap() == [[2, 'y'], [2, 21] ] + result.unwrap() == [[2, 'y'], [2, 22] ] + result.unwrap() == [[2, 'y'], [2, 23] ] + result.unwrap() == Channel.STOP } def testCross2() { - setup: def ch1 = Channel.create() def ch2 = Channel.of ( ['PF00006', 'PF00006_mafft.aln'], ['PF00006', 'PF00006_clustalo.aln']) @@ -761,15 +699,13 @@ class OperatorImplTest extends Specification { def result = ch1.cross(ch2) then: - result.val == [ ['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_mafft.aln'] ] - result.val == [ ['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_clustalo.aln'] ] - result.val == Channel.STOP - + result.unwrap() == [['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_mafft.aln'] ] + result.unwrap() == [['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_clustalo.aln'] ] + result.unwrap() == Channel.STOP } def testCross3() { - setup: def ch1 = Channel.of(['PF00006', 'PF00006.sp_lib']) def ch2 = Channel.create ( ) @@ -779,27 +715,25 @@ class OperatorImplTest extends Specification { def result = ch1.cross(ch2) then: - result.val == [ ['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_mafft.aln'] ] - result.val == [ ['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_clustalo.aln'] ] - result.val == Channel.STOP - + result.unwrap() == [['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_mafft.aln'] ] + result.unwrap() == [['PF00006', 'PF00006.sp_lib'], ['PF00006', 'PF00006_clustalo.aln'] ] + result.unwrap() == Channel.STOP } def testConcat() { - when: def c1 = Channel.of(1,2,3) def c2 = Channel.of('a','b','c') def all = c1.concat(c2) then: - all.val == 1 - all.val == 2 - all.val == 3 - all.val == 'a' - all.val == 'b' - all.val == 'c' - all.val == Channel.STOP + all.unwrap() == 1 + all.unwrap() == 2 + all.unwrap() == 3 + all.unwrap() == 'a' + all.unwrap() == 'b' + all.unwrap() == 'c' + all.unwrap() == Channel.STOP when: def d1 = Channel.create() @@ -811,55 +745,50 @@ class OperatorImplTest extends Specification { Thread.start { sleep 100; d1 << 1 << 2 << Channel.STOP } then: - result.val == 1 - result.val == 2 - result.val == 'a' - result.val == 'b' - result.val == 'c' - result.val == 'p' - result.val == 'q' - result.val == Channel.STOP - + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 'a' + result.unwrap() == 'b' + result.unwrap() == 'c' + result.unwrap() == 'p' + result.unwrap() == 'q' + result.unwrap() == Channel.STOP } def testContactWithSingleton() { when: def result = Channel.value(1).concat( Channel.of(2,3) ) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } - def testGroupTuple() { - when: def result = Channel .from([1,'a'], [1,'b'], [2,'x'], [3, 'q'], [1,'c'], [2, 'y'], [3, 'q']) .groupTuple() then: - result.val == [1, ['a', 'b','c'] ] - result.val == [2, ['x', 'y'] ] - result.val == [3, ['q', 'q'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'b','c'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == [3, ['q', 'q'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithCount() { - when: def result = Channel .from([1,'a'], [1,'b'], [2,'x'], [3, 'q'], [1,'d'], [1,'c'], [2, 'y'], [1,'f']) .groupTuple(size: 2) then: - result.val == [1, ['a', 'b'] ] - result.val == [1, ['d', 'c'] ] - result.val == [2, ['x', 'y'] ] - result.val == Channel.STOP + result.unwrap() == [1, ['a', 'b'] ] + result.unwrap() == [1, ['d', 'c'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == Channel.STOP when: result = Channel @@ -867,27 +796,25 @@ class OperatorImplTest extends Specification { .groupTuple(size: 2, remainder: true) then: - result.val == [1, ['a', 'b'] ] - result.val == [1, ['d', 'c'] ] - result.val == [2, ['x', 'y'] ] - result.val == [3, ['q']] - result.val == [1, ['f']] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'b'] ] + result.unwrap() == [1, ['d', 'c'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == [3, ['q']] + result.unwrap() == [1, ['f']] + result.unwrap() == Channel.STOP } def testGroupTupleWithSortNatural() { - when: def result = Channel .from([1,'z'], [1,'w'], [1,'a'], [1,'b'], [2, 'y'], [2,'x'], [3, 'q'], [1,'c'], [3, 'p']) .groupTuple(sort: true) then: - result.val == [1, ['a', 'b','c','w','z'] ] - result.val == [2, ['x','y'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP + result.unwrap() == [1, ['a', 'b','c','w','z'] ] + result.unwrap() == [2, ['x','y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP when: result = Channel @@ -895,77 +822,66 @@ class OperatorImplTest extends Specification { .groupTuple(sort: 'natural') then: - result.val == [1, ['a', 'b','c','w','z'] ] - result.val == [2, ['x','y'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'b','c','w','z'] ] + result.unwrap() == [2, ['x','y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP } - def testGroupTupleWithSortHash() { - when: def result = Channel .from([1,'z'], [1,'w'], [1,'a'], [1,'b'], [2, 'y'], [2,'x'], [3, 'q'], [1,'c'], [3, 'p']) .groupTuple(sort: 'hash') then: - result.val == [1, ['a', 'c','z','b','w'] ] - result.val == [2, ['y','x'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'c','z','b','w'] ] + result.unwrap() == [2, ['y','x'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithComparator() { - when: def result = Channel .from([1,'z'], [1,'w'], [1,'a'], [1,'b'], [2, 'y'], [2,'x'], [3, 'q'], [1,'c'], [3, 'p']) .groupTuple(sort: { o1, o2 -> o2<=>o1 } as Comparator ) then: - result.val == [1, ['z','w','c','b','a'] ] - result.val == [2, ['y','x'] ] - result.val == [3, ['q','p'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['z','w','c','b','a'] ] + result.unwrap() == [2, ['y','x'] ] + result.unwrap() == [3, ['q','p'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithClosureWithSingle() { - when: def result = Channel .from([1,'z'], [1,'w'], [1,'a'], [1,'b'], [2, 'y'], [2,'x'], [3, 'q'], [1,'c'], [3, 'p']) .groupTuple(sort: { it -> it } ) then: - result.val == [1, ['a', 'b','c','w','z'] ] - result.val == [2, ['x','y'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'b','c','w','z'] ] + result.unwrap() == [2, ['x','y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithComparatorWithPair() { - when: def result = Channel .from([1,'z'], [1,'w'], [1,'a'], [1,'b'], [2, 'y'], [2,'x'], [3, 'q'], [1,'c'], [3, 'p']) .groupTuple(sort: { o1, o2 -> o2<=>o1 } ) then: - result.val == [1, ['z','w','c','b','a'] ] - result.val == [2, ['y','x'] ] - result.val == [3, ['q','p'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['z','w','c','b','a'] ] + result.unwrap() == [2, ['y','x'] ] + result.unwrap() == [3, ['q','p'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithIndex () { - given: def file1 = Paths.get('/path/file_1') def file2 = Paths.get('/path/file_2') @@ -977,10 +893,10 @@ class OperatorImplTest extends Specification { .groupTuple(by: 2) then: - result.val == [ [1,3,3], ['a','q','q'], file1 ] - result.val == [ [1,2], ['b','x'], file2 ] - result.val == [ [1,2], ['c','y'], file3 ] - result.val == Channel.STOP + result.unwrap() == [ [1,3,3], ['a','q','q'], file1 ] + result.unwrap() == [ [1,2], ['b','x'], file2 ] + result.unwrap() == [ [1,2], ['c','y'], file3 ] + result.unwrap() == Channel.STOP when: @@ -989,10 +905,10 @@ class OperatorImplTest extends Specification { .groupTuple(by: [2]) then: - result.val == [ [1,3,3], ['a','q','q'], file1 ] - result.val == [ [1,2], ['b','x'], file2 ] - result.val == [ [1,2], ['c','y'], file3 ] - result.val == Channel.STOP + result.unwrap() == [ [1,3,3], ['a','q','q'], file1 ] + result.unwrap() == [ [1,2], ['b','x'], file2 ] + result.unwrap() == [ [1,2], ['c','y'], file3 ] + result.unwrap() == Channel.STOP when: @@ -1001,17 +917,15 @@ class OperatorImplTest extends Specification { .groupTuple(by: [0,2]) then: - result.val == [ 1, ['a','q'], file1 ] - result.val == [ 1, ['b','c','z'], file2 ] - result.val == [ 2, ['x','y'], file2 ] - result.val == [ 3, ['y','c'], file3 ] - result.val == [ 3, ['q'], file1 ] - result.val == Channel.STOP - + result.unwrap() == [ 1, ['a','q'], file1 ] + result.unwrap() == [ 1, ['b','c','z'], file2 ] + result.unwrap() == [ 2, ['x','y'], file2 ] + result.unwrap() == [ 3, ['y','c'], file3 ] + result.unwrap() == [ 3, ['q'], file1 ] + result.unwrap() == Channel.STOP } def testGroupTupleWithNotMatchingCardinality() { - when: def result = Channel .of([1,'a'], @@ -1024,15 +938,13 @@ class OperatorImplTest extends Specification { .groupTuple() then: - result.val == [1, ['a', 'b', 'c'], ['d'] ] - result.val == [2, ['x', 'y'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP - + result.unwrap() == [1, ['a', 'b', 'c'], ['d'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithNotMatchingCardinalityAndFixedSize() { - when: def result = Channel .of([1,'a'], @@ -1045,14 +957,13 @@ class OperatorImplTest extends Specification { .groupTuple(size:2) then: - result.val == [1, ['a', 'b'] ] - result.val == [2, ['x', 'y'] ] - result.val == [3, ['p', 'q'] ] - result.val == Channel.STOP + result.unwrap() == [1, ['a', 'b'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == Channel.STOP } def testGroupTupleWithNotMatchingCardinalityAndFixedSizeAndRemainder() { - when: def result = Channel .of([1,'a'], @@ -1065,64 +976,57 @@ class OperatorImplTest extends Specification { .groupTuple(size:2, remainder: true) then: - result.val == [1, ['a', 'b'] ] - result.val == [2, ['x', 'y'] ] - result.val == [3, ['p', 'q'] ] - result.val == [1, ['c'], ['d']] - result.val == Channel.STOP + result.unwrap() == [1, ['a', 'b'] ] + result.unwrap() == [2, ['x', 'y'] ] + result.unwrap() == [3, ['p', 'q'] ] + result.unwrap() == [1, ['c'], ['d']] + result.unwrap() == Channel.STOP } def testChannelIfEmpty() { - - def result - when: - result = Channel.of(1,2,3).ifEmpty(100) + def result = Channel.of(1,2,3).ifEmpty(100) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.empty().ifEmpty(100) then: - result.val == 100 - result.val == Channel.STOP + result.unwrap() == 100 + result.unwrap() == Channel.STOP when: result = Channel.empty().ifEmpty { 1+2 } then: - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 3 + result.unwrap() == Channel.STOP when: result = Channel.value(1).ifEmpty(100) then: result instanceof DataflowVariable - result.val == 1 + result.unwrap() == 1 when: result = Channel.empty().ifEmpty(100) then: result instanceof DataflowQueue - result.val == 100 - + result.unwrap() == 100 } def 'should create a channel given a list'() { - when: def result = [10,20,30].channel() then: - result.val == 10 - result.val == 20 - result.val == 30 - result.val == Channel.STOP - + result.unwrap() == 10 + result.unwrap() == 20 + result.unwrap() == 30 + result.unwrap() == Channel.STOP } - def 'should assign a channel to new variable' () { given: def session = new Session() @@ -1133,40 +1037,36 @@ class OperatorImplTest extends Specification { .set { result } then: - session.binding.result.val == 12 - session.binding.result.val == 22 - session.binding.result.val == 32 - session.binding.result.val == Channel.STOP - + session.binding.result.unwrap() == 12 + session.binding.result.unwrap() == 22 + session.binding.result.unwrap() == 32 + session.binding.result.unwrap() == Channel.STOP } def 'should always the same value' () { - when: - def x = Channel.value('Hello') + def result = Channel.value('Hello') then: - x.val == 'Hello' - x.val == 'Hello' - x.val == 'Hello' + result.unwrap() == 'Hello' + result.val == 'Hello' + result.val == 'Hello' } def 'should emit channel items until the condition is verified' () { - when: def result = Channel.of(1,2,3,4).until { it == 3 } then: - result.val == 1 - result.val == 2 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == Channel.STOP when: result = Channel.of(1,2,3).until { it == 5 } then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP - + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } @@ -1178,10 +1078,9 @@ class OperatorImplTest extends Specification { Channel.value('Hello').set { result } then: - session.binding.result.val == 'Hello' - session.binding.result.val == 'Hello' - session.binding.result.val == 'Hello' - + session.binding.result.unwrap() == 'Hello' + session.binding.result.unwrap() == 'Hello' + session.binding.result.unwrap() == 'Hello' } def 'should assign queue channel to a new variable' () { @@ -1192,10 +1091,10 @@ class OperatorImplTest extends Specification { Channel.of(1,2,3).set { result } then: - session.binding.result.val == 1 - session.binding.result.val == 2 - session.binding.result.val == 3 - session.binding.result.val == Channel.STOP + session.binding.result.unwrap() == 1 + session.binding.result.unwrap() == 2 + session.binding.result.unwrap() == 3 + session.binding.result.unwrap() == Channel.STOP } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/RandomSampleTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/RandomSampleTest.groovy index 9ce69c5e91..6212d2cb24 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/RandomSampleTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/RandomSampleTest.groovy @@ -37,7 +37,7 @@ class RandomSampleTest extends Specification { def sampler = new RandomSampleOp(ch, 10) when: - def result = (List)sampler.apply().toList().val + def result = (List)sampler.apply().toList().unwrap() then: result.size() == 10 result.unique().size() == 10 @@ -52,7 +52,7 @@ class RandomSampleTest extends Specification { def sampler = new RandomSampleOp(ch, 20) when: - def result = (List)sampler.apply().toList().val + def result = (List)sampler.apply().toList().unwrap() then: result.size() == 10 result.unique().size() == 10 @@ -66,7 +66,7 @@ class RandomSampleTest extends Specification { def sampler = new RandomSampleOp(ch, 10) when: - def result = (List)sampler.apply().toList().val + def result = (List)sampler.apply().toList().unwrap() then: result.size() == 10 result.unique().size() == 10 @@ -83,8 +83,8 @@ class RandomSampleTest extends Specification { def secondSampler = new RandomSampleOp(ch2, 10, seed) when: - def resultFirstRun = (List)firstSampler.apply().toList().val - def resultSecondRun = (List)secondSampler.apply().toList().val + def resultFirstRun = (List)firstSampler.apply().toList().unwrap() + def resultSecondRun = (List)secondSampler.apply().toList().unwrap() then: resultFirstRun == resultSecondRun diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/SetOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/SetOpTest.groovy index 3620277fb6..5accad0636 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/SetOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/SetOpTest.groovy @@ -33,9 +33,9 @@ class SetOpTest extends Dsl2Spec { foo | map { it *2 } /) then: - result.val == 2 - result.val == 4 - result.val == 6 + result.unwrap() == 2 + result.unwrap() == 4 + result.unwrap() == 6 when: result = dsl_eval(/ @@ -43,7 +43,7 @@ class SetOpTest extends Dsl2Spec { foo | map { it *2 } /) then: - result.val == 10 + result.unwrap() == 10 } def 'should invoke set with dot notation' () { @@ -53,9 +53,9 @@ class SetOpTest extends Dsl2Spec { foo.map { it *2 } /) then: - result.val == 2 - result.val == 4 - result.val == 6 + result.unwrap() == 2 + result.unwrap() == 4 + result.unwrap() == 6 when: result = dsl_eval(/ @@ -63,7 +63,7 @@ class SetOpTest extends Dsl2Spec { foo.map { it.toUpperCase() } /) then: - result.val == 'HELLO' + result.unwrap() == 'HELLO' } @@ -77,7 +77,7 @@ class SetOpTest extends Dsl2Spec { return foo /) then: - result.val == 'X' + result.unwrap() == 'X' when: result = dsl_eval(/ @@ -88,8 +88,8 @@ class SetOpTest extends Dsl2Spec { return bar /) then: - result[0].val == 'X' - result[1].val == 'Y' + result[0].unwrap() == 'X' + result[1].unwrap() == 'Y' } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastaOperatorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastaOperatorTest.groovy index 74612ff488..65f0f38c6e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastaOperatorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastaOperatorTest.groovy @@ -87,11 +87,11 @@ class SplitFastaOperatorTest extends Specification { given: def records = Channel.of(fasta1, fasta2).splitFasta(record:[id:true]) expect: - records.val == [id:'1aboA'] - records.val == [id:'1ycsB'] - records.val == [id:'1pht'] - records.val == [id:'alpha123'] - records.val == Channel.STOP + records.unwrap() == [id:'1aboA'] + records.unwrap() == [id:'1ycsB'] + records.unwrap() == [id:'1pht'] + records.unwrap() == [id:'alpha123'] + records.unwrap() == Channel.STOP } @@ -104,11 +104,11 @@ class SplitFastaOperatorTest extends Specification { .map{ record, code -> [record.id, code] } expect: - result.val == ['1aboA', 'one'] - result.val == ['1ycsB', 'one'] - result.val == ['1pht', 'one'] - result.val == ['alpha123', 'two'] - result.val == Channel.STOP + result.unwrap() == ['1aboA', 'one'] + result.unwrap() == ['1ycsB', 'one'] + result.unwrap() == ['1pht', 'one'] + result.unwrap() == ['alpha123', 'two'] + result.unwrap() == Channel.STOP } @@ -119,11 +119,11 @@ class SplitFastaOperatorTest extends Specification { Channel.of(fasta1,fasta2).splitFasta(record:[id:true], into: target) expect: - target.val == [id:'1aboA'] - target.val == [id:'1ycsB'] - target.val == [id:'1pht'] - target.val == [id:'alpha123'] - target.val == Channel.STOP + target.unwrap() == [id:'1aboA'] + target.unwrap() == [id:'1ycsB'] + target.unwrap() == [id:'1pht'] + target.unwrap() == [id:'alpha123'] + target.unwrap() == Channel.STOP } @@ -154,10 +154,10 @@ class SplitFastaOperatorTest extends Specification { when: Channel.of(F1,F3).splitFasta(by:2, into: target) then: - target.val == '>1\nAAA\n>2\nBBB\n' - target.val == '>3\nCCC\n' - target.val == '>1\nEEE\n>2\nFFF\n' - target.val == '>3\nGGG\n' + target.unwrap() == '>1\nAAA\n>2\nBBB\n' + target.unwrap() == '>3\nCCC\n' + target.unwrap() == '>1\nEEE\n>2\nFFF\n' + target.unwrap() == '>3\nGGG\n' } def 'should apply count on multiple entries with a limit'() { @@ -195,9 +195,9 @@ class SplitFastaOperatorTest extends Specification { when: Channel.of(F1,F3).splitFasta(by:2, limit:4, into: target) then: - target.val == '>1\nAAA\n>2\nBBB\n' - target.val == '>3\nCCC\n>4\nDDD\n' - target.val == '>1\nEEE\n>2\nFFF\n' - target.val == '>3\nGGG\n>4\nHHH\n' + target.unwrap() == '>1\nAAA\n>2\nBBB\n' + target.unwrap() == '>3\nCCC\n>4\nDDD\n' + target.unwrap() == '>1\nEEE\n>2\nFFF\n' + target.unwrap() == '>3\nGGG\n>4\nHHH\n' } } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastqOperatorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastqOperatorTest.groovy index 6046f85308..aaca44fe80 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastqOperatorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/SplitFastqOperatorTest.groovy @@ -79,7 +79,7 @@ class SplitFastqOperatorTest extends Specification { when: def target = Channel.of(READS).splitFastq(by:2) then: - target.val == ''' + target.unwrap() == ''' @SRR636272.19519409/1 GGCCCGGCAGCAGGATGATGCTCTCCCGGGCCAAGCCGGCTGTGGGGAGCACCCCGCCGCAGGGGGACAGGCGGAGGAAGAAAGGGAAGAAGGTGCCACAGATCG + @@ -91,7 +91,7 @@ class SplitFastqOperatorTest extends Specification { ''' .stripIndent().leftTrim() - target.val == ''' + target.unwrap() == ''' @SRR636272.21107783/1 CGGGGAGCGCGGGCCCGGCAGCAGGATGATGCTCTCCCGGGCCAAGCCGGCTGTAGGGAGCACCCCGCCGCAGGGGGACAGGCGAGATCGGAAGAGCACACGTCT + @@ -103,7 +103,7 @@ class SplitFastqOperatorTest extends Specification { ''' .stripIndent().leftTrim() - target.val == Channel.STOP + target.unwrap() == Channel.STOP } def 'should split a fastq to gzip chunks' () { @@ -114,7 +114,7 @@ class SplitFastqOperatorTest extends Specification { when: def target = Channel.of(READS).splitFastq(by:2, compress:true, file:folder) then: - gunzip(target.val) == ''' + gunzip(target.unwrap()) == ''' @SRR636272.19519409/1 GGCCCGGCAGCAGGATGATGCTCTCCCGGGCCAAGCCGGCTGTGGGGAGCACCCCGCCGCAGGGGGACAGGCGGAGGAAGAAAGGGAAGAAGGTGCCACAGATCG + @@ -126,7 +126,7 @@ class SplitFastqOperatorTest extends Specification { ''' .stripIndent().leftTrim() - gunzip(target.val) == ''' + gunzip(target.unwrap()) == ''' @SRR636272.21107783/1 CGGGGAGCGCGGGCCCGGCAGCAGGATGATGCTCTCCCGGGCCAAGCCGGCTGTAGGGAGCACCCCGCCGCAGGGGGACAGGCGAGATCGGAAGAGCACACGTCT + @@ -138,7 +138,7 @@ class SplitFastqOperatorTest extends Specification { ''' .stripIndent().leftTrim() - target.val == Channel.STOP + target.unwrap() == Channel.STOP cleanup: folder.deleteDir() @@ -147,7 +147,7 @@ class SplitFastqOperatorTest extends Specification { def 'should split read pairs' () { when: - def result = Channel.of(['sample_id',READS,READS2]).splitFastq(by:1, elem:[1,2]).toList().val + def result = Channel.of(['sample_id',READS,READS2]).splitFastq(by:1, elem:[1,2]).toList().unwrap() then: result.size() ==4 @@ -218,7 +218,7 @@ class SplitFastqOperatorTest extends Specification { when: channel = Channel.of(['sample_id',file1,file2]).splitFastq(by:1, pe:true) - result = channel.val + result = channel.unwrap() then: result[0] == 'sample_id' result[1] == ''' @@ -235,7 +235,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'sample_id' result[1] == ''' @@ -252,7 +252,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'sample_id' result[1] == ''' @@ -269,7 +269,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'sample_id' result[1] == ''' @@ -286,7 +286,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result == Channel.STOP @@ -307,7 +307,7 @@ class SplitFastqOperatorTest extends Specification { channel = Channel .from([ ['aaa_id',file_a_1,file_a_2], ['bbb_id',file_b_1,file_b_2] ]) .splitFastq(by:1, pe:true, file:folder) - result = channel.val + result = channel.unwrap() then: result[0] == 'aaa_id' result[1].name == 'aaa_1.1.fq' @@ -326,7 +326,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'aaa_id' result[1].name == 'aaa_1.2.fq' @@ -345,7 +345,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'aaa_id' result[1].name == 'aaa_1.3.fq' @@ -364,7 +364,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'aaa_id' result[1].name == 'aaa_1.4.fq' @@ -383,7 +383,7 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'bbb_id' result[1].name == 'bbb_1.1.fq' @@ -402,28 +402,28 @@ class SplitFastqOperatorTest extends Specification { '''.stripIndent().leftTrim() when: - result = channel.val + result = channel.unwrap() then: result[0] == 'bbb_id' result[1].name == 'bbb_1.2.fq' result[2].name == 'bbb_2.2.fq' when: - result = channel.val + result = channel.unwrap() then: result[0] == 'bbb_id' result[1].name == 'bbb_1.3.fq' result[2].name == 'bbb_2.3.fq' when: - result = channel.val + result = channel.unwrap() then: result[0] == 'bbb_id' result[1].name == 'bbb_1.4.fq' result[2].name == 'bbb_2.4.fq' when: - result = channel.val + result = channel.unwrap() then: result == Channel.STOP sleep 1_000 diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/UntilManyOpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/UntilManyOpTest.groovy index c64bee82ce..3625199211 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/UntilManyOpTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/UntilManyOpTest.groovy @@ -19,10 +19,13 @@ package nextflow.extension import nextflow.Channel import spock.lang.Specification +import spock.lang.Timeout + /** * * @author Paolo Di Tommaso */ +@Timeout(10) class UntilManyOpTest extends Specification { def 'should emit channel items until the condition is verified' () { @@ -31,18 +34,18 @@ class UntilManyOpTest extends Specification { def source = Channel.of(1,2,3,4) def result = new UntilManyOp([source], { it==3 }).apply().get(0) then: - result.val == 1 - result.val == 2 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == Channel.STOP when: source = Channel.of(1,2,3) result = new UntilManyOp([source], { it==5 }).apply().get(0) then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP } @@ -57,17 +60,17 @@ class UntilManyOpTest extends Specification { def (X,Y,Z) = new UntilManyOp([A,B,C], condition).apply() then: - X.val == 1 - Y.val == 'alpha' - Z.val == 'foo' + X.unwrap() == 1 + Y.unwrap() == 'alpha' + Z.unwrap() == 'foo' and: - X.val == 2 - Y.val == 'beta' - Z.val == 'bar' + X.unwrap() == 2 + Y.unwrap() == 'beta' + Z.unwrap() == 'bar' and: - X.val == Channel.STOP - Y.val == Channel.STOP - Z.val == Channel.STOP + X.unwrap() == Channel.STOP + Y.unwrap() == Channel.STOP + Z.unwrap() == Channel.STOP } def 'should emit channels until list condition is verified' () { @@ -81,18 +84,17 @@ class UntilManyOpTest extends Specification { def (X,Y,Z) = new UntilManyOp([A,B,C], condition).apply() then: - X.val == 1 - Y.val == 'alpha' - Z.val == 'foo' + X.unwrap() == 1 + Y.unwrap() == 'alpha' + Z.unwrap() == 'foo' and: - X.val == 2 - Y.val == 'beta' - Z.val == 'bar' + X.unwrap() == 2 + Y.unwrap() == 'beta' + Z.unwrap() == 'bar' and: - X.val == Channel.STOP - Y.val == Channel.STOP - Z.val == Channel.STOP + X.unwrap() == Channel.STOP + Y.unwrap() == Channel.STOP + Z.unwrap() == Channel.STOP } - } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/ViewOperatorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/ViewOperatorTest.groovy index 12f6c6511b..82a564027d 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/ViewOperatorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/ViewOperatorTest.groovy @@ -44,10 +44,10 @@ class ViewOperatorTest extends Specification{ when: def result = Channel.of(1,2,3).view() then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP capture.toString() == '1\n2\n3\n' } @@ -57,10 +57,10 @@ class ViewOperatorTest extends Specification{ when: def result = Channel.of(1,2,3).view { "~ $it " } then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP capture.toString() == '~ 1 \n~ 2 \n~ 3 \n' @@ -71,10 +71,10 @@ class ViewOperatorTest extends Specification{ when: def result = Channel.of(1,2,3).view(newLine:false) { " ~ $it" } then: - result.val == 1 - result.val == 2 - result.val == 3 - result.val == Channel.STOP + result.unwrap() == 1 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == Channel.STOP capture.toString() == ' ~ 1 ~ 2 ~ 3' } @@ -83,7 +83,7 @@ class ViewOperatorTest extends Specification{ when: def result = Channel.value(1).view { ">> $it" } then: - result.val == 1 + result.unwrap() == 1 capture.toString() == ">> 1\n" } @@ -91,7 +91,7 @@ class ViewOperatorTest extends Specification{ when: def result = Channel.value(Channel.STOP).view { ">> $it" } then: - result.val == Channel.STOP + result.unwrap() == Channel.STOP capture.toString() == '' } diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/op/OpClosureTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/op/OpClosureTest.groovy new file mode 100644 index 0000000000..d672001822 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/extension/op/OpClosureTest.groovy @@ -0,0 +1,54 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + + +import spock.lang.Specification +/** + * + * @author Paolo Di Tommaso + */ +class OpClosureTest extends Specification { + + def 'should invoke target closure' () { + given: + def code = { a,b -> a+b } + def context = new ContextSequential() + def wrapper = new OpClosure(code, context) + + when: + def result = wrapper.call(1,2) + then: + result == 3 + and: + context.getOperatorRun() != null + } + + def 'should instrument a closure'() { + given: + def code = { int x, int y -> x+y } + def v1 = 1 + def v2 = 2 + + when: + def c = new OpClosure(code, new ContextSequential()) + def z = c.call([v1, v2] as Object[]) + then: + z == 3 + } +} diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/op/OpTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/op/OpTest.groovy new file mode 100644 index 0000000000..123c4372d1 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/extension/op/OpTest.groovy @@ -0,0 +1,81 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.extension.op + +import groovyx.gpars.dataflow.DataflowQueue +import groovyx.gpars.dataflow.operator.DataflowEventListener +import spock.lang.Specification + +/** + * + * @author Paolo Di Tommaso + */ +class OpTest extends Specification { + + def 'should validate operator params' () { + when: + def p1 = new Op().toMap() + then: + p1.inputs == List.of() + p1.outputs == List.of() + p1.listeners == List.of() + + when: + def s1 = new DataflowQueue() + def t1 = new DataflowQueue() + def l1 = Mock(DataflowEventListener) + def c1 = { return 'foo' } + and: + def p2 = new Op() + .withInput(s1) + .withOutput(t1) + .withListener(l1) + .withCode(c1) + then: + p2.inputs == List.of(s1) + p2.outputs == List.of(t1) + p2.listeners == List.of(l1) + p2.context instanceof ContextSequential + p2.code == c1 + and: + p2.toMap().inputs == List.of(s1) + p2.toMap().outputs == List.of(t1) + p2.toMap().listeners == List.of(l1) + + when: + def s2 = new DataflowQueue() + def t2 = new DataflowQueue() + def l2 = Mock(DataflowEventListener) + and: + def p3 = new Op() + .withInputs([s1,s2]) + .withOutputs([t1,t2]) + .withListeners([l1,l2]) + .withContext(new ContextGrouping()) + then: + p3.inputs == List.of(s1,s2) + p3.outputs == List.of(t1,t2) + p3.listeners == List.of(l1,l2) + p3.context instanceof ContextGrouping + and: + p3.toMap().inputs == List.of(s1,s2) + p3.toMap().outputs == List.of(t1,t2) + p3.toMap().listeners == List.of(l1,l2) + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/plugin/ChannelFactoryInstanceTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/plugin/ChannelFactoryInstanceTest.groovy index ea2fb42193..a040a8b188 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/plugin/ChannelFactoryInstanceTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/plugin/ChannelFactoryInstanceTest.groovy @@ -98,10 +98,10 @@ class ChannelFactoryInstanceTest extends Specification { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 'one' - result.val == 'two' - result.val == 'three' - result.val == Channel.STOP + result.unwrap() == 'one' + result.unwrap() == 'two' + result.unwrap() == 'three' + result.unwrap() == Channel.STOP and: ext1.initCount == 1 ext1.initSession instanceof Session @@ -128,10 +128,10 @@ class ChannelFactoryInstanceTest extends Specification { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 'one' - result.val == 'two' - result.val == 'three' - result.val == Channel.STOP + result.unwrap() == 'one' + result.unwrap() == 'two' + result.unwrap() == 'three' + result.unwrap() == Channel.STOP and: ext1.initCount == 1 ext1.initSession instanceof Session @@ -178,7 +178,7 @@ class ChannelFactoryInstanceTest extends Specification { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == ['1 X', '2 Y', '3 Z'] + result.unwrap() == ['1 X', '2 Y', '3 Z'] and: ext1.initCount == 1 @@ -209,9 +209,9 @@ class ChannelFactoryInstanceTest extends Specification { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 2 - result.val == 3 - result.val == 4 + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 4 and: ext1.initCount == 1 diff --git a/modules/nextflow/src/test/groovy/nextflow/extension/plugin/PluginExtensionProviderTest.groovy b/modules/nextflow/src/test/groovy/nextflow/extension/plugin/PluginExtensionProviderTest.groovy index a32ebdc287..54a71530b8 100644 --- a/modules/nextflow/src/test/groovy/nextflow/extension/plugin/PluginExtensionProviderTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/extension/plugin/PluginExtensionProviderTest.groovy @@ -55,8 +55,8 @@ class PluginExtensionProviderTest extends Specification { def result = ext.invokeExtensionMethod(ch, 'map', { it -> it * it }) then: result instanceof DataflowReadChannel - result.val == 1 - result.val == 4 - result.val == 9 + result.unwrap() == 1 + result.unwrap() == 4 + result.unwrap() == 9 } } diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskBeanTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskBeanTest.groovy index 4cfcb2a1fd..f2e16a0616 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskBeanTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskBeanTest.groovy @@ -54,7 +54,7 @@ class TaskBeanTest extends Specification { config.stageOutMode = 'rsync' def task = Mock(TaskRun) - task.getId() >> '123' + task.getId() >> TaskId.of('123') task.getName() >> 'Hello' task.getStdin() >> 'input from stdin' task.getScratch() >> '/tmp/x' diff --git a/modules/nextflow/src/test/groovy/nextflow/processor/TaskProcessorTest.groovy b/modules/nextflow/src/test/groovy/nextflow/processor/TaskProcessorTest.groovy index 751feeb03f..814ecf8a68 100644 --- a/modules/nextflow/src/test/groovy/nextflow/processor/TaskProcessorTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/processor/TaskProcessorTest.groovy @@ -901,40 +901,40 @@ class TaskProcessorTest extends Specification { when: processor.fairBindOutputs0(emission3, task3) then: - processor.@fairBuffers[2] == emission3 + processor.@fairBuffers[2] == new TaskProcessor.FairEntry(emission3,task3) 0 * processor.bindOutputs0(_) when: processor.fairBindOutputs0(emission2, task2) then: - processor.@fairBuffers[1] == emission2 + processor.@fairBuffers[1] == new TaskProcessor.FairEntry(emission2,task2) 0 * processor.bindOutputs0(_) when: processor.fairBindOutputs0(emission5, task5) then: - processor.@fairBuffers[4] == emission5 + processor.@fairBuffers[4] == new TaskProcessor.FairEntry(emission5, task5) 0 * processor.bindOutputs0(_) when: processor.fairBindOutputs0(emission1, task1) then: - 1 * processor.bindOutputs0(emission1) + 1 * processor.bindOutputs0(emission1, task1) then: - 1 * processor.bindOutputs0(emission2) + 1 * processor.bindOutputs0(emission2, task2) then: - 1 * processor.bindOutputs0(emission3) + 1 * processor.bindOutputs0(emission3, task3) and: processor.@fairBuffers.size() == 2 processor.@fairBuffers[0] == null - processor.@fairBuffers[1] == emission5 + processor.@fairBuffers[1] == new TaskProcessor.FairEntry(emission5, task5) when: processor.fairBindOutputs0(emission4, task4) then: - 1 * processor.bindOutputs0(emission4) + 1 * processor.bindOutputs0(emission4, task4) then: - 1 * processor.bindOutputs0(emission5) + 1 * processor.bindOutputs0(emission5, task5) then: processor.@fairBuffers.size()==0 } diff --git a/modules/nextflow/src/test/groovy/nextflow/prov/ProvTest.groovy b/modules/nextflow/src/test/groovy/nextflow/prov/ProvTest.groovy new file mode 100644 index 0000000000..125f07700e --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/prov/ProvTest.groovy @@ -0,0 +1,1057 @@ +package nextflow.prov + +import static test.TestHelper.* + +import nextflow.config.ConfigParser +import nextflow.processor.TaskId +import nextflow.processor.TaskProcessor +import spock.lang.Ignore +import spock.lang.Timeout +import test.Dsl2Spec +/** + * + * @author Paolo Di Tommaso + */ +@Timeout(5) +class ProvTest extends Dsl2Spec { + + def setup() { + Prov.clear() + TaskId.clear() + TaskProcessor.allTasks.clear() + } + + ConfigObject globalConfig() { + new ConfigParser().parse(''' + process.fair = true + ''') + } + + def 'should chain two process'() { + + when: + dsl_eval(globalConfig(), ''' + workflow { + p1 | map { x-> x } | map { x-> x+1 } | p2 + } + + process p1 { + output: val(x) + exec: + x =1 + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1'] + } + + def 'should track provenance with multiMap operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) | p1 + + p1.out.multiMap { v -> + foo: v + 1 + bar: v * v + } + .set { result } + + result.foo | p2 + result.bar | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (3)'] + and: + upstreamTasksOf('p3 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p3 (2)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p3 (3)') + .name == ['p1 (3)'] + } + + def 'should track provenance with branch operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,10,20) \ + | p1 \ + | branch { left: it <=10; right: it >10 } \ + | set { result } + + result.left | p2 + result.right | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x+1 + } + + process p2 { + input: val(x) + exec: + println x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p3 (1)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p3 (2)') + .name == ['p1 (3)'] + } + + def 'should track provenance with flatMap operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2) \ + | p1 \ + | flatMap \ + | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = [x, x*x] + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p2 (4)') + .name == ['p1 (2)'] + } + + def 'should track the provenance of two processes and reduce operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) \ + | p1 \ + | reduce {a,b -> return a+b} \ + | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + } + + def 'should track the provenance of two tasks and collectFile operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of('a','b','c') \ + | p1 \ + | collectFile(name: 'sample.txt') \ + | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: file(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + + } + + def 'should track the provenance of two tasks and toList operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of('a','b','c') | p1 | toList | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + + } + + def 'should track the provenance of two tasks and toSortedList operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of('a','b','c') | p1 | toList | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + + } + + def 'should track provenance with filter operator'() { + + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,3,2) | p1 | filter { x-> x<30 } | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x*10 + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + then: + upstreamTasksOf('p2 (2)') + .name == ['p1 (3)'] + } + + def 'should track provenance with unique operator'() { + + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,2,3) | p1 | unique | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + then: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + then: + upstreamTasksOf('p2 (3)') + .name == ['p1 (4)'] + } + + + def 'should track provenance with distinct operator'() { + + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,2,2,3) | p1 | unique | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + def upstream1 = upstreamTasksOf('p2 (1)') + upstream1.size() == 1 + upstream1.first.name == 'p1 (1)' + + then: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + + then: + upstreamTasksOf('p2 (3)') + .name == ['p1 (5)'] + } + + def 'should track provenance with first operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) | p1 | first | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)'] + } + + def 'should track provenance with take operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3,4,5) | p1 | take(2) | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + then: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + + } + + def 'should track provenance with last operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3,4,5) | p1 | last | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (5)'] + } + + def 'should track provenance with collect operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) | p1 | collect | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + } + + def 'should track provenance with collate operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1, 2, 3, 4) | p1 | collate( 3, 1 ) | p2 + } + + // + // the "collate" emits the following values: + // + // [1, 2, 3] + // [2, 3, 4] + // [3, 4] + // [4] + // + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println "$task.process ($task.index) = ${x}" + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)' ] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)', 'p1 (3)', 'p1 (4)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (3)', 'p1 (4)'] + and: + upstreamTasksOf('p2 (4)') + .name == ['p1 (4)'] + + } + + def 'should track provenance with count value operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.value(1) | p1 | count | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1'] + } + + def 'should track provenance with count many operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) | p1 | count | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + } + + @Ignore // the semantic of this should be reviewed + def 'should track provenance with min operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(3,2,1) | p1 | min | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (3)'] + } + + @Ignore // the semantic of this should be reviewed + def 'should track provenance with max operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(3,2,1) | p1 | max | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)'] + } + + def 'should track provenance with sum operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(3,2,1) | p1 | sum | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + } + + def 'should track provenance with mean operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(3,2,1) | p1 | mean | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2') + .name == ['p1 (1)', 'p1 (2)', 'p1 (3)'] + } + + def 'should track provenance with buffer operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3,4,5) | p1 | buffer(size:2, remainder:true) | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)', 'p1 (2)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (3)', 'p1 (4)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (5)'] + } + + @Ignore // disabling because some assertions are not deterministic + def 'should track provenance with mix operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + def c1 = channel.of(1,2) | p1 + def c2 = channel.of(3,4) | p2 + p1.out | mix(p2.out) | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p3 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p3 (2)') + .name == ['p2 (1)'] + } + + def 'should track provenance with join operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + def c1 = channel.of(['a',10], ['b',20]) | p1 + def c2 = channel.of(['a',11], ['b',21], ['c',31]) | p2 + p1.out | join(p2.out, remainder:true) | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p3 (1)') + .name.sort() == ['p1 (1)', 'p2 (1)'] + and: + upstreamTasksOf('p3 (2)') + .name.sort() == ['p1 (2)', 'p2 (2)'] + and: + upstreamTasksOf('p3 (3)') + .name.sort() == ['p2 (3)'] + } + + @Ignore // disabling because some assertions are not deterministic + def 'should track provenance with combine operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + def c1 = channel.of(1,2) | p1 + def c2 = channel.of('a','b') | p2 + p1.out | combine(p2.out) | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p3 (1)') + .name.sort() == ['p1 (1)', 'p2 (1)'] + and: + upstreamTasksOf('p3 (2)') + .name.sort() == ['p1 (1)', 'p2 (2)'] + and: + upstreamTasksOf('p3 (3)') + .name.sort() == ['p1 (2)', 'p2 (1)'] + and: + upstreamTasksOf('p3 (4)') + .name.sort() == ['p1 (2)', 'p2 (2)'] + + } + + def 'should track provenance with concat operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + def c1 = channel.of(1,2) | p1 + def c2 = channel.of(3,4) | p2 + p1.out | concat(p2.out) | p3 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p3 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p3 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p3 (2)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p3 (3)') + .name == ['p2 (1)'] + and: + upstreamTasksOf('p3 (4)') + .name == ['p2 (2)'] + } + + def 'should track provenance with flatten operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of([1,'a'], [2,'b']) \ + | p1 \ + | flatten \ + | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p2 (4)') + .name == ['p1 (2)'] + } + + def 'should track provenance with groupTuple operator' () { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of( [1, 'A'], [1, 'B'], [2, 'C'], [3, 'B'], [1, 'C'], [2, 'A'], [3, 'D'] ) \ + | p1 \ + | groupTuple \ + | p2 + } + + // the groupTuple should emit the following values + // + // [1, [A, B, C]] + // [2, [C, A]] + // [3, [B, D]] + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println "$task.process ($task.index) = ${x}" + } + ''') + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)', 'p1 (2)', 'p1 (5)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (3)', 'p1 (6)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (4)', 'p1 (7)'] + } + + def 'should track provenance with until operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3,4) | p1 | until{ it->it>3 } | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (3)'] + } + + def 'should track provenance with ifEmpty operator'() { + when: + dsl_eval(globalConfig(), ''' + workflow { + channel.of(1,2,3) | p1 | ifEmpty('nope') | p2 + } + + process p1 { + input: val(x) + output: val(y) + exec: + y = x + } + + process p2 { + input: val(x) + exec: + println x + } + ''') + + then: + upstreamTasksOf('p2 (1)') + .name == ['p1 (1)'] + and: + upstreamTasksOf('p2 (2)') + .name == ['p1 (2)'] + and: + upstreamTasksOf('p2 (3)') + .name == ['p1 (3)'] + } +} diff --git a/modules/nextflow/src/test/groovy/nextflow/prov/TrackerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/prov/TrackerTest.groovy new file mode 100644 index 0000000000..8a2adc1518 --- /dev/null +++ b/modules/nextflow/src/test/groovy/nextflow/prov/TrackerTest.groovy @@ -0,0 +1,123 @@ +/* + * Copyright 2013-2024, Seqera Labs + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package nextflow.prov + +import groovyx.gpars.dataflow.DataflowQueue +import groovyx.gpars.dataflow.DataflowWriteChannel +import nextflow.processor.TaskConfig +import nextflow.processor.TaskId +import nextflow.processor.TaskProcessor +import nextflow.processor.TaskRun +import spock.lang.Specification + +/** + * + * @author Paolo Di Tommaso + */ +class TrackerTest extends Specification { + + def 'should normalize null values' () { + given: + def prov = new Tracker() + and: + def t1 = new TaskRun(id: new TaskId(1), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + def t2 = new TaskRun(id: new TaskId(2), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + and: + def msg1 = [Tracker.Msg.of('foo')] + def msg2 = [Tracker.Msg.of('foo'), Tracker.Msg.of(null)] + + when: + def result1 = prov.receiveInputs(t1, msg1) + then: + result1 == msg1.value + and: + t1.upstreamTasks == [] as Set + + when: + def result2 = prov.receiveInputs(t2, msg2) + then: + result2 == ['foo', null] + and: + t2.upstreamTasks == [] as Set + } + + def 'should bind value to task run' () { + given: + def prov = new Tracker() + and: + def t1 = new TaskRun(id: new TaskId(1), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + def c1 = new DataflowQueue() + def v1 = 'foo' + + when: + def m1 = prov.bindOutput(t1, c1, v1) + then: + c1.val.is(m1) + and: + prov.@messages.get(System.identityHashCode(m1)) == t1 + } + + def 'should determine upstream tasks' () { + given: + def prov = new Tracker() + and: + def t1 = new TaskRun(id: new TaskId(1), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + def t2 = new TaskRun(id: new TaskId(2), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + def t3 = new TaskRun(id: new TaskId(3), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + and: + def v1 = new Object() + def v2 = new Object() + + when: + def m1 = prov.bindOutput(t1, Mock(DataflowWriteChannel), v1) + def m2 = prov.bindOutput(t2, Mock(DataflowWriteChannel), v2) + and: + prov.receiveInputs(t3, [m1, m2]) + then: + t3.upstreamTasks == [t1.id, t2.id] as Set + } + + def 'should determine upstream task with operator' () { + given: + def prov = new Tracker() + and: + def v1 = Integer.valueOf(1) + def v2 = Integer.valueOf(2) + def v3 = Integer.valueOf(3) + and: + def t1 = new TaskRun(id: new TaskId(1), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + def p2 = new OperatorRun() + def t3 = new TaskRun(id: new TaskId(3), processor: Mock(TaskProcessor), config: Mock(TaskConfig)) + + when: + prov.receiveInputs(t1, []) + def m1 = prov.bindOutput(t1, Mock(DataflowWriteChannel), v1) + and: + prov.receiveInputs(p2, [m1]) + and: + def m2 = prov.bindOutput(p2, Mock(DataflowWriteChannel), v2) + and: + prov.receiveInputs(t3, [m2]) + and: + def m3 = prov.bindOutput(t3, Mock(DataflowWriteChannel), v3) + + then: + t3.upstreamTasks == [t1.id] as Set + } + +} diff --git a/modules/nextflow/src/test/groovy/nextflow/script/ScriptDslTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/ScriptDslTest.groovy index 3758ad497a..138a256147 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/ScriptDslTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/ScriptDslTest.groovy @@ -36,8 +36,8 @@ class ScriptDslTest extends Dsl2Spec { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result[0].val == 'Hello' - result[1].val == 'world' + result[0].unwrap() == 'Hello' + result[1].unwrap() == 'world' } def 'should execute basic workflow' () { @@ -52,7 +52,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'Hello world' + result.unwrap() == 'Hello world' } def 'should execute emit' () { @@ -66,7 +66,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'Hello world' + result.unwrap() == 'Hello world' } def 'should emit expression' () { @@ -82,7 +82,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'HELLO WORLD' + result.unwrap() == 'HELLO WORLD' } def 'should emit process out' () { @@ -101,7 +101,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'Hello' + result.unwrap() == 'Hello' } @@ -142,7 +142,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'HELLO MUNDO' + result.unwrap() == 'HELLO MUNDO' } @@ -229,7 +229,7 @@ class ScriptDslTest extends Dsl2Spec { } /) then: - result.val == 'HELLO' + result.unwrap() == 'HELLO' } def 'should branch and view' () { @@ -250,12 +250,12 @@ class ScriptDslTest extends Dsl2Spec { [ch1, ch2] /) then: - result[0].val == 1 - result[0].val == 2 - result[0].val == 3 + result[0].unwrap() == 1 + result[0].unwrap() == 2 + result[0].unwrap() == 3 and: - result[1].val == 40 - result[1].val == 50 + result[1].unwrap() == 40 + result[1].unwrap() == 50 } @@ -278,9 +278,9 @@ class ScriptDslTest extends Dsl2Spec { ''') then: - result.val == 'hello' - result.val == 'world' - result.val == Channel.STOP + result.unwrap() == 'hello' + result.unwrap() == 'world' + result.unwrap() == Channel.STOP } def 'should allow process and operator composition' () { @@ -304,9 +304,9 @@ class ScriptDslTest extends Dsl2Spec { then: - result.val == 'hello' - result.val == 'world' - result.val == Channel.STOP + result.unwrap() == 'hello' + result.unwrap() == 'world' + result.unwrap() == Channel.STOP } def 'should run entry flow' () { @@ -335,7 +335,7 @@ class ScriptDslTest extends Dsl2Spec { then: - result.val == 'world' + result.unwrap() == 'world' } @@ -601,7 +601,7 @@ class ScriptDslTest extends Dsl2Spec { ''' then: - result.val == 'Hello' + result.unwrap() == 'Hello' } diff --git a/modules/nextflow/src/test/groovy/nextflow/script/ScriptIncludesTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/ScriptIncludesTest.groovy index 81a53a492f..54634490a7 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/ScriptIncludesTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/ScriptIncludesTest.groovy @@ -145,7 +145,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'dlrow olleh' + result.unwrap() == 'dlrow olleh' } def 'should allow duplicate functions' () { @@ -174,7 +174,7 @@ class ScriptIncludesTest extends Dsl2Spec { when: def result = new MockScriptRunner() .setScript(SCRIPT).execute() - def map = result.val + def map = result.unwrap() then: map map.witharg == 'hello world'.reverse() @@ -213,9 +213,9 @@ class ScriptIncludesTest extends Dsl2Spec { def result = new MockScriptRunner() .setScript(SCRIPT).execute() then: - result.val == '1-2' - result.val == '2-4' - result.val == '3-6' + result.unwrap() == '1-2' + result.unwrap() == '2-4' + result.unwrap() == '3-6' cleanup: NextflowMeta.instance.strictMode(false) @@ -301,7 +301,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'HELLO MUNDO' + result.unwrap() == 'HELLO MUNDO' binding.variables.alpha == null } @@ -351,7 +351,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'HELLO MUNDO' + result.unwrap() == 'HELLO MUNDO' !binding.hasVariable('alpha') !binding.hasVariable('foo') !binding.hasVariable('bar') @@ -397,7 +397,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'HELLO MUNDO' + result.unwrap() == 'HELLO MUNDO' } def 'should gather process outputs' () { @@ -440,7 +440,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'HELLO MUNDO' + result.unwrap() == 'HELLO MUNDO' !vars.containsKey('data') !vars.containsKey('foo') !vars.containsKey('bar') @@ -476,7 +476,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: noExceptionThrown() - result.val == 'echo Hello world' + result.unwrap() == 'echo Hello world' cleanup: folder?.deleteDir() @@ -516,7 +516,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: noExceptionThrown() - result.val == 'echo sample=world pairId=x reads=/some/file' + result.unwrap() == 'echo sample=world pairId=x reads=/some/file' } @@ -563,7 +563,7 @@ class ScriptIncludesTest extends Dsl2Spec { when: def result = dsl_eval(SCRIPT) then: - result.val == 'echo Ciao world' + result.unwrap() == 'echo Ciao world' } def 'should use multiple assignment' () { @@ -610,8 +610,8 @@ class ScriptIncludesTest extends Dsl2Spec { when: def result = dsl_eval(SCRIPT) then: - result[0].val == 'Ciao' - result[1].val == 'world' + result[0].unwrap() == 'Ciao' + result[1].unwrap() == 'world' } @@ -648,7 +648,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: noExceptionThrown() - result.val == 'echo Hello world' + result.unwrap() == 'echo Hello world' } @@ -716,7 +716,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: noExceptionThrown() - result.val == 'echo Hola mundo' + result.unwrap() == 'echo Hola mundo' } def 'should not fail when invoking a process in a module' () { @@ -866,8 +866,8 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result[0].val == 'HELLO' - result[1].val == 'WORLD' + result[0].unwrap() == 'HELLO' + result[1].unwrap() == 'WORLD' } @@ -919,8 +919,8 @@ class ScriptIncludesTest extends Dsl2Spec { """) then: - result[0].val == 'CMD CONSUMER 1' - result[1].val == 'CMD CONSUMER 2' + result[0].unwrap() == 'CMD CONSUMER 1' + result[1].unwrap() == 'CMD CONSUMER 2' } @@ -957,8 +957,8 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result[0].val == 'HELLO' - result[1].val == 'WORLD' + result[0].unwrap() == 'HELLO' + result[1].unwrap() == 'WORLD' } def 'should inherit module params' () { @@ -993,7 +993,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'OWNER LAST' + result.unwrap() == 'OWNER LAST' } def 'should override module params' () { @@ -1028,7 +1028,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'AAA ZZZ' + result.unwrap() == 'AAA ZZZ' } def 'should extends module params' () { @@ -1065,7 +1065,7 @@ class ScriptIncludesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'ONE ZZZ' + result.unwrap() == 'ONE ZZZ' } def 'should declare moduleDir path' () { diff --git a/modules/nextflow/src/test/groovy/nextflow/script/ScriptPipesTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/ScriptPipesTest.groovy index 20c44c2525..db38b87a36 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/ScriptPipesTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/ScriptPipesTest.groovy @@ -43,8 +43,8 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result[0].val == 'olleH mundo' - result[1].val == 'OLLEH' + result[0].unwrap() == 'olleH mundo' + result[1].unwrap() == 'OLLEH' } @@ -75,7 +75,7 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'DLROW ALOH' + result.unwrap() == 'DLROW ALOH' } def 'should pipe multi-outputs with multi-inputs' () { @@ -114,7 +114,7 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'olleh + HELLO' + result.unwrap() == 'olleh + HELLO' } def 'should pipe process with operator' () { @@ -145,9 +145,9 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'aloh' - result.val == 'HOLA' - result.val == 'hola' + result.unwrap() == 'aloh' + result.unwrap() == 'HOLA' + result.unwrap() == 'hola' } @@ -171,7 +171,7 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'aloh' + result.unwrap() == 'aloh' } @@ -201,7 +201,7 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val == 'HOLA' + result.unwrap() == 'HOLA' } @@ -225,7 +225,7 @@ class ScriptPipesTest extends Dsl2Spec { def result = runner.setScript(SCRIPT).execute() then: - result.val.sort() == [1, 4, 9] + result.unwrap().sort() == [1, 4, 9] } @@ -238,9 +238,9 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 10 - result.val == 20 - result.val == 30 + result.unwrap() == 10 + result.unwrap() == 20 + result.unwrap() == 30 } @@ -262,7 +262,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 40 + result.unwrap() == 40 } def 'should compose custom funs' () { @@ -285,7 +285,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 22 + result.unwrap() == 22 } @@ -312,7 +312,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == [1,2] + result.unwrap() == [1,2] } @@ -340,7 +340,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 'hi'.reverse() + result.unwrap() == 'hi'.reverse() } @@ -368,7 +368,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 'hello'.reverse() + result.unwrap() == 'hello'.reverse() } @@ -400,7 +400,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 'HI'.reverse() + result.unwrap() == 'HI'.reverse() } @@ -430,7 +430,7 @@ class ScriptPipesTest extends Dsl2Spec { when: def result = new MockScriptRunner().setScript(SCRIPT).execute() then: - result.val == 33 + result.unwrap() == 33 cleanup: diff --git a/modules/nextflow/src/test/groovy/nextflow/script/ScriptRecurseTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/ScriptRecurseTest.groovy index 2723b2a051..9a1282d81a 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/ScriptRecurseTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/ScriptRecurseTest.groovy @@ -53,10 +53,10 @@ class ScriptRecurseTest extends Dsl2Spec { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 2 - result.val == 3 - result.val == 4 - result.val == Channel.STOP + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 4 + result.unwrap() == Channel.STOP } def 'should recourse a process until a condition is verified' () { @@ -79,10 +79,10 @@ class ScriptRecurseTest extends Dsl2Spec { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 2 - result.val == 3 - result.val == 4 - result.val == Channel.STOP + result.unwrap() == 2 + result.unwrap() == 3 + result.unwrap() == 4 + result.unwrap() == Channel.STOP } @@ -121,10 +121,10 @@ class ScriptRecurseTest extends Dsl2Spec { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 4 - result.val == 25 - result.val == 676 - result.val == Channel.STOP + result.unwrap() == 4 + result.unwrap() == 25 + result.unwrap() == 676 + result.unwrap() == Channel.STOP } def 'should recurse with scan' () { @@ -151,9 +151,9 @@ class ScriptRecurseTest extends Dsl2Spec { def runner = new MockScriptRunner() def result = runner.setScript(SCRIPT).execute() then: - result.val == 11 // 10 +1 - result.val == 32 // 20 + 11 +1 - result.val == 74 // 30 + 11 + 32 +1 + result.unwrap() == 11 // 10 +1 + result.unwrap() == 32 // 20 + 11 +1 + result.unwrap() == 74 // 30 + 11 + 32 +1 } diff --git a/modules/nextflow/src/test/groovy/nextflow/script/ScriptRunnerTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/ScriptRunnerTest.groovy index 682337368d..900eb3171c 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/ScriptRunnerTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/ScriptRunnerTest.groovy @@ -66,7 +66,7 @@ class ScriptRunnerTest extends Dsl2Spec { // when no outputs are specified, the 'stdout' is the default output then: result instanceof DataflowVariable - result.val == "echo Hello world" + result.unwrap() == "echo Hello world" } @@ -130,7 +130,7 @@ class ScriptRunnerTest extends Dsl2Spec { def result = new MockScriptRunner().setScript(script).execute() then: - result.val == 'echo 1 - 3' + result.unwrap() == 'echo 1 - 3' } @@ -160,7 +160,7 @@ class ScriptRunnerTest extends Dsl2Spec { runner.execute() then: - runner.result.val == 'echo 1' + runner.result.unwrap() == 'echo 1' TaskProcessor.currentProcessor().name == 'simpleTask' } @@ -191,7 +191,7 @@ class ScriptRunnerTest extends Dsl2Spec { def runner = new MockScriptRunner().setScript(script) def result = runner.execute() then: - result.val == '1-2-3' + result.unwrap() == '1-2-3' } @@ -221,7 +221,7 @@ class ScriptRunnerTest extends Dsl2Spec { def runner = new MockScriptRunner().setScript(script) def result = runner.execute() then: - result.val == '1-2-3' + result.unwrap() == '1-2-3' } @@ -272,7 +272,7 @@ class ScriptRunnerTest extends Dsl2Spec { def config = [executor: 'nope', env: [HELLO: 'Hello world!']] expect: - new MockScriptRunner(config).setScript(script).execute().val == 'Hello world!' + new MockScriptRunner(config).setScript(script).execute().unwrap() == 'Hello world!' } @@ -300,7 +300,7 @@ class ScriptRunnerTest extends Dsl2Spec { def config = [executor: 'nope'] expect: - new MockScriptRunner(config).setScript(script).execute().val == 'cat filename' + new MockScriptRunner(config).setScript(script).execute().unwrap() == 'cat filename' } @@ -536,7 +536,7 @@ class ScriptRunnerTest extends Dsl2Spec { def result = new MockScriptRunner(session) .setScript(script) .execute() - .getVal() + .unwrap() .toString() .stripIndent() .trim() @@ -586,7 +586,7 @@ class ScriptRunnerTest extends Dsl2Spec { def result = new MockScriptRunner(config) .setScript(script) .execute() - .getVal() + .unwrap() .toString() .stripIndent() .trim() @@ -692,7 +692,7 @@ class ScriptRunnerTest extends Dsl2Spec { // when no outputs are specified, the 'stdout' is the default output then: result instanceof DataflowVariable - result.val == "echo foo" + result.unwrap() == "echo foo" } @@ -729,7 +729,7 @@ class ScriptRunnerTest extends Dsl2Spec { // when no outputs are specified, the 'stdout' is the default output then: result instanceof DataflowVariable - result.val == "echo foo" + result.unwrap() == "echo foo" } @@ -767,7 +767,7 @@ class ScriptRunnerTest extends Dsl2Spec { // when no outputs are specified, the 'stdout' is the default output then: result instanceof DataflowVariable - result.val == "echo foo" + result.unwrap() == "echo foo" } } diff --git a/modules/nextflow/src/test/groovy/nextflow/script/params/EachInParamTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/params/EachInParamTest.groovy index 8fde47710d..1a66bb4fda 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/params/EachInParamTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/params/EachInParamTest.groovy @@ -39,11 +39,11 @@ class EachInParamTest extends Specification { def each = new EachInParam(Mock(Binding), []) expect: - each.normalizeToVariable(1).val == [1] - each.normalizeToVariable([3,4,5]).val == [3,4,5] - each.normalizeToVariable(channel).val == [1,2,3,5] - each.normalizeToVariable(value).val == ['a'] - each.normalizeToVariable(list).val == [4,5,6] + each.normalizeToVariable(1).unwrap() == [1] + each.normalizeToVariable([3,4,5]).unwrap() == [3,4,5] + each.normalizeToVariable(channel).unwrap() == [1,2,3,5] + each.normalizeToVariable(value).unwrap() == ['a'] + each.normalizeToVariable(list).unwrap() == [4,5,6] } diff --git a/modules/nextflow/src/test/groovy/nextflow/script/params/ParamsInTest.groovy b/modules/nextflow/src/test/groovy/nextflow/script/params/ParamsInTest.groovy index 9f3e412d85..8bc325bb0e 100644 --- a/modules/nextflow/src/test/groovy/nextflow/script/params/ParamsInTest.groovy +++ b/modules/nextflow/src/test/groovy/nextflow/script/params/ParamsInTest.groovy @@ -72,21 +72,21 @@ class ParamsInTest extends Dsl2Spec { in1.class == ValueInParam in1.name == 'x' - in1.inChannel.val == 'Hello' + in1.inChannel.unwrap() == 'Hello' in2.class == ValueInParam in2.name == 'x' - in2.inChannel.val == 'Hola' + in2.inChannel.unwrap() == 'Hola' in3.class == ValueInParam in3.name == 'x' - in3.inChannel.val == 'ciao' + in3.inChannel.unwrap() == 'ciao' in4.class == ValueInParam in4.name == 'x' - in4.inChannel.val == 1 - in4.inChannel.val == 2 - in4.inChannel.val == Channel.STOP + in4.inChannel.unwrap() == 1 + in4.inChannel.unwrap() == 2 + in4.inChannel.unwrap() == Channel.STOP } @@ -122,19 +122,19 @@ class ParamsInTest extends Dsl2Spec { then: in1.name == 'x' - in1.inChannel.val == 1 - in1.inChannel.val == 2 - in1.inChannel.val == Channel.STOP + in1.inChannel.unwrap() == 1 + in1.inChannel.unwrap() == 2 + in1.inChannel.unwrap() == Channel.STOP in2.name == 'y' - in2.inChannel.val == 'a' - in2.inChannel.val == 'b' - in2.inChannel.val == Channel.STOP + in2.inChannel.unwrap() == 'a' + in2.inChannel.unwrap() == 'b' + in2.inChannel.unwrap() == Channel.STOP in3.name == 'z' - in3.inChannel.val == 3 - in3.inChannel.val == 4 - in3.inChannel.val == Channel.STOP + in3.inChannel.unwrap() == 3 + in3.inChannel.unwrap() == 4 + in3.inChannel.unwrap() == Channel.STOP } @@ -172,27 +172,27 @@ class ParamsInTest extends Dsl2Spec { in1.name == 'x' in1.filePattern == '*' - in1.inChannel.val == Paths.get('file.x') + in1.inChannel.unwrap() == Paths.get('file.x') in1.index == 0 in2.name == 'f1' in2.filePattern == '*' - in2.inChannel.val == Paths.get('file.x') + in2.inChannel.unwrap() == Paths.get('file.x') in2.index == 1 in3.name == 'f2' in3.filePattern == 'abc' - in3.inChannel.val == Paths.get('file.x') + in3.inChannel.unwrap() == Paths.get('file.x') in3.index == 2 in4.name == 'f3' in4.filePattern == '*.fa' - in4.inChannel.val == Paths.get('file.x') + in4.inChannel.unwrap() == Paths.get('file.x') in4.index == 3 in5.name == 'file.txt' in5.filePattern == 'file.txt' - in5.inChannel.val == Paths.get('file.x') + in5.inChannel.unwrap() == Paths.get('file.x') in5.index == 4 } @@ -228,15 +228,15 @@ class ParamsInTest extends Dsl2Spec { in1.name == '__$fileinparam<0>' in1.getFilePattern(ctx) == 'main.txt' - in1.inChannel.val == Paths.get('file.txt') + in1.inChannel.unwrap() == Paths.get('file.txt') in2.name == '__$fileinparam<1>' in2.getFilePattern(ctx) == 'hello.txt' - in2.inChannel.val == "str" + in2.inChannel.unwrap() == "str" in3.name == 'f2' in3.getFilePattern(ctx) == 'the_file_name.fa' - in3.inChannel.val == Paths.get('file.txt') + in3.inChannel.unwrap() == Paths.get('file.txt') } @@ -273,19 +273,19 @@ class ParamsInTest extends Dsl2Spec { in1.name == '__$fileinparam<0>' in1.getFilePattern(ctx) == 'main.txt' - in1.inChannel.val == Paths.get('file.txt') + in1.inChannel.unwrap() == Paths.get('file.txt') in2.name == '__$fileinparam<1>' in2.getFilePattern(ctx) == 'hello.txt' - in2.inChannel.val == "str" + in2.inChannel.unwrap() == "str" in3.name == 'f2' in3.getFilePattern(ctx) == 'the_file_name.fa' - in3.inChannel.val == Paths.get('file.txt') + in3.inChannel.unwrap() == Paths.get('file.txt') in4.name == 'f3' in4.getFilePattern(ctx) == 'the_file_name.txt' - in4.inChannel.val == Paths.get('file.txt') + in4.inChannel.unwrap() == Paths.get('file.txt') } @@ -318,11 +318,11 @@ class ParamsInTest extends Dsl2Spec { in1.class == StdInParam in1.name == '-' - in1.inChannel.val == 'Hola mundo' + in1.inChannel.unwrap() == 'Hola mundo' in2.class == StdInParam in2.name == '-' - in2.inChannel.val == 'Ciao mondo' + in2.inChannel.unwrap() == 'Ciao mondo' } def testInputEnv() { @@ -354,13 +354,13 @@ class ParamsInTest extends Dsl2Spec { in1.class == EnvInParam in1.name == 'VAR_X' - in1.inChannel.val == 'aaa' + in1.inChannel.unwrap() == 'aaa' in2.class == EnvInParam in2.name == 'VAR_Y' - in2.inChannel.val == 1 - in2.inChannel.val == 2 - in2.inChannel.val == Channel.STOP + in2.inChannel.unwrap() == 1 + in2.inChannel.unwrap() == 2 + in2.inChannel.unwrap() == Channel.STOP } @@ -402,7 +402,7 @@ class ParamsInTest extends Dsl2Spec { in1.inner.get(0).index == 0 in1.inner.get(0).mapIndex == 0 in1.inner.get(0).name == 'p' - in1.inChannel.val == 'Hola mundo' + in1.inChannel.unwrap() == 'Hola mundo' in2.inner.size() == 2 in2.inner.get(0) instanceof ValueInParam @@ -413,7 +413,7 @@ class ParamsInTest extends Dsl2Spec { in2.inner.get(1).name == 'q' in2.inner.get(1).index == 1 in2.inner.get(1).mapIndex == 1 - in2.inChannel.val == 'Hola mundo' + in2.inChannel.unwrap() == 'Hola mundo' in3.inner.size() == 2 in3.inner.get(0) instanceof ValueInParam @@ -425,7 +425,7 @@ class ParamsInTest extends Dsl2Spec { in3.inner.get(1).filePattern == 'file_name.fa' in3.inner.get(1).index == 2 in3.inner.get(1).mapIndex == 1 - in3.inChannel.val == 'str' + in3.inChannel.unwrap() == 'str' in4.inner.size() == 3 in4.inner.get(0) instanceof ValueInParam @@ -441,7 +441,7 @@ class ParamsInTest extends Dsl2Spec { in4.inner.get(2).name == '-' in4.inner.get(2).index == 3 in4.inner.get(2).mapIndex == 2 - in4.inChannel.val == 'ciao' + in4.inChannel.unwrap() == 'ciao' in5.inner.size() == 2 in5.inner.get(0) instanceof ValueInParam @@ -453,7 +453,7 @@ class ParamsInTest extends Dsl2Spec { in5.inner.get(1).filePattern == 'file.fa' in5.inner.get(1).index == 4 in5.inner.get(1).mapIndex == 1 - in5.inChannel.val == 0 + in5.inChannel.unwrap() == 0 } @@ -493,7 +493,7 @@ class ParamsInTest extends Dsl2Spec { def ctx = [x:'the_file', str: 'fastq'] then: - in0.inChannel.val == 'the file content' + in0.inChannel.unwrap() == 'the file content' in0.inner[0] instanceof FileInParam (in0.inner[0] as FileInParam).name == 'name_$x' (in0.inner[0] as FileInParam).getFilePattern(ctx) == 'name_$x' @@ -546,7 +546,7 @@ class ParamsInTest extends Dsl2Spec { process.config.getInputs().size() == 3 in0.name == '__$tupleinparam<0>' - in0.inChannel.val == 1 + in0.inChannel.unwrap() == 1 in0.inner.size() == 3 in0.inner.get(0) instanceof ValueInParam in0.inner.get(0).name == 'a' @@ -629,21 +629,21 @@ class ParamsInTest extends Dsl2Spec { in0.class == EachInParam in0.inChannel instanceof DataflowVariable - in0.inChannel.val == ['aaa'] + in0.inChannel.unwrap() == ['aaa'] in0.inner.name == 'x' in0.inner.owner == in0 in1.class == EachInParam in1.name == '__$eachinparam<1>' in1.inChannel instanceof DataflowVariable - in1.inChannel.val == [1,2] + in1.inChannel.unwrap() == [1,2] in1.inner.name == 'p' in1.inner instanceof ValueInParam in1.inner.owner == in1 in2.class == EachInParam in2.name == '__$eachinparam<2>' - in2.inChannel.val == [1,2,3] + in2.inChannel.unwrap() == [1,2,3] in2.inner instanceof ValueInParam in2.inner.name == 'z' in2.inner.owner == in2 @@ -651,7 +651,7 @@ class ParamsInTest extends Dsl2Spec { in3.class == EachInParam in3.name == '__$eachinparam<3>' in3.inChannel instanceof DataflowVariable - in3.inChannel.val == ['file-a.txt'] + in3.inChannel.unwrap() == ['file-a.txt'] in3.inner instanceof FileInParam in3.inner.name == 'foo' in3.inner.owner == in3 @@ -659,7 +659,7 @@ class ParamsInTest extends Dsl2Spec { in4.class == EachInParam in4.name == '__$eachinparam<4>' in4.inChannel instanceof DataflowVariable - in4.inChannel.val == ['file-x.fa'] + in4.inChannel.unwrap() == ['file-x.fa'] in4.inner instanceof FileInParam in4.inner.name == 'bar' in4.inner.filePattern == 'bar' @@ -735,28 +735,28 @@ class ParamsInTest extends Dsl2Spec { in0.name == 'x' in0.filePattern == '*' - in0.inChannel.val == FILE + in0.inChannel.unwrap() == FILE in0.index == 0 in0.isPathQualifier() in0.arity == new ArityParam.Range(1, 1) in1.name == 'f1' in1.filePattern == '*' - in1.inChannel.val == FILE + in1.inChannel.unwrap() == FILE in1.index == 1 in1.isPathQualifier() in1.arity == new ArityParam.Range(1, 2) in2.name == '*.fa' in2.filePattern == '*.fa' - in2.inChannel.val == FILE + in2.inChannel.unwrap() == FILE in2.index == 2 in2.isPathQualifier() in2.arity == new ArityParam.Range(1, Integer.MAX_VALUE) in3.name == 'file.txt' in3.filePattern == 'file.txt' - in3.inChannel.val == FILE + in3.inChannel.unwrap() == FILE in3.index == 3 in3.isPathQualifier() @@ -800,12 +800,12 @@ class ParamsInTest extends Dsl2Spec { in1.name == '__$pathinparam<0>' in1.getFilePattern(ctx) == 'main.txt' - in1.inChannel.val == 'file.txt' + in1.inChannel.unwrap() == 'file.txt' in1.isPathQualifier() in2.name == '__$pathinparam<1>' in2.getFilePattern(ctx) == 'hello.txt' - in2.inChannel.val == "str" + in2.inChannel.unwrap() == "str" in2.isPathQualifier() } @@ -835,7 +835,7 @@ class ParamsInTest extends Dsl2Spec { def ctx = [x:'the_file', str: 'fastq'] then: - in1.inChannel.val == '/the/file/path' + in1.inChannel.unwrap() == '/the/file/path' in1.inner[0] instanceof FileInParam (in1.inner[0] as FileInParam).getName() == '__$pathinparam<0:0>' (in1.inner[0] as FileInParam).getFilePattern(ctx) == 'hola_the_file' @@ -875,7 +875,7 @@ class ParamsInTest extends Dsl2Spec { process.config.getInputs().size() == 3 in0.name == '__$tupleinparam<0>' - in0.inChannel.val == 1 + in0.inChannel.unwrap() == 1 in0.inner.size() == 2 in0.inner.get(0) instanceof ValueInParam in0.inner.get(0).name == 'a' @@ -938,7 +938,7 @@ class ParamsInTest extends Dsl2Spec { in0.class == EachInParam in0.name == '__$eachinparam<0>' in0.inChannel instanceof DataflowVariable - in0.inChannel.val == ['file-a.txt'] + in0.inChannel.unwrap() == ['file-a.txt'] in0.inner instanceof FileInParam (in0.inner as FileInParam).name == 'foo' (in0.inner as FileInParam).owner == in0 @@ -947,7 +947,7 @@ class ParamsInTest extends Dsl2Spec { in1.class == EachInParam in1.name == '__$eachinparam<1>' in1.inChannel instanceof DataflowVariable - in1.inChannel.val == ['file-x.fa'] + in1.inChannel.unwrap() == ['file-x.fa'] in1.inner instanceof FileInParam (in1.inner as FileInParam).name == 'bar' (in1.inner as FileInParam).filePattern == 'bar' diff --git a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt index 5aef2f4795..c4bbe2e8ba 100644 --- a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt +++ b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper-with-trace.txt @@ -1,5 +1,6 @@ #!/bin/bash ### --- +### id: '200' ### name: 'Hello 2' ### ... set -e diff --git a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt index 3bb4f34fe5..2025790233 100644 --- a/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt +++ b/modules/nextflow/src/test/resources/nextflow/executor/test-bash-wrapper.txt @@ -2,6 +2,7 @@ #BSUB -x 1 #BSUB -y 2 ### --- +### id: '100' ### name: 'Hello 1' ### ... set -e diff --git a/modules/nextflow/src/testFixtures/groovy/test/Dsl2Spec.groovy b/modules/nextflow/src/testFixtures/groovy/test/Dsl2Spec.groovy index a30c57f916..63c736b365 100644 --- a/modules/nextflow/src/testFixtures/groovy/test/Dsl2Spec.groovy +++ b/modules/nextflow/src/testFixtures/groovy/test/Dsl2Spec.groovy @@ -46,11 +46,14 @@ class Dsl2Spec extends BaseSpec { new MockScriptRunner().setScript(str).execute() } + def dsl_eval(Map config, String str) { + new MockScriptRunner(config).setScript(str).execute() + } + def dsl_eval(Path path) { new MockScriptRunner().setScript(path).execute() } - def dsl_eval(String entry, String str) { new MockScriptRunner() .setScript(str).execute(null, entry) diff --git a/modules/nextflow/src/testFixtures/groovy/test/MockHelpers.groovy b/modules/nextflow/src/testFixtures/groovy/test/MockHelpers.groovy index d4f5065c0d..be772ff133 100644 --- a/modules/nextflow/src/testFixtures/groovy/test/MockHelpers.groovy +++ b/modules/nextflow/src/testFixtures/groovy/test/MockHelpers.groovy @@ -127,7 +127,7 @@ class MockExecutor extends Executor { @Override TaskHandler createTaskHandler(TaskRun task) { - return new MockTaskHandler(task) + return new MockTaskHandler(task) } } diff --git a/modules/nextflow/src/testFixtures/groovy/test/TestHelper.groovy b/modules/nextflow/src/testFixtures/groovy/test/TestHelper.groovy index dce6834090..8ca3fcb841 100644 --- a/modules/nextflow/src/testFixtures/groovy/test/TestHelper.groovy +++ b/modules/nextflow/src/testFixtures/groovy/test/TestHelper.groovy @@ -15,6 +15,7 @@ */ package test + import java.nio.file.Files import java.nio.file.Path import java.util.zip.GZIPInputStream @@ -22,6 +23,9 @@ import java.util.zip.GZIPInputStream import com.google.common.jimfs.Configuration import com.google.common.jimfs.Jimfs import groovy.transform.Memoized +import nextflow.processor.TaskId +import nextflow.processor.TaskProcessor +import nextflow.processor.TaskRun /** * * @author Paolo Di Tommaso @@ -91,4 +95,36 @@ class TestHelper { // Convert the decoded bytes into a string return new String(decodedBytes); } + + static List upstreamTasksOf(v) { + if( v instanceof TaskRun ) + return upstreamTasksOf(v as TaskRun) + + if( v instanceof CharSequence ) { + TaskRun t = getTaskByName(v.toString()) + if( t ) + return upstreamTasksOf(t) + else + throw new IllegalArgumentException("Cannot find any task with name: $v") + } + + TaskRun t = getTaskById(v) + if( !t ) + throw new IllegalArgumentException("Cannot find any task with id: $v") + return upstreamTasksOf(t) + } + + static List upstreamTasksOf(TaskRun t) { + final ids = t.upstreamTasks ?: Set.of() + return ids.collect(it -> getTaskById(it)) + } + + static TaskRun getTaskByName(String name) { + TaskProcessor.allTasks.values().find( it -> it.name==name ) + } + + static TaskRun getTaskById(id) { + TaskProcessor.allTasks.get(TaskId.of(id)) + } + } diff --git a/modules/nf-commons/src/test/nextflow/plugin/extension/PluginExtensionMethodsTest.groovy b/modules/nf-commons/src/test/nextflow/plugin/extension/PluginExtensionMethodsTest.groovy index 82cf64ebd6..a7434efa5b 100644 --- a/modules/nf-commons/src/test/nextflow/plugin/extension/PluginExtensionMethodsTest.groovy +++ b/modules/nf-commons/src/test/nextflow/plugin/extension/PluginExtensionMethodsTest.groovy @@ -19,6 +19,7 @@ package nextflow.plugin.extension import java.nio.file.Path +import groovyx.gpars.dataflow.DataflowReadChannel import nextflow.Channel import nextflow.exception.DuplicateModuleFunctionException import nextflow.plugin.Plugins @@ -67,11 +68,11 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: - result.val == 'Bye bye folks' - result.val == Channel.STOP + result.unwrap() == 'Bye bye folks' + result.unwrap() == Channel.STOP } def 'should execute custom operator extension/2' () { @@ -85,11 +86,11 @@ class PluginExtensionMethodsTest extends Dsl2Spec { .goodbye() ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: - result.val == 'Bye bye folks' - result.val == Channel.STOP + result.unwrap() == 'Bye bye folks' + result.unwrap() == Channel.STOP } @@ -102,12 +103,12 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: result - result.val == 'a string'.reverse() - result.val == Channel.STOP + result.unwrap() == 'a string'.reverse() + result.unwrap() == Channel.STOP } @@ -122,12 +123,12 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: result - result.val == 'a string'.reverse() - result.val == Channel.STOP + result.unwrap() == 'a string'.reverse() + result.unwrap() == Channel.STOP } @@ -142,13 +143,13 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: - result.val == 100 - result.val == 200 - result.val == 300 - result.val == Channel.STOP + result.unwrap() == 100 + result.unwrap() == 200 + result.unwrap() == 300 + result.unwrap() == Channel.STOP } def 'should execute custom factory as alias extension' () { @@ -161,12 +162,12 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: result - result.val == 'reverse this string'.reverse() - result.val == Channel.STOP + result.unwrap() == 'reverse this string'.reverse() + result.unwrap() == Channel.STOP } @@ -207,11 +208,11 @@ class PluginExtensionMethodsTest extends Dsl2Spec { def 'should execute custom functions'() { when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: - result.val == EXPECTED - result.val == Channel.STOP + result.unwrap() == EXPECTED + result.unwrap() == Channel.STOP where: SCRIPT_TEXT | EXPECTED @@ -223,13 +224,13 @@ class PluginExtensionMethodsTest extends Dsl2Spec { def 'should call init plugin in custom functions'() { when: - def result = dsl_eval(""" + dsl_eval(""" include { sayHello } from 'plugin/nf-test-plugin-hello' sayHello() """) then: - true + noExceptionThrown() } def 'should throw function not found'() { @@ -279,11 +280,11 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT_TEXT) + def result = dsl_eval(SCRIPT_TEXT) as DataflowReadChannel then: - result.val == 'hi' - result.val == Channel.STOP + result.unwrap() == 'hi' + result.unwrap() == Channel.STOP } def 'should not include a non annotated function'() { @@ -334,10 +335,10 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT) + def result = dsl_eval(SCRIPT) as DataflowReadChannel then: - result.val == 'hi' + result.unwrap() == 'hi' } @@ -390,10 +391,10 @@ class PluginExtensionMethodsTest extends Dsl2Spec { ''' when: - def result = dsl_eval(SCRIPT) + def result = dsl_eval(SCRIPT) as DataflowReadChannel then: - result.val == 'hola' + result.unwrap() == 'hola' } def 'should execute custom functions and channel extension at the same time'() { @@ -403,11 +404,11 @@ class PluginExtensionMethodsTest extends Dsl2Spec { SCRIPT.text = SCRIPT_TEXT when: - def result = dsl_eval(SCRIPT) + def result = dsl_eval(SCRIPT) as DataflowReadChannel then: - result.val == EXPECTED - result.val == Channel.STOP + result.unwrap() == EXPECTED + result.unwrap() == Channel.STOP where: SCRIPT_TEXT | EXPECTED diff --git a/modules/nf-commons/src/testFixtures/groovy/nextflow/plugin/hello/HelloExtension.groovy b/modules/nf-commons/src/testFixtures/groovy/nextflow/plugin/hello/HelloExtension.groovy index bbfc82a514..6f35191931 100644 --- a/modules/nf-commons/src/testFixtures/groovy/nextflow/plugin/hello/HelloExtension.groovy +++ b/modules/nf-commons/src/testFixtures/groovy/nextflow/plugin/hello/HelloExtension.groovy @@ -27,11 +27,10 @@ import nextflow.Global import nextflow.NF import nextflow.Session import nextflow.extension.CH -import nextflow.extension.DataflowHelper +import nextflow.extension.SubscribeOp import nextflow.plugin.extension.Function import nextflow.plugin.extension.Operator import nextflow.plugin.extension.PluginExtensionPoint - /** * @author : jorge * @@ -106,7 +105,10 @@ class HelloExtension extends PluginExtensionPoint { final done = { target.bind(Channel.STOP) } - DataflowHelper.subscribeImpl(source, [onNext: next, onComplete: done]) + new SubscribeOp() + .withSource(source) + .withEvents([onNext: next, onComplete: done]) + .apply() return target } diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/GoogleLifeSciencesScriptLauncherTest.groovy b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/GoogleLifeSciencesScriptLauncherTest.groovy index 4273b9f30c..44bfecdfef 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/GoogleLifeSciencesScriptLauncherTest.groovy +++ b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/GoogleLifeSciencesScriptLauncherTest.groovy @@ -21,6 +21,7 @@ import java.nio.file.Files import nextflow.Session import nextflow.cloud.google.GoogleSpecification import nextflow.processor.TaskBean +import nextflow.processor.TaskId import nextflow.util.MustacheTemplateEngine /** * @@ -72,6 +73,7 @@ class GoogleLifeSciencesScriptLauncherTest extends GoogleSpecification { } } def bean = [name: 'Hello 1', + taskId: TaskId.of(10), script: 'echo Hello world!', workDir: WORK_DIR] as TaskBean /* diff --git a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt index c7382062a1..5226420f71 100644 --- a/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt +++ b/plugins/nf-google/src/test/nextflow/cloud/google/lifesciences/bash-wrapper-gcp.txt @@ -1,5 +1,6 @@ #!/bin/bash ### --- +### id: '10' ### name: 'Hello 1' ### ... set -e