From 4d246e35aa4b32f7489b088641cd62c39b3c1eb8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Mon, 28 Apr 2025 13:00:22 +0200 Subject: [PATCH 01/13] qs --- .../interpreter/InterpreterSetup.scala | 3 +- .../flink/ResultCollectingListenerSpec.scala | 318 +++++++++++++++++- .../registrar/FlinkProcessRegistrar.scala | 4 + .../engine/api/process/ModelSettings.scala | 13 + .../api/process/ProcessConfigCreator.scala | 10 + .../src/universal/conf/dev-application.conf | 1 + .../engine/canonize/MaybeArtificial.scala | 9 + .../engine/canonize/ProcessCanonizer.scala | 2 +- .../engine/compile/ProcessCompiler.scala | 17 +- .../nodecompilation/NodeCompiler.scala | 68 +++- .../definition/model/ModelDefinition.scala | 10 +- ...DefinitionFromConfigCreatorExtractor.scala | 8 +- .../testing/ModelDefinitionBuilder.scala | 5 +- .../nussknacker/engine/InterpreterSpec.scala | 3 +- .../compile/CustomNodeValidationSpec.scala | 1 + .../GenericTransformationValidationSpec.scala | 3 +- 16 files changed, 438 insertions(+), 37 deletions(-) create mode 100644 extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala diff --git a/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala b/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala index b06114c3912..1bb2d27e661 100644 --- a/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala +++ b/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala @@ -83,7 +83,8 @@ class InterpreterSetup[T: ClassTag] { ComponentDefinitionExtractionMode.FinalDefinition ), ModelDefinitionBuilder.emptyExpressionConfig, - ClassExtractionSettings.Default + ClassExtractionSettings.Default, + ModelSettings.Default, ) val definitionsWithTypes = ModelDefinitionWithClasses(definitions) diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala index fd8de9c1ae0..f4e372f63ae 100644 --- a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala @@ -5,16 +5,21 @@ import com.typesafe.scalalogging.LazyLogging import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers +import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, ProcessListener, Service} import pl.touk.nussknacker.engine.api.component.ComponentDefinition -import pl.touk.nussknacker.engine.api.process.SourceFactory +import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo +import pl.touk.nussknacker.engine.api.process._ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode._ import pl.touk.nussknacker.engine.compile.FragmentResolver +import pl.touk.nussknacker.engine.flink.ResultCollectingListenerSpec.AllowingToEndScenarioWithoutSink import pl.touk.nussknacker.engine.flink.test.FlinkSpec import pl.touk.nussknacker.engine.flink.test.ScalatestMiniClusterJobStatusCheckingOps.miniClusterWithServicesToOps import pl.touk.nussknacker.engine.flink.util.source.EmitWatermarkAfterEachElementCollectionSource import pl.touk.nussknacker.engine.flink.util.transformer.FlinkBaseComponentProvider import pl.touk.nussknacker.engine.flink.util.transformer.aggregate.AggregateWindowsConfig +import pl.touk.nussknacker.engine.graph.node import pl.touk.nussknacker.engine.process.helpers.ConfigCreatorWithCollectingListener import pl.touk.nussknacker.engine.process.runner.FlinkScenarioUnitTestJob import pl.touk.nussknacker.engine.testing.LocalModelData @@ -25,6 +30,7 @@ import pl.touk.nussknacker.test.VeryPatientScalaFutures import java.time.{Duration, Instant} import scala.jdk.CollectionConverters._ +import scala.util.Try class ResultCollectingListenerSpec extends AnyFunSuite @@ -109,6 +115,79 @@ class ResultCollectingListenerSpec ) } + test("union of two sources with additional variable in only one of the branches - without sinks") { + val scenario = removeSinks( + ScenarioBuilder + .streaming("sample-union") + .sources( + GraphBuilder + .source("start-foo", "start1") + .branchEnd("foo", "union"), + GraphBuilder + .source("start-bar", "start2") + .buildSimpleVariable("bv1", "customVariableInBarBranch", "#input/2".spel) + .branchEnd("bar", "union"), + GraphBuilder + .join( + "union", + "union", + Some("dataIsFrom"), + List( + "foo" -> List("Output expression" -> "'foo source'".spel), + "bar" -> List("Output expression" -> "'bar source'".spel) + ) + ) + .emptySink("end", "dead-end") // sink must be created by the DSL, but is then removed + ) + ) + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-union", 8) + transitionVariables(testResults, "start-foo", Some("union")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "start-bar", Some("bv1")) shouldBe Set( + Map("input" -> 100), + Map("input" -> 200), + Map("input" -> 300), + Map("input" -> 400), + ) + transitionVariables(testResults, "bv1", Some("union")) shouldBe Set( + Map("input" -> 100, "customVariableInBarBranch" -> 50), + Map("input" -> 200, "customVariableInBarBranch" -> 100), + Map("input" -> 300, "customVariableInBarBranch" -> 150), + Map("input" -> 400, "customVariableInBarBranch" -> 200), + ) + transitionVariables(testResults, "union", Some("artificialDeadEndSink-after-union")) shouldBe Set( + Map("input" -> 10, "dataIsFrom" -> "foo source"), + Map("input" -> 20, "dataIsFrom" -> "foo source"), + Map("input" -> 30, "dataIsFrom" -> "foo source"), + Map("input" -> 40, "dataIsFrom" -> "foo source"), + Map("input" -> 100, "customVariableInBarBranch" -> 50, "dataIsFrom" -> "bar source"), + Map("input" -> 200, "customVariableInBarBranch" -> 100, "dataIsFrom" -> "bar source"), + Map("input" -> 300, "customVariableInBarBranch" -> 150, "dataIsFrom" -> "bar source"), + Map("input" -> 400, "customVariableInBarBranch" -> 200, "dataIsFrom" -> "bar source"), + ) + transitionVariables(testResults, "artificialDeadEndSink-after-union", None) shouldBe Set( + Map("input" -> 10, "dataIsFrom" -> "foo source"), + Map("input" -> 20, "dataIsFrom" -> "foo source"), + Map("input" -> 30, "dataIsFrom" -> "foo source"), + Map("input" -> 40, "dataIsFrom" -> "foo source"), + Map("input" -> 100, "customVariableInBarBranch" -> 50, "dataIsFrom" -> "bar source"), + Map("input" -> 200, "customVariableInBarBranch" -> 100, "dataIsFrom" -> "bar source"), + Map("input" -> 300, "customVariableInBarBranch" -> 150, "dataIsFrom" -> "bar source"), + Map("input" -> 400, "customVariableInBarBranch" -> 200, "dataIsFrom" -> "bar source"), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + test("there is for-each node") { val scenario = ScenarioBuilder .streaming("sample-for-each") @@ -215,6 +294,96 @@ class ResultCollectingListenerSpec ) } + test("there is a split - fail to compile without sinks") { + val scenario = removeSinks( + ScenarioBuilder + .streaming("sample-split") + .source("start-foo", "start1") + .split( + "split", + GraphBuilder + .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) + .emptySink("end1", "dead-end"), + GraphBuilder + .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) + .emptySink("end2", "dead-end") + ) + ) + + catchExceptionMessage( + withCollectingTestResults(scenario, _ => ()) + ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv1)), InvalidTailOfBranch(Set(bv2))" + + } + + test("there is a split - without sinks") { + val scenario = removeSinks( + ScenarioBuilder + .streaming("sample-split") + .source("start-foo", "start1") + .split( + "split", + GraphBuilder + .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) + .emptySink("end1", "dead-end"), + GraphBuilder + .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) + .emptySink("end2", "dead-end") + ) + ) + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-bv1", 4) + assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-bv2", 4) + transitionVariables(testResults, "start-foo", Some("split")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "split", Some("bv1")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "split", Some("bv2")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "bv1", Some("artificialDeadEndSink-after-bv1")) shouldBe Set( + Map("input" -> 10, "timesTwo" -> 20), + Map("input" -> 20, "timesTwo" -> 40), + Map("input" -> 30, "timesTwo" -> 60), + Map("input" -> 40, "timesTwo" -> 80), + ) + transitionVariables(testResults, "bv2", Some("artificialDeadEndSink-after-bv2")) shouldBe Set( + Map("input" -> 10, "timesFour" -> 40), + Map("input" -> 20, "timesFour" -> 80), + Map("input" -> 30, "timesFour" -> 120), + Map("input" -> 40, "timesFour" -> 160), + ) + transitionVariables(testResults, "artificialDeadEndSink-after-bv1", None) shouldBe Set( + Map("input" -> 10, "timesTwo" -> 20), + Map("input" -> 20, "timesTwo" -> 40), + Map("input" -> 30, "timesTwo" -> 60), + Map("input" -> 40, "timesTwo" -> 80), + ) + transitionVariables(testResults, "artificialDeadEndSink-after-bv2", None) shouldBe Set( + Map("input" -> 10, "timesFour" -> 40), + Map("input" -> 20, "timesFour" -> 80), + Map("input" -> 30, "timesFour" -> 120), + Map("input" -> 40, "timesFour" -> 160), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + test("there is a fragment") { val scenarioWithFragment = ScenarioBuilder .streaming("sample-scenario-with-fragment") @@ -280,6 +449,78 @@ class ResultCollectingListenerSpec ) } + test("there is a fragment - without sinks") { + val scenarioWithFragment = removeSinks( + ScenarioBuilder + .streaming("sample-scenario-with-fragment") + .source("source", "start1") + .fragment( + "sub", + "fragment1", + List("fragment1_input" -> "#input".spel), + Map("output" -> "fragmentResult"), + Map("output" -> GraphBuilder.emptySink("end", "dead-end")) + ) + ) + + val fragment = ScenarioBuilder + .fragment("fragment1", "fragment1_input" -> classOf[Int]) + .filter("filter", "#fragment1_input != 10".spel) + .fragmentOutput("fragmentEnd", "output", "output" -> "#fragment1_input".spel) + + val scenario = FragmentResolver(List(fragment)).resolve(scenarioWithFragment).toOption.get + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-sub-fragmentEnd", 3) + assertNumberOfSamplesThatFinishedInNode(testResults, "sub-filter", 1) + transitionVariables(testResults, "source", Some("sub")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( + Map("fragment1_input" -> 10), + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( + Map("fragment1_input" -> 10), + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + // This sample is filtered out and does not proceed further + transitionVariables(testResults, "sub-filter", None) shouldBe Set( + Map("fragment1_input" -> 10), + ) + transitionVariables(testResults, "sub-filter", Some("sub-fragmentEnd")) shouldBe Set( + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + transitionVariables( + testResults, + "sub-fragmentEnd", + Some("artificialDeadEndSink-after-sub-fragmentEnd") + ) shouldBe Set( + Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), + Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), + Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), + ) + transitionVariables(testResults, "artificialDeadEndSink-after-sub-fragmentEnd", None) shouldBe Set( + Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), + Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), + Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + private def transitionVariables(testResults: TestResults[Any], fromNodeId: String, toNodeId: Option[String]) = testResults .nodeTransitionResults(NodeTransition(fromNodeId, toNodeId)) @@ -299,10 +540,11 @@ class ResultCollectingListenerSpec private def withCollectingTestResults( canonicalProcess: CanonicalProcess, - assertions: TestResults[Any] => Unit + assertions: TestResults[Any] => Unit, + allowEndingScenarioWithoutSink: Boolean = false, ): Unit = { ResultsCollectingListenerHolder.withListener { collectingListener => - val model = modelData(collectingListener) + val model = modelData(collectingListener, AggregateWindowsConfig.Default, allowEndingScenarioWithoutSink) flinkMiniCluster.withDetachedStreamExecutionEnvironment { env => val executionResult = new FlinkScenarioUnitTestJob(model).run(canonicalProcess, env) flinkMiniCluster.waitForJobIsFinished(executionResult.getJobID) @@ -313,12 +555,16 @@ class ResultCollectingListenerSpec private def modelData( collectingListener: => ResultsCollectingListener[Any], - aggregateWindowsConfig: AggregateWindowsConfig = AggregateWindowsConfig.Default, + aggregateWindowsConfig: AggregateWindowsConfig, + allowEndingScenarioWithoutSink: Boolean, ): LocalModelData = { def sourceComponent(data: List[Int]) = SourceFactory.noParamUnboundedStreamFactory[Int]( EmitWatermarkAfterEachElementCollectionSource .create[Int](data, _ => Instant.now.toEpochMilli, Duration.ofHours(1)) ) + val configCreator = new ConfigCreatorWithCollectingListener(collectingListener) + val decoratedConfigCreator = + if (allowEndingScenarioWithoutSink) new AllowingToEndScenarioWithoutSink(configCreator) else configCreator LocalModelData( ConfigFactory.empty(), ComponentDefinition("start1", sourceComponent(data1)) :: @@ -327,8 +573,70 @@ class ResultCollectingListenerSpec DocsConfig.Default, aggregateWindowsConfig ) ::: FlinkBaseComponentProvider.Components, - configCreator = new ConfigCreatorWithCollectingListener(collectingListener) + configCreator = decoratedConfigCreator, ) } + private def removeSinks(canonicalProcess: CanonicalProcess): CanonicalProcess = { + canonicalProcess.mapAllNodes(removeSinks) + } + + private def removeSinks(nodes: List[CanonicalNode]): List[CanonicalNode] = { + nodes.flatMap { + case FlatNode(_: node.Sink) => + None + case FlatNode(other) => + Some(FlatNode(other)) + case SplitNode(data, nexts) => + Some(SplitNode(data, nexts.map(removeSinks))) + case FilterNode(data, nextFalse) => + Some(FilterNode(data, removeSinks(nextFalse))) + case SwitchNode(data, cases, defaultNext) => + Some(SwitchNode(data, cases.map(c => Case(c.expression, removeSinks(c.nodes))), removeSinks(defaultNext))) + case Fragment(data, outputs) => + Some(Fragment(data, outputs.toList.map { case (key, value) => (key, removeSinks(value)) }.toMap)) + } + } + + private def catchExceptionMessage(f: => Any): String = Try(f).failed.get.getMessage + +} + +object ResultCollectingListenerSpec { + + class AllowingToEndScenarioWithoutSink(underlying: ProcessConfigCreator) extends ProcessConfigCreator { + + override def modelSettings(modelDependencies: ProcessObjectDependencies): ModelSettings = { + ModelSettings(allowEndingScenarioWithoutSink = true) + } + + override def customStreamTransformers( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[CustomStreamTransformer]] = + underlying.customStreamTransformers(modelDependencies) + + override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = + underlying.services(modelDependencies) + + override def sourceFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SourceFactory]] = + underlying.sourceFactories(modelDependencies) + + override def sinkFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SinkFactory]] = + underlying.sinkFactories(modelDependencies) + + override def listeners(modelDependencies: ProcessObjectDependencies): Seq[ProcessListener] = + underlying.listeners(modelDependencies) + + override def expressionConfig(modelDependencies: ProcessObjectDependencies): ExpressionConfig = + underlying.expressionConfig(modelDependencies) + + override def modelInfo(): ModelInfo = + underlying.modelInfo() + + } + } diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala index 89368e5e4bd..2067eea94ef 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala @@ -14,6 +14,7 @@ import pl.touk.nussknacker.engine.api.component.NodeComponentInfo import pl.touk.nussknacker.engine.api.context.{JoinContextTransformation, ValidationContext} import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.ArtificialDeadEndSink import pl.touk.nussknacker.engine.compiledgraph.part._ import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.FlinkScenarioCompilationDependencies @@ -21,6 +22,7 @@ import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters import pl.touk.nussknacker.engine.flink.api.compat.ExplicitUidInOperatorsSupport import pl.touk.nussknacker.engine.flink.api.process._ import pl.touk.nussknacker.engine.flink.api.typeinformation.TypeInformationDetection +import pl.touk.nussknacker.engine.flink.util.sink.EmptySink import pl.touk.nussknacker.engine.graph.node.{BranchEndDefinition, NodeData} import pl.touk.nussknacker.engine.node.NodeComponentInfoExtractor.fromScenarioNode import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkCompatibilityProvider, FlinkJobConfig} @@ -256,6 +258,8 @@ class FlinkProcessRegistrar( processPart match { case part @ SinkPart(sink: FlinkSink, _, contextBefore, _) => registerSinkPark(start, part, sink, contextBefore) + case part @ SinkPart(ArtificialDeadEndSink, _, contextBefore, _) => + registerSinkPark(start, part, EmptySink, contextBefore) case part: SinkPart => // TODO: fixme "part.obj" is not stringified well // (eg. Scenario can only use flink sinks, instead given: pl.touk.nussknacker.engine.management.sample.sink.LiteDeadEndSink$@21220fd7) diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala new file mode 100644 index 00000000000..5c04319322f --- /dev/null +++ b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala @@ -0,0 +1,13 @@ +package pl.touk.nussknacker.engine.api.process + +/** + * General settings for model + * @param allowEndingScenarioWithoutSink - indicates, whether it is allowed to have nodes other than sinks as final nodes of the scenario + */ +final case class ModelSettings( + allowEndingScenarioWithoutSink: Boolean, +) + +object ModelSettings { + val Default: ModelSettings = ModelSettings(allowEndingScenarioWithoutSink = false) +} diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala index 28b6c19cd0c..4070ed93504 100644 --- a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala +++ b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala @@ -3,6 +3,8 @@ package pl.touk.nussknacker.engine.api.process import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, ProcessListener, Service} import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo +import scala.util.Try + /** * There Nussknacker fetches information about user defined model. * Any invocation of user defined logic or resource goes through this class. @@ -32,4 +34,12 @@ trait ProcessConfigCreator extends Serializable { def classExtractionSettings(modelDependencies: ProcessObjectDependencies): ClassExtractionSettings = ClassExtractionSettings.Default + def modelSettings(modelDependencies: ProcessObjectDependencies): ModelSettings = { + Try( + ModelSettings( + allowEndingScenarioWithoutSink = modelDependencies.config.getBoolean("allowEndingScenarioWithoutSink") + ) + ).getOrElse(ModelSettings.Default) + } + } diff --git a/nussknacker-dist/src/universal/conf/dev-application.conf b/nussknacker-dist/src/universal/conf/dev-application.conf index bc599bf6f2f..d502b0f66f0 100644 --- a/nussknacker-dist/src/universal/conf/dev-application.conf +++ b/nussknacker-dist/src/universal/conf/dev-application.conf @@ -130,6 +130,7 @@ scenarioTypes { useDataSampleParamForSchemalessJsonTopicBasedKafkaSource: true } } + allowEndingScenarioWithoutSink: true } category: "Default" diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala index 97d30155eb4..e50d378d10b 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala @@ -56,4 +56,13 @@ private[engine] object MaybeArtificial { node.SourceNode(node.Source(generateArtificialName(), SourceRef(artificalSourceSinkRef, Nil)), _) ) + def artificialDeadEndSink(previousNodeId: String): MaybeArtificial[node.SubsequentNode] = + new MaybeArtificial( + node.EndingNode(new ArtificialDeadEndSink(previousNodeId)), + List.empty, + ) + + class ArtificialDeadEndSink(val previousNodeId: String) + extends node.Sink(s"artificialDeadEndSink-after-$previousNodeId", SinkRef(artificalSourceSinkRef, Nil), None) + } diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala index e5aa39a6424..97e4c7351df 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala @@ -108,7 +108,7 @@ object ProcessCanonizer { MaybeArtificial.artificialSink(InvalidTailOfBranch(invalidHead.id)) case Nil => - MaybeArtificial.artificialSink(InvalidTailOfBranch(previous.id)) + MaybeArtificial.artificialDeadEndSink(previous.id) } } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala index 0873937edfe..1a6909db63c 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala @@ -2,7 +2,6 @@ package pl.touk.nussknacker.engine.compile import cats.data.{NonEmptyList, Validated, ValidatedNel} import cats.data.Validated._ -import cats.instances.list._ import com.typesafe.scalalogging.LazyLogging import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api.{JobData, NodeId} @@ -251,11 +250,17 @@ protected trait ProcessCompilerBase { implicit scenarioCompilationDependencies: ScenarioCompilationDependencies, nodeId: NodeId ): CompilationResult[part.SinkPart] = { - val NodeCompilationResult(typingInfo, parameters, _, compiledSink, _) = nodeCompiler.compileSink(node.data, ctx) - val nodeTypingInfo = Map(node.id -> NodeTypingInfo(ctx, typingInfo, parameters)) - CompilationResult.map2(sub.validate(node, ctx), CompilationResult(nodeTypingInfo, compiledSink))((_, obj) => - compiledgraph.part.SinkPart(obj, node, ctx, ctx) - ) + nodeCompiler + .compileSinkOption(node.data, ctx) + .map { case NodeCompilationResult(typingInfo, parameters, _, compiledSink, _) => + val nodeTypingInfo = Map(node.id -> NodeTypingInfo(ctx, typingInfo, parameters)) + CompilationResult.map2(sub.validate(node, ctx), CompilationResult(nodeTypingInfo, compiledSink))((_, obj) => + compiledgraph.part.SinkPart(obj, node, ctx, ctx) + ) + } + .getOrElse( + nodeCompiler.handleMissingSinkFactory(node) + ) } def compileEndingCustomNodePart( diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala index 2b006242499..367d85c4e35 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala @@ -15,13 +15,9 @@ import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process.Source import pl.touk.nussknacker.engine.api.typed.ReturningType import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} -import pl.touk.nussknacker.engine.compile.{ - ComponentExecutorFactory, - ExpressionCompiler, - FragmentSourceWithTestWithParametersSupportFactory, - NodeValidationExceptionHandler -} -import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult +import pl.touk.nussknacker.engine.canonize.MaybeArtificial +import pl.touk.nussknacker.engine.compile._ +import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.{ArtificialDeadEndSink, NodeCompilationResult} import pl.touk.nussknacker.engine.compiledgraph.{CompiledParameter, TypedParameter} import pl.touk.nussknacker.engine.definition.component.ComponentDefinitionWithImplementation import pl.touk.nussknacker.engine.definition.component.dynamic.{ @@ -40,6 +36,7 @@ import pl.touk.nussknacker.engine.graph.node._ import pl.touk.nussknacker.engine.graph.service.ServiceRef import pl.touk.nussknacker.engine.resultcollector.ResultCollector import pl.touk.nussknacker.engine.spel.SpelExpressionParser +import pl.touk.nussknacker.engine.splittedgraph.splittednode.EndingNode import pl.touk.nussknacker.engine.variables.GlobalVariablesPreparer import shapeless.Typeable import shapeless.syntax.typeable._ @@ -60,6 +57,8 @@ object NodeCompiler { } + object ArtificialDeadEndSink extends api.process.Sink + } class NodeCompiler( @@ -272,21 +271,60 @@ class NodeCompiler( implicit nodeId: NodeId, scenarioCompilationDependencies: ScenarioCompilationDependencies ): NodeCompilationResult[api.process.Sink] = { + compileSinkOption(sink, ctx).getOrElse { + val error = invalid(MissingSinkFactory(sink.ref.typ)).toValidatedNel + NodeCompilationResult(Map.empty[String, ExpressionTypingInfo], None, Valid(ctx), error) + } + } + + def compileSinkOption( + sink: Sink, + ctx: ValidationContext + )( + implicit nodeId: NodeId, + scenarioCompilationDependencies: ScenarioCompilationDependencies + ): Option[NodeCompilationResult[api.process.Sink]] = { val ref = sink.ref - definitions.getComponent(ComponentType.Sink, ref.typ) match { - case Some(definition) => + definitions + .getComponent(ComponentType.Sink, ref.typ) + .map { compileComponentWithContextTransformation[api.process.Sink]( sink.parameters, Nil, Left(ctx), None, - definition, + _, _ => Valid(ctx) ).map(_._1) - case None => - val error = invalid(MissingSinkFactory(sink.ref.typ)).toValidatedNel - NodeCompilationResult(Map.empty[String, ExpressionTypingInfo], None, Valid(ctx), error) + } + } + + def handleMissingSinkFactory( + sinkNode: EndingNode[Sink], + )( + implicit nodeId: NodeId, + scenarioCompilationDependencies: ScenarioCompilationDependencies + ): CompilationResult[compiledgraph.part.SinkPart] = { + import scenarioCompilationDependencies._ + + lazy val allowEndingScenarioWithoutSink = definitions.modelSettings.allowEndingScenarioWithoutSink + lazy val isFragment = metaData.typeSpecificData.isFragment + + sinkNode.data match { + case _: MaybeArtificial.ArtificialDeadEndSink if allowEndingScenarioWithoutSink && !isFragment => + CompilationResult.pure( + compiledgraph.part.SinkPart( + obj = ArtificialDeadEndSink, + node = sinkNode, + contextBefore = ValidationContext.empty, + validationContext = ValidationContext.empty + ) + ) + case sink: MaybeArtificial.ArtificialDeadEndSink => + compilationErrorResult(InvalidTailOfBranch(Set(sink.previousNodeId))) + case _ => + compilationErrorResult(MissingSinkFactory(sinkNode.data.ref.typ)) } } @@ -407,6 +445,10 @@ class NodeCompiler( } } + private def compilationErrorResult[T](error: ProcessCompilationError): CompilationResult[T] = { + CompilationResult[T](Invalid(NonEmptyList.of[ProcessCompilationError](error))) + } + private def compileEagerService( serviceRef: ServiceRef, componentDefinition: ComponentDefinitionWithImplementation, diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala index f7e0b6a71ba..90bf54a8fac 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala @@ -2,14 +2,15 @@ package pl.touk.nussknacker.engine.definition.model import pl.touk.nussknacker.engine.api.component.ComponentId import pl.touk.nussknacker.engine.api.component.ComponentType.ComponentType -import pl.touk.nussknacker.engine.api.process.ClassExtractionSettings +import pl.touk.nussknacker.engine.api.process.{ClassExtractionSettings, ModelSettings} import pl.touk.nussknacker.engine.definition.component.{ComponentDefinitionWithImplementation, Components} import pl.touk.nussknacker.engine.definition.globalvariables.ExpressionConfigDefinition case class ModelDefinition private ( components: Components, expressionConfig: ExpressionConfigDefinition, - settings: ClassExtractionSettings + settings: ClassExtractionSettings, + modelSettings: ModelSettings, ) { def withComponent(component: ComponentDefinitionWithImplementation): ModelDefinition = { @@ -48,7 +49,8 @@ object ModelDefinition { def apply( components: Components, expressionConfig: ExpressionConfigDefinition, - settings: ClassExtractionSettings, - ): ModelDefinition = new ModelDefinition(components, expressionConfig, settings) + classExtractionSettings: ClassExtractionSettings, + modelSettings: ModelSettings, + ): ModelDefinition = new ModelDefinition(components, expressionConfig, classExtractionSettings, modelSettings) } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala index 452af5afd03..be15d787919 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala @@ -49,12 +49,14 @@ object ModelDefinitionFromConfigCreatorExtractor { componentDefinitionExtractionMode, ) - val settings = creator.classExtractionSettings(modelDependencies) + val classExtractionSettings = creator.classExtractionSettings(modelDependencies) + val modelSettings = creator.modelSettings(modelDependencies) ModelDefinition( components, toDefinition(expressionConfig, categoryOpt), - settings + classExtractionSettings, + modelSettings, ) } @@ -104,7 +106,7 @@ object ModelDefinitionFromConfigCreatorExtractor { expressionConfig.methodExecutionForUnknownAllowed, expressionConfig.dynamicPropertyAccessAllowed, expressionConfig.spelExpressionExcludeList, - expressionConfig.customConversionsProviders + expressionConfig.customConversionsProviders, ) } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala index 4e1cf473e39..4873e9977e3 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala @@ -9,7 +9,7 @@ import pl.touk.nussknacker.engine.api.component.{ } import pl.touk.nussknacker.engine.api.component.Component.AllowedProcessingModes import pl.touk.nussknacker.engine.api.definition.Parameter -import pl.touk.nussknacker.engine.api.process.ClassExtractionSettings +import pl.touk.nussknacker.engine.api.process.{ClassExtractionSettings, ModelSettings} import pl.touk.nussknacker.engine.api.process.ExpressionConfig._ import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} import pl.touk.nussknacker.engine.definition.component._ @@ -205,7 +205,8 @@ final case class ModelDefinitionBuilder( ModelDefinition( Components.empty(componentDefinitionExtractionMode).withComponents(components), emptyExpressionConfig.copy(globalVariables = globalVariablesDefinition), - ClassExtractionSettings.Default + ClassExtractionSettings.Default, + ModelSettings.Default, ) } diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala index b51baca9b19..2bb9e4c806a 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala @@ -196,7 +196,8 @@ class InterpreterSpec extends AnyFunSuite with Matchers { ComponentDefinitionExtractionMode.FinalDefinition ), ModelDefinitionBuilder.emptyExpressionConfig, - ClassExtractionSettings.Default + ClassExtractionSettings.Default, + ModelSettings.Default, ) val definitionsWithTypes = ModelDefinitionWithClasses(definitions) ProcessCompilerData.prepare( diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala index 743f127e214..d6889d10b6d 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala @@ -61,6 +61,7 @@ class CustomNodeValidationSpec extends AnyFunSuite with Matchers with OptionValu ), ModelDefinitionBuilder.emptyExpressionConfig, ClassExtractionSettings.Default, + ModelSettings.Default, ) private val validator = ProcessValidator.default( diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala index cc328109aa2..236f517aa07 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala @@ -67,7 +67,8 @@ class GenericTransformationValidationSpec extends AnyFunSuite with Matchers with ComponentDefinitionExtractionMode.FinalDefinition ), ModelDefinitionBuilder.emptyExpressionConfig, - ClassExtractionSettings.Default + ClassExtractionSettings.Default, + ModelSettings.Default, ) private val validator = ProcessValidator.default( From 8aaedb14f55e060da702d8bf1a6ca4e795805acb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Mon, 28 Apr 2025 18:08:07 +0200 Subject: [PATCH 02/13] hopefully fix --- .../engine/canonize/ProcessCanonizer.scala | 4 +- .../marshall/ProcessMarshallerSpec.scala | 37 +++++++++++++++---- .../engine/compile/CompilationResult.scala | 16 +++++++- 3 files changed, 47 insertions(+), 10 deletions(-) diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala index 97e4c7351df..da74d69e42b 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala @@ -96,8 +96,8 @@ object ProcessCanonizer { node.SwitchNode(data, nextsV, Some(defaultNextV)) } - case (a @ canonicalnode.SplitNode(bare, Nil)) :: Nil => - MaybeArtificial.artificialSink(InvalidTailOfBranch(bare.id)) + case canonicalnode.SplitNode(bare, Nil) :: Nil => + MaybeArtificial.artificialDeadEndSink(bare.id) case (a @ canonicalnode.SplitNode(bare, nexts)) :: Nil => nexts.map(uncanonize(a, _)).sequence[MaybeArtificial, node.SubsequentNode].map { uncanonized => diff --git a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala index cb602a7d4f1..337c556f835 100644 --- a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala +++ b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala @@ -1,7 +1,6 @@ package pl.touk.nussknacker.engine.marshall -import cats.data.NonEmptyList -import cats.data.Validated.{Invalid, Valid} +import cats.data.Validated.Valid import io.circe.{Codec, Json} import io.circe.generic.extras.semiauto.deriveConfiguredCodec import io.circe.syntax._ @@ -9,15 +8,14 @@ import org.scalatest.{Inside, OptionValues} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import org.scalatest.prop.TableDrivenPropertyChecks -import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api._ -import pl.touk.nussknacker.engine.api.CirceUtil._ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.{canonicalnode, CanonicalProcess} import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.{CanonicalNode, FlatNode} -import pl.touk.nussknacker.engine.canonize.ProcessCanonizer +import pl.touk.nussknacker.engine.canonize.{MaybeArtificial, ProcessCanonizer} import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.expression.Expression.Language +import pl.touk.nussknacker.engine.graph.node import pl.touk.nussknacker.engine.graph.node._ import pl.touk.nussknacker.engine.graph.source.SourceRef @@ -223,10 +221,35 @@ class ProcessMarshallerSpec it should "detect bad branch" in { def checkOneInvalid(expectedBadNodeId: String, nodes: CanonicalNode*) = { - inside(ProcessCanonizer.uncanonize(CanonicalProcess(MetaData("1", StreamMetaData()), nodes.toList, List.empty))) { - case Invalid(NonEmptyList(canonize.InvalidTailOfBranch(id), Nil)) => id shouldBe expectedBadNodeId + inside( + ProcessCanonizer + .uncanonize(CanonicalProcess(MetaData("1", StreamMetaData()), nodes.toList, List.empty)) + .map(ProcessCanonizer.canonize) + .map(canonicalProcess => sinks(canonicalProcess.nodes)) + ) { case Valid(sinks) => + sinks.collectFirst { + case sink: MaybeArtificial.ArtificialDeadEndSink if sink.previousNodeId == expectedBadNodeId => () + } shouldBe defined } } + + def sinks(nodes: List[CanonicalNode]): List[node.Sink] = { + nodes.flatMap { + case canonicalnode.FlatNode(sink: node.Sink) => + List(sink) + case canonicalnode.FlatNode(_) => + List.empty + case canonicalnode.SplitNode(_, nexts) => + nexts.flatMap(sinks) + case canonicalnode.FilterNode(_, nextFalse) => + sinks(nextFalse) + case canonicalnode.SwitchNode(_, cases, defaultNext) => + cases.flatMap(c => sinks(c.nodes)) ::: sinks(defaultNext) + case canonicalnode.Fragment(_, outputs) => + outputs.toList.flatMap { case (_, value) => sinks(value) } + } + } + val source = FlatNode(Source("s1", SourceRef("a", List()))) checkOneInvalid("filter", source, canonicalnode.FilterNode(Filter("filter", Expression(Language.Spel, "")), List())) diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala index 1f14162e6f6..f96e940b15f 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala @@ -115,11 +115,25 @@ object CompilationResult extends Applicative[CompilationResult] { case head :: tail => rawValue.copy( typing = rawValue.typing.filterKeysNow(key => !key.startsWith(MaybeArtificial.DummyObjectNamePrefix)), - result = Invalid(fromUncanonizationErrors(NonEmptyList(head, tail))) + result = Invalid(fromUncanonizationErrors(NonEmptyList(head, tail ++ uncanonizationErrors(rawValue)))) ) } } + private def uncanonizationErrors[A]( + compilationResult: CompilationResult[A] + ): List[canonize.ProcessUncanonizationError] = { + compilationResult.result match { + case Valid(_) => List.empty + case Invalid(e) => + e.collect { + case EmptyProcess => List(canonize.EmptyProcess) + case InvalidRootNode(nodeIds) => nodeIds.map(canonize.InvalidRootNode).toList + case InvalidTailOfBranch(nodeIds) => nodeIds.map(canonize.InvalidTailOfBranch).toList + }.flatten + } + } + implicit def mergingTypingInfoSemigroup: Semigroup[NodeTypingInfo] = new Semigroup[NodeTypingInfo] with LazyLogging { override def combine(x: NodeTypingInfo, y: NodeTypingInfo): NodeTypingInfo = { From 7ccbf28f0b965f9c9baf966f1eea33035f995137 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Mon, 28 Apr 2025 19:46:07 +0200 Subject: [PATCH 03/13] qs --- .../nussknacker/engine/marshall/ProcessMarshallerSpec.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala index 337c556f835..7c6c60990a1 100644 --- a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala +++ b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala @@ -8,7 +8,9 @@ import org.scalatest.{Inside, OptionValues} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import org.scalatest.prop.TableDrivenPropertyChecks +import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api._ +import pl.touk.nussknacker.engine.api.CirceUtil._ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.{canonicalnode, CanonicalProcess} import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.{CanonicalNode, FlatNode} From 9f2e637f2ae049fa54dfac92727b5dbd88b537b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Tue, 29 Apr 2025 10:52:54 +0200 Subject: [PATCH 04/13] qs --- .../engine/canonize/MaybeArtificial.scala | 6 +- .../engine/canonize/ProcessCanonizer.scala | 67 +++++++++++++------ .../marshall/ProcessMarshallerSpec.scala | 35 +++------- .../engine/compile/CompilationResult.scala | 16 +---- .../engine/compile/ProcessCompiler.scala | 41 +++++++----- .../nodecompilation/NodeCompiler.scala | 64 +++++------------- 6 files changed, 96 insertions(+), 133 deletions(-) diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala index e50d378d10b..0250a6e77ad 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala @@ -45,18 +45,18 @@ private[engine] object MaybeArtificial { // we need to make sure it's unique to prevent weird errors private def generateArtificialName() = s"$DummyObjectNamePrefix-${UUID.randomUUID()}" - def artificialSink(errors: ProcessUncanonizationError*): MaybeArtificial[node.SubsequentNode] = + def missingSinkError(errors: ProcessUncanonizationError*): MaybeArtificial[node.SubsequentNode] = new MaybeArtificial( node.EndingNode(node.Sink(generateArtificialName(), SinkRef(artificalSourceSinkRef, Nil), None)), errors.toList ) def artificialSource(errors: ProcessUncanonizationError*): MaybeArtificial[node.SourceNode] = - artificialSink(errors: _*).map( + missingSinkError(errors: _*).map( node.SourceNode(node.Source(generateArtificialName(), SourceRef(artificalSourceSinkRef, Nil)), _) ) - def artificialDeadEndSink(previousNodeId: String): MaybeArtificial[node.SubsequentNode] = + def addedArtificialDeadEndSink(previousNodeId: String): MaybeArtificial[node.SubsequentNode] = new MaybeArtificial( node.EndingNode(new ArtificialDeadEndSink(previousNodeId)), List.empty, diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala index da74d69e42b..74fb3a7ae5c 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala @@ -22,21 +22,30 @@ object ProcessCanonizer { ) } - def uncanonize(canonicalProcess: CanonicalProcess): ValidatedNel[ProcessUncanonizationError, EspProcess] = - uncanonizeArtificial(canonicalProcess).toValidNel + def uncanonize( + canonicalProcess: CanonicalProcess, + allowEndingScenarioWithoutSink: Boolean, + ): ValidatedNel[ProcessUncanonizationError, EspProcess] = + uncanonizeArtificial(canonicalProcess, allowEndingScenarioWithoutSink).toValidNel - def uncanonizeArtificial(canonicalProcess: CanonicalProcess): MaybeArtificial[EspProcess] = { + def uncanonizeArtificial( + canonicalProcess: CanonicalProcess, + allowEndingScenarioWithoutSink: Boolean, + ): MaybeArtificial[EspProcess] = { val branches: MaybeArtificial[NonEmptyList[pl.touk.nussknacker.engine.graph.node.SourceNode]] = - canonicalProcess.allStartNodes.map(uncanonizeSource).sequence + canonicalProcess.allStartNodes.map(uncanonizeSource(_, allowEndingScenarioWithoutSink)).sequence branches.map(bList => EspProcess(canonicalProcess.metaData, bList)) } - private def uncanonizeSource(canonicalNode: List[canonicalnode.CanonicalNode]): MaybeArtificial[node.SourceNode] = + private def uncanonizeSource( + canonicalNode: List[canonicalnode.CanonicalNode], + allowEndingScenarioWithoutSink: Boolean, + ): MaybeArtificial[node.SourceNode] = canonicalNode match { case (a @ canonicalnode.FlatNode(data: node.StartingNodeData)) :: tail => - uncanonize(a, tail).map(node.SourceNode(data, _)) + uncanonize(a, tail, allowEndingScenarioWithoutSink).map(node.SourceNode(data, _)) case other :: _ => MaybeArtificial.artificialSource(InvalidRootNode(other.id)) @@ -47,7 +56,8 @@ object ProcessCanonizer { private def uncanonize( previous: canonicalnode.CanonicalNode, - canonicalNode: List[canonicalnode.CanonicalNode] + canonicalNode: List[canonicalnode.CanonicalNode], + allowEndingScenarioWithoutSink: Boolean, ): MaybeArtificial[node.SubsequentNode] = canonicalNode match { case canonicalnode.FlatNode(data: node.BranchEndData) :: Nil => @@ -57,30 +67,31 @@ object ProcessCanonizer { new MaybeArtificial(node.EndingNode(data), Nil) case (a @ canonicalnode.FlatNode(data: node.OneOutputSubsequentNodeData)) :: tail => - uncanonize(a, tail).map(node.OneOutputSubsequentNode(data, _)) + uncanonize(a, tail, allowEndingScenarioWithoutSink).map(node.OneOutputSubsequentNode(data, _)) case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail if nextFalse.isEmpty => - uncanonize(a, tail).map(nextTrue => node.FilterNode(data, Some(nextTrue), None)) + uncanonize(a, tail, allowEndingScenarioWithoutSink).map(nextTrue => node.FilterNode(data, Some(nextTrue), None)) case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail if tail.isEmpty => - uncanonize(a, nextFalse).map { nextFalseV => + uncanonize(a, nextFalse, allowEndingScenarioWithoutSink).map { nextFalseV => node.FilterNode(data, None, Some(nextFalseV)) } case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail => - (uncanonize(a, tail), uncanonize(a, nextFalse)).mapN { (nextTrue, nextFalseV) => - node.FilterNode(data, Some(nextTrue), Some(nextFalseV)) - } + (uncanonize(a, tail, allowEndingScenarioWithoutSink), uncanonize(a, nextFalse, allowEndingScenarioWithoutSink)) + .mapN { (nextTrue, nextFalseV) => + node.FilterNode(data, Some(nextTrue), Some(nextFalseV)) + } case (a @ canonicalnode.SwitchNode(data, Nil, defaultNext)) :: Nil => - uncanonize(a, defaultNext).map { defaultNextV => + uncanonize(a, defaultNext, allowEndingScenarioWithoutSink).map { defaultNextV => node.SwitchNode(data, Nil, Some(defaultNextV)) } case (a @ canonicalnode.SwitchNode(data, nexts, defaultNext)) :: Nil if defaultNext.isEmpty => nexts .map { casee => - uncanonize(a, casee.nodes).map(node.Case(casee.expression, _)) + uncanonize(a, casee.nodes, allowEndingScenarioWithoutSink).map(node.Case(casee.expression, _)) } .sequence[MaybeArtificial, node.Case] .map(node.SwitchNode(data, _, None)) @@ -88,29 +99,41 @@ object ProcessCanonizer { case (a @ canonicalnode.SwitchNode(data, nexts, defaultNext)) :: Nil => val unFlattenNexts = nexts .map { casee => - uncanonize(a, casee.nodes).map(node.Case(casee.expression, _)) + uncanonize(a, casee.nodes, allowEndingScenarioWithoutSink).map(node.Case(casee.expression, _)) } .sequence[MaybeArtificial, node.Case] - (unFlattenNexts, uncanonize(a, defaultNext)).mapN { (nextsV, defaultNextV) => + (unFlattenNexts, uncanonize(a, defaultNext, allowEndingScenarioWithoutSink)).mapN { (nextsV, defaultNextV) => node.SwitchNode(data, nextsV, Some(defaultNextV)) } case canonicalnode.SplitNode(bare, Nil) :: Nil => - MaybeArtificial.artificialDeadEndSink(bare.id) + handleMissingSink(bare.id, allowEndingScenarioWithoutSink) case (a @ canonicalnode.SplitNode(bare, nexts)) :: Nil => - nexts.map(uncanonize(a, _)).sequence[MaybeArtificial, node.SubsequentNode].map { uncanonized => - node.SplitNode(bare, uncanonized) + nexts.map(uncanonize(a, _, allowEndingScenarioWithoutSink)).sequence[MaybeArtificial, node.SubsequentNode].map { + uncanonized => + node.SplitNode(bare, uncanonized) } case invalidHead :: _ => - MaybeArtificial.artificialSink(InvalidTailOfBranch(invalidHead.id)) + MaybeArtificial.missingSinkError(InvalidTailOfBranch(invalidHead.id)) case Nil => - MaybeArtificial.artificialDeadEndSink(previous.id) + handleMissingSink(previous.id, allowEndingScenarioWithoutSink) } + private def handleMissingSink( + previousNodeId: String, + allowEndingScenarioWithoutSink: Boolean + ): MaybeArtificial[node.SubsequentNode] = { + if (allowEndingScenarioWithoutSink) { + MaybeArtificial.addedArtificialDeadEndSink(previousNodeId) + } else { + MaybeArtificial.missingSinkError(InvalidTailOfBranch(previousNodeId)) + } + } + } object NodeCanonizer { diff --git a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala index 7c6c60990a1..095ab65d2a5 100644 --- a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala +++ b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala @@ -1,6 +1,7 @@ package pl.touk.nussknacker.engine.marshall -import cats.data.Validated.Valid +import cats.data.NonEmptyList +import cats.data.Validated.{Invalid, Valid} import io.circe.{Codec, Json} import io.circe.generic.extras.semiauto.deriveConfiguredCodec import io.circe.syntax._ @@ -224,34 +225,14 @@ class ProcessMarshallerSpec def checkOneInvalid(expectedBadNodeId: String, nodes: CanonicalNode*) = { inside( - ProcessCanonizer - .uncanonize(CanonicalProcess(MetaData("1", StreamMetaData()), nodes.toList, List.empty)) - .map(ProcessCanonizer.canonize) - .map(canonicalProcess => sinks(canonicalProcess.nodes)) - ) { case Valid(sinks) => - sinks.collectFirst { - case sink: MaybeArtificial.ArtificialDeadEndSink if sink.previousNodeId == expectedBadNodeId => () - } shouldBe defined - } - } - - def sinks(nodes: List[CanonicalNode]): List[node.Sink] = { - nodes.flatMap { - case canonicalnode.FlatNode(sink: node.Sink) => - List(sink) - case canonicalnode.FlatNode(_) => - List.empty - case canonicalnode.SplitNode(_, nexts) => - nexts.flatMap(sinks) - case canonicalnode.FilterNode(_, nextFalse) => - sinks(nextFalse) - case canonicalnode.SwitchNode(_, cases, defaultNext) => - cases.flatMap(c => sinks(c.nodes)) ::: sinks(defaultNext) - case canonicalnode.Fragment(_, outputs) => - outputs.toList.flatMap { case (_, value) => sinks(value) } + ProcessCanonizer.uncanonize( + CanonicalProcess(MetaData("1", StreamMetaData()), nodes.toList, List.empty), + allowEndingScenarioWithoutSink = false + ) + ) { case Invalid(NonEmptyList(canonize.InvalidTailOfBranch(id), Nil)) => + id shouldBe expectedBadNodeId } } - val source = FlatNode(Source("s1", SourceRef("a", List()))) checkOneInvalid("filter", source, canonicalnode.FilterNode(Filter("filter", Expression(Language.Spel, "")), List())) diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala index f96e940b15f..1f14162e6f6 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/CompilationResult.scala @@ -115,25 +115,11 @@ object CompilationResult extends Applicative[CompilationResult] { case head :: tail => rawValue.copy( typing = rawValue.typing.filterKeysNow(key => !key.startsWith(MaybeArtificial.DummyObjectNamePrefix)), - result = Invalid(fromUncanonizationErrors(NonEmptyList(head, tail ++ uncanonizationErrors(rawValue)))) + result = Invalid(fromUncanonizationErrors(NonEmptyList(head, tail))) ) } } - private def uncanonizationErrors[A]( - compilationResult: CompilationResult[A] - ): List[canonize.ProcessUncanonizationError] = { - compilationResult.result match { - case Valid(_) => List.empty - case Invalid(e) => - e.collect { - case EmptyProcess => List(canonize.EmptyProcess) - case InvalidRootNode(nodeIds) => nodeIds.map(canonize.InvalidRootNode).toList - case InvalidTailOfBranch(nodeIds) => nodeIds.map(canonize.InvalidTailOfBranch).toList - }.flatten - } - } - implicit def mergingTypingInfoSemigroup: Semigroup[NodeTypingInfo] = new Semigroup[NodeTypingInfo] with LazyLogging { override def combine(x: NodeTypingInfo, y: NodeTypingInfo): NodeTypingInfo = { diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala index 1a6909db63c..e1865c3495b 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala @@ -10,10 +10,10 @@ import pl.touk.nussknacker.engine.api.context._ import pl.touk.nussknacker.engine.api.context.ProcessCompilationError._ import pl.touk.nussknacker.engine.api.dict.DictRegistry import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.canonize.ProcessCanonizer +import pl.touk.nussknacker.engine.canonize.{MaybeArtificial, ProcessCanonizer} import pl.touk.nussknacker.engine.compile.FragmentValidator.validateUniqueFragmentOutputNames import pl.touk.nussknacker.engine.compile.nodecompilation.{LazyParameterCreationStrategy, NodeCompiler} -import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult +import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.{ArtificialDeadEndSink, NodeCompilationResult} import pl.touk.nussknacker.engine.compiledgraph.{part, CompiledProcessParts} import pl.touk.nussknacker.engine.compiledgraph.part.{PotentiallyStartPart, TypedEnd} import pl.touk.nussknacker.engine.definition.fragment.FragmentParametersDefinitionExtractor @@ -114,7 +114,10 @@ protected trait ProcessCompilerBase { ): CompilationResult[CompiledProcessParts] = { ThreadUtils.withContextClassLoader(classLoader) { val compilationResultWithArtificial = - ProcessCanonizer.uncanonizeArtificial(process).map(ProcessSplitter.split).map(compile) + ProcessCanonizer + .uncanonizeArtificial(process, nodeCompiler.allowEndingScenarioWithoutSink) + .map(ProcessSplitter.split) + .map(compile) compilationResultWithArtificial.extract } } @@ -188,14 +191,22 @@ protected trait ProcessCompilerBase { implicit scenarioCompilationDependencies: ScenarioCompilationDependencies ): CompilationResult[List[compiledgraph.part.SubsequentPart]] = { import CompilationResult._ - parts - .map(p => + parts.map { + case SinkPart(node @ EndingNode(_: MaybeArtificial.ArtificialDeadEndSink)) => + CompilationResult.pure( + compiledgraph.part.SinkPart( + obj = ArtificialDeadEndSink, + node = node, + contextBefore = ValidationContext.empty, + validationContext = ValidationContext.empty + ) + ) + case p => ctx .get(p.id) .map(compileSubsequentPart(p, _)) .getOrElse(CompilationResult(Invalid(NonEmptyList.of[ProcessCompilationError](MissingPart(p.id))))) - ) - .sequence + }.sequence } private def compileSubsequentPart(part: SubsequentPart, ctx: ValidationContext)( @@ -250,17 +261,11 @@ protected trait ProcessCompilerBase { implicit scenarioCompilationDependencies: ScenarioCompilationDependencies, nodeId: NodeId ): CompilationResult[part.SinkPart] = { - nodeCompiler - .compileSinkOption(node.data, ctx) - .map { case NodeCompilationResult(typingInfo, parameters, _, compiledSink, _) => - val nodeTypingInfo = Map(node.id -> NodeTypingInfo(ctx, typingInfo, parameters)) - CompilationResult.map2(sub.validate(node, ctx), CompilationResult(nodeTypingInfo, compiledSink))((_, obj) => - compiledgraph.part.SinkPart(obj, node, ctx, ctx) - ) - } - .getOrElse( - nodeCompiler.handleMissingSinkFactory(node) - ) + val NodeCompilationResult(typingInfo, parameters, _, compiledSink, _) = nodeCompiler.compileSink(node.data, ctx) + val nodeTypingInfo = Map(node.id -> NodeTypingInfo(ctx, typingInfo, parameters)) + CompilationResult.map2(sub.validate(node, ctx), CompilationResult(nodeTypingInfo, compiledSink))((_, obj) => + compiledgraph.part.SinkPart(obj, node, ctx, ctx) + ) } def compileEndingCustomNodePart( diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala index 367d85c4e35..c17f9830797 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala @@ -15,9 +15,8 @@ import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process.Source import pl.touk.nussknacker.engine.api.typed.ReturningType import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} -import pl.touk.nussknacker.engine.canonize.MaybeArtificial import pl.touk.nussknacker.engine.compile._ -import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.{ArtificialDeadEndSink, NodeCompilationResult} +import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult import pl.touk.nussknacker.engine.compiledgraph.{CompiledParameter, TypedParameter} import pl.touk.nussknacker.engine.definition.component.ComponentDefinitionWithImplementation import pl.touk.nussknacker.engine.definition.component.dynamic.{ @@ -36,7 +35,6 @@ import pl.touk.nussknacker.engine.graph.node._ import pl.touk.nussknacker.engine.graph.service.ServiceRef import pl.touk.nussknacker.engine.resultcollector.ResultCollector import pl.touk.nussknacker.engine.spel.SpelExpressionParser -import pl.touk.nussknacker.engine.splittedgraph.splittednode.EndingNode import pl.touk.nussknacker.engine.variables.GlobalVariablesPreparer import shapeless.Typeable import shapeless.syntax.typeable._ @@ -73,6 +71,15 @@ class NodeCompiler( nonServicesLazyParamStrategy: LazyParameterCreationStrategy, ) { + def allowEndingScenarioWithoutSink( + implicit scenarioCompilationDependencies: ScenarioCompilationDependencies + ): Boolean = { + import scenarioCompilationDependencies._ + lazy val allowEndingScenarioWithoutSink = definitions.modelSettings.allowEndingScenarioWithoutSink + lazy val isFragment = metaData.typeSpecificData.isFragment + allowEndingScenarioWithoutSink && !isFragment + } + def withLabelsDictTyper: NodeCompiler = { new NodeCompiler( definitions, @@ -271,60 +278,21 @@ class NodeCompiler( implicit nodeId: NodeId, scenarioCompilationDependencies: ScenarioCompilationDependencies ): NodeCompilationResult[api.process.Sink] = { - compileSinkOption(sink, ctx).getOrElse { - val error = invalid(MissingSinkFactory(sink.ref.typ)).toValidatedNel - NodeCompilationResult(Map.empty[String, ExpressionTypingInfo], None, Valid(ctx), error) - } - } - - def compileSinkOption( - sink: Sink, - ctx: ValidationContext - )( - implicit nodeId: NodeId, - scenarioCompilationDependencies: ScenarioCompilationDependencies - ): Option[NodeCompilationResult[api.process.Sink]] = { val ref = sink.ref - definitions - .getComponent(ComponentType.Sink, ref.typ) - .map { + definitions.getComponent(ComponentType.Sink, ref.typ) match { + case Some(definition) => compileComponentWithContextTransformation[api.process.Sink]( sink.parameters, Nil, Left(ctx), None, - _, + definition, _ => Valid(ctx) ).map(_._1) - } - } - - def handleMissingSinkFactory( - sinkNode: EndingNode[Sink], - )( - implicit nodeId: NodeId, - scenarioCompilationDependencies: ScenarioCompilationDependencies - ): CompilationResult[compiledgraph.part.SinkPart] = { - import scenarioCompilationDependencies._ - - lazy val allowEndingScenarioWithoutSink = definitions.modelSettings.allowEndingScenarioWithoutSink - lazy val isFragment = metaData.typeSpecificData.isFragment - - sinkNode.data match { - case _: MaybeArtificial.ArtificialDeadEndSink if allowEndingScenarioWithoutSink && !isFragment => - CompilationResult.pure( - compiledgraph.part.SinkPart( - obj = ArtificialDeadEndSink, - node = sinkNode, - contextBefore = ValidationContext.empty, - validationContext = ValidationContext.empty - ) - ) - case sink: MaybeArtificial.ArtificialDeadEndSink => - compilationErrorResult(InvalidTailOfBranch(Set(sink.previousNodeId))) - case _ => - compilationErrorResult(MissingSinkFactory(sinkNode.data.ref.typ)) + case None => + val error = invalid(MissingSinkFactory(sink.ref.typ)).toValidatedNel + NodeCompilationResult(Map.empty[String, ExpressionTypingInfo], None, Valid(ctx), error) } } From 1edb07029b13827467e1a9dedec7f2f07e8622bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Tue, 29 Apr 2025 11:01:03 +0200 Subject: [PATCH 05/13] qs --- .../engine/process/registrar/FlinkProcessRegistrar.scala | 2 +- .../touk/nussknacker/engine/compile/ProcessCompiler.scala | 4 +++- .../engine/compile/nodecompilation/NodeCompiler.scala | 6 ------ 3 files changed, 4 insertions(+), 8 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala index 2067eea94ef..d086e07ba67 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala @@ -14,7 +14,7 @@ import pl.touk.nussknacker.engine.api.component.NodeComponentInfo import pl.touk.nussknacker.engine.api.context.{JoinContextTransformation, ValidationContext} import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.ArtificialDeadEndSink +import pl.touk.nussknacker.engine.compile.ArtificialDeadEndSink import pl.touk.nussknacker.engine.compiledgraph.part._ import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.FlinkScenarioCompilationDependencies diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala index e1865c3495b..0fb15a72380 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala @@ -13,7 +13,7 @@ import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.canonize.{MaybeArtificial, ProcessCanonizer} import pl.touk.nussknacker.engine.compile.FragmentValidator.validateUniqueFragmentOutputNames import pl.touk.nussknacker.engine.compile.nodecompilation.{LazyParameterCreationStrategy, NodeCompiler} -import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.{ArtificialDeadEndSink, NodeCompilationResult} +import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult import pl.touk.nussknacker.engine.compiledgraph.{part, CompiledProcessParts} import pl.touk.nussknacker.engine.compiledgraph.part.{PotentiallyStartPart, TypedEnd} import pl.touk.nussknacker.engine.definition.fragment.FragmentParametersDefinitionExtractor @@ -410,3 +410,5 @@ object ProcessValidator { } } + +object ArtificialDeadEndSink extends api.process.Sink diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala index c17f9830797..6e01e9614b6 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala @@ -55,8 +55,6 @@ object NodeCompiler { } - object ArtificialDeadEndSink extends api.process.Sink - } class NodeCompiler( @@ -413,10 +411,6 @@ class NodeCompiler( } } - private def compilationErrorResult[T](error: ProcessCompilationError): CompilationResult[T] = { - CompilationResult[T](Invalid(NonEmptyList.of[ProcessCompilationError](error))) - } - private def compileEagerService( serviceRef: ServiceRef, componentDefinition: ComponentDefinitionWithImplementation, From af931753f38cbc21d73ec8de9c3d74ceddf7f7d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Tue, 29 Apr 2025 11:33:19 +0200 Subject: [PATCH 06/13] qs --- .../nussknacker/engine/flink/ResultCollectingListenerSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala index f4e372f63ae..5ed20989e63 100644 --- a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala @@ -312,7 +312,7 @@ class ResultCollectingListenerSpec catchExceptionMessage( withCollectingTestResults(scenario, _ => ()) - ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv1)), InvalidTailOfBranch(Set(bv2))" + ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv2, bv1))" } From c24cf792ba90d867ccba53157ce9600f08515fd0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Tue, 29 Apr 2025 12:28:18 +0200 Subject: [PATCH 07/13] docs --- docs/Changelog.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/Changelog.md b/docs/Changelog.md index 73c0ca3fab4..f0a7f101f75 100644 --- a/docs/Changelog.md +++ b/docs/Changelog.md @@ -162,6 +162,9 @@ * [#7953](https://github.com/TouK/nussknacker/pull/7953) Add implicit conversion from records produced by kafka source to map. Also typing of spel expressions was refactored, which improved typing of selection on a map. * [#8005](https://github.com/TouK/nussknacker/pull/8005) Add implicit conversion from records produced by table api source to map. +* [#8004](https://github.com/TouK/nussknacker/pull/8004) Scenarios no longer have to end with final `Sink` node + * set `modelConfig.allowEndingScenarioWithoutSink` of the scenarioType in the `scenarioTypes` config section to `true` in order to allow ending scenarios with nodes other than sinks + * the flag is optional, the default value of the flag is `false` (no changes in behavior) ## 1.18 From 00b02a039d9f78415cd147d737f1691e32054853 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Wed, 30 Apr 2025 14:20:59 +0200 Subject: [PATCH 08/13] review changes --- .../interpreter/InterpreterSetup.scala | 2 +- .../process/ProcessObjectDependencies.scala | 30 +++++++++- .../flink/ResultCollectingListenerSpec.scala | 55 +++---------------- .../FlinkKafkaComponentProvider.scala | 2 +- .../engine/api/process/ModelSettings.scala | 13 ----- .../api/process/ProcessConfigCreator.scala | 10 ---- .../nodecompilation/NodeCompiler.scala | 2 +- .../definition/model/ModelDefinition.scala | 9 +-- ...DefinitionFromConfigCreatorExtractor.scala | 3 +- .../testing/ModelDefinitionBuilder.scala | 4 +- .../nussknacker/engine/InterpreterSpec.scala | 3 +- .../compile/CustomNodeValidationSpec.scala | 2 +- .../GenericTransformationValidationSpec.scala | 2 +- 13 files changed, 50 insertions(+), 87 deletions(-) delete mode 100644 extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala diff --git a/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala b/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala index 1bb2d27e661..b9294367cce 100644 --- a/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala +++ b/benchmarks/src/test/scala/pl/touk/nussknacker/engine/benchmarks/interpreter/InterpreterSetup.scala @@ -84,7 +84,7 @@ class InterpreterSetup[T: ClassTag] { ), ModelDefinitionBuilder.emptyExpressionConfig, ClassExtractionSettings.Default, - ModelSettings.Default, + allowEndingScenarioWithoutSink = false, ) val definitionsWithTypes = ModelDefinitionWithClasses(definitions) diff --git a/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala b/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala index c5b98561a1a..ad5f79c737d 100644 --- a/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala +++ b/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala @@ -1,15 +1,39 @@ package pl.touk.nussknacker.engine.api.process import com.typesafe.config.Config +import net.ceedubs.ficus.Ficus.toFicusConfig +import net.ceedubs.ficus.readers.AnyValReaders._ +import net.ceedubs.ficus.readers.OptionReader._ import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig // TODO: Rename to ModelDependencies + rename config to modelConfig -final case class ProcessObjectDependencies private (config: Config, namingStrategy: NamingStrategy) extends Serializable +final case class ProcessObjectDependencies private (modelConfig: ModelConfig, namingStrategy: NamingStrategy) + extends Serializable { + def config: Config = modelConfig.underlyingConfig +} object ProcessObjectDependencies { - def withConfig(modelConfig: Config): ProcessObjectDependencies = { - ProcessObjectDependencies(modelConfig, NamingStrategy.fromConfig(modelConfig)) + def apply(underlyingConfig: Config, namingStrategy: NamingStrategy): ProcessObjectDependencies = { + ProcessObjectDependencies(parseModelConfig(underlyingConfig), namingStrategy) + } + + def withConfig(config: Config): ProcessObjectDependencies = { + ProcessObjectDependencies(parseModelConfig(config), NamingStrategy.fromConfig(config)) } + def parseModelConfig(modelConfig: Config): ModelConfig = { + ModelConfig( + allowEndingScenarioWithoutSink = modelConfig.getOrElse[Boolean]("allowEndingScenarioWithoutSink", false), + underlyingConfig = modelConfig, + ) + } + + final case class ModelConfig( + allowEndingScenarioWithoutSink: Boolean, + // TODO: we should parse this underlying config as ModelConfig class fields instead of passing raw config + underlyingConfig: Config, + ) + } diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala index 5ed20989e63..62f2fc7459d 100644 --- a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala @@ -5,15 +5,12 @@ import com.typesafe.scalalogging.LazyLogging import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers -import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, ProcessListener, Service} import pl.touk.nussknacker.engine.api.component.ComponentDefinition -import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo import pl.touk.nussknacker.engine.api.process._ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode._ import pl.touk.nussknacker.engine.compile.FragmentResolver -import pl.touk.nussknacker.engine.flink.ResultCollectingListenerSpec.AllowingToEndScenarioWithoutSink import pl.touk.nussknacker.engine.flink.test.FlinkSpec import pl.touk.nussknacker.engine.flink.test.ScalatestMiniClusterJobStatusCheckingOps.miniClusterWithServicesToOps import pl.touk.nussknacker.engine.flink.util.source.EmitWatermarkAfterEachElementCollectionSource @@ -562,18 +559,21 @@ class ResultCollectingListenerSpec EmitWatermarkAfterEachElementCollectionSource .create[Int](data, _ => Instant.now.toEpochMilli, Duration.ofHours(1)) ) - val configCreator = new ConfigCreatorWithCollectingListener(collectingListener) - val decoratedConfigCreator = - if (allowEndingScenarioWithoutSink) new AllowingToEndScenarioWithoutSink(configCreator) else configCreator + val config = + if (allowEndingScenarioWithoutSink) { + ConfigFactory.parseString("""allowEndingScenarioWithoutSink: true""") + } else { + ConfigFactory.empty() + } LocalModelData( - ConfigFactory.empty(), + config, ComponentDefinition("start1", sourceComponent(data1)) :: ComponentDefinition("start2", sourceComponent(data2)) :: FlinkBaseUnboundedComponentProvider.create( DocsConfig.Default, aggregateWindowsConfig ) ::: FlinkBaseComponentProvider.Components, - configCreator = decoratedConfigCreator, + configCreator = new ConfigCreatorWithCollectingListener(collectingListener), ) } @@ -601,42 +601,3 @@ class ResultCollectingListenerSpec private def catchExceptionMessage(f: => Any): String = Try(f).failed.get.getMessage } - -object ResultCollectingListenerSpec { - - class AllowingToEndScenarioWithoutSink(underlying: ProcessConfigCreator) extends ProcessConfigCreator { - - override def modelSettings(modelDependencies: ProcessObjectDependencies): ModelSettings = { - ModelSettings(allowEndingScenarioWithoutSink = true) - } - - override def customStreamTransformers( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[CustomStreamTransformer]] = - underlying.customStreamTransformers(modelDependencies) - - override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = - underlying.services(modelDependencies) - - override def sourceFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SourceFactory]] = - underlying.sourceFactories(modelDependencies) - - override def sinkFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SinkFactory]] = - underlying.sinkFactories(modelDependencies) - - override def listeners(modelDependencies: ProcessObjectDependencies): Seq[ProcessListener] = - underlying.listeners(modelDependencies) - - override def expressionConfig(modelDependencies: ProcessObjectDependencies): ExpressionConfig = - underlying.expressionConfig(modelDependencies) - - override def modelInfo(): ModelInfo = - underlying.modelInfo() - - } - -} diff --git a/engine/flink/components/kafka/src/main/scala/pl/touk/nussknacker/engine/flink/util/transformer/FlinkKafkaComponentProvider.scala b/engine/flink/components/kafka/src/main/scala/pl/touk/nussknacker/engine/flink/util/transformer/FlinkKafkaComponentProvider.scala index 35a99d9c435..add185ceb8a 100644 --- a/engine/flink/components/kafka/src/main/scala/pl/touk/nussknacker/engine/flink/util/transformer/FlinkKafkaComponentProvider.scala +++ b/engine/flink/components/kafka/src/main/scala/pl/touk/nussknacker/engine/flink/util/transformer/FlinkKafkaComponentProvider.scala @@ -86,7 +86,7 @@ private[transformer] object TemporaryKafkaConfigMapping { val kafkaConfig = config.getConfig("config") val kafkaConfigMergedWithGlobalConfig = dependencies.config.withValue(KafkaConfig.DefaultGlobalKafkaConfigPath, fromAnyRef(kafkaConfig.root())) - dependencies.copy(config = kafkaConfigMergedWithGlobalConfig) + ProcessObjectDependencies.withConfig(kafkaConfigMergedWithGlobalConfig) } } diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala deleted file mode 100644 index 5c04319322f..00000000000 --- a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ModelSettings.scala +++ /dev/null @@ -1,13 +0,0 @@ -package pl.touk.nussknacker.engine.api.process - -/** - * General settings for model - * @param allowEndingScenarioWithoutSink - indicates, whether it is allowed to have nodes other than sinks as final nodes of the scenario - */ -final case class ModelSettings( - allowEndingScenarioWithoutSink: Boolean, -) - -object ModelSettings { - val Default: ModelSettings = ModelSettings(allowEndingScenarioWithoutSink = false) -} diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala index 4070ed93504..28b6c19cd0c 100644 --- a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala +++ b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessConfigCreator.scala @@ -3,8 +3,6 @@ package pl.touk.nussknacker.engine.api.process import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, ProcessListener, Service} import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo -import scala.util.Try - /** * There Nussknacker fetches information about user defined model. * Any invocation of user defined logic or resource goes through this class. @@ -34,12 +32,4 @@ trait ProcessConfigCreator extends Serializable { def classExtractionSettings(modelDependencies: ProcessObjectDependencies): ClassExtractionSettings = ClassExtractionSettings.Default - def modelSettings(modelDependencies: ProcessObjectDependencies): ModelSettings = { - Try( - ModelSettings( - allowEndingScenarioWithoutSink = modelDependencies.config.getBoolean("allowEndingScenarioWithoutSink") - ) - ).getOrElse(ModelSettings.Default) - } - } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala index 6e01e9614b6..78babac8794 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala @@ -73,7 +73,7 @@ class NodeCompiler( implicit scenarioCompilationDependencies: ScenarioCompilationDependencies ): Boolean = { import scenarioCompilationDependencies._ - lazy val allowEndingScenarioWithoutSink = definitions.modelSettings.allowEndingScenarioWithoutSink + lazy val allowEndingScenarioWithoutSink = definitions.allowEndingScenarioWithoutSink lazy val isFragment = metaData.typeSpecificData.isFragment allowEndingScenarioWithoutSink && !isFragment } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala index 90bf54a8fac..c6e70fd3de1 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinition.scala @@ -2,7 +2,7 @@ package pl.touk.nussknacker.engine.definition.model import pl.touk.nussknacker.engine.api.component.ComponentId import pl.touk.nussknacker.engine.api.component.ComponentType.ComponentType -import pl.touk.nussknacker.engine.api.process.{ClassExtractionSettings, ModelSettings} +import pl.touk.nussknacker.engine.api.process.ClassExtractionSettings import pl.touk.nussknacker.engine.definition.component.{ComponentDefinitionWithImplementation, Components} import pl.touk.nussknacker.engine.definition.globalvariables.ExpressionConfigDefinition @@ -10,7 +10,7 @@ case class ModelDefinition private ( components: Components, expressionConfig: ExpressionConfigDefinition, settings: ClassExtractionSettings, - modelSettings: ModelSettings, + allowEndingScenarioWithoutSink: Boolean, ) { def withComponent(component: ComponentDefinitionWithImplementation): ModelDefinition = { @@ -50,7 +50,8 @@ object ModelDefinition { components: Components, expressionConfig: ExpressionConfigDefinition, classExtractionSettings: ClassExtractionSettings, - modelSettings: ModelSettings, - ): ModelDefinition = new ModelDefinition(components, expressionConfig, classExtractionSettings, modelSettings) + allowEndingScenarioWithoutSink: Boolean, + ): ModelDefinition = + new ModelDefinition(components, expressionConfig, classExtractionSettings, allowEndingScenarioWithoutSink) } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala index be15d787919..56c9553ab9a 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/definition/model/ModelDefinitionFromConfigCreatorExtractor.scala @@ -50,13 +50,12 @@ object ModelDefinitionFromConfigCreatorExtractor { ) val classExtractionSettings = creator.classExtractionSettings(modelDependencies) - val modelSettings = creator.modelSettings(modelDependencies) ModelDefinition( components, toDefinition(expressionConfig, categoryOpt), classExtractionSettings, - modelSettings, + modelDependencies.modelConfig.allowEndingScenarioWithoutSink, ) } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala index 4873e9977e3..f1452a143dd 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/testing/ModelDefinitionBuilder.scala @@ -9,7 +9,7 @@ import pl.touk.nussknacker.engine.api.component.{ } import pl.touk.nussknacker.engine.api.component.Component.AllowedProcessingModes import pl.touk.nussknacker.engine.api.definition.Parameter -import pl.touk.nussknacker.engine.api.process.{ClassExtractionSettings, ModelSettings} +import pl.touk.nussknacker.engine.api.process.ClassExtractionSettings import pl.touk.nussknacker.engine.api.process.ExpressionConfig._ import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} import pl.touk.nussknacker.engine.definition.component._ @@ -206,7 +206,7 @@ final case class ModelDefinitionBuilder( Components.empty(componentDefinitionExtractionMode).withComponents(components), emptyExpressionConfig.copy(globalVariables = globalVariablesDefinition), ClassExtractionSettings.Default, - ModelSettings.Default, + allowEndingScenarioWithoutSink = false, ) } diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala index 2bb9e4c806a..556fa6cdaca 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala @@ -29,6 +29,7 @@ import pl.touk.nussknacker.engine.api.exception.NuExceptionInfo import pl.touk.nussknacker.engine.api.expression._ import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process._ +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.api.test.InvocationCollectors import pl.touk.nussknacker.engine.api.test.InvocationCollectors.ServiceInvocationCollector import pl.touk.nussknacker.engine.api.typed.typing @@ -197,7 +198,7 @@ class InterpreterSpec extends AnyFunSuite with Matchers { ), ModelDefinitionBuilder.emptyExpressionConfig, ClassExtractionSettings.Default, - ModelSettings.Default, + allowEndingScenarioWithoutSink = false, ) val definitionsWithTypes = ModelDefinitionWithClasses(definitions) ProcessCompilerData.prepare( diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala index d6889d10b6d..d94e2cc1fae 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/CustomNodeValidationSpec.scala @@ -61,7 +61,7 @@ class CustomNodeValidationSpec extends AnyFunSuite with Matchers with OptionValu ), ModelDefinitionBuilder.emptyExpressionConfig, ClassExtractionSettings.Default, - ModelSettings.Default, + allowEndingScenarioWithoutSink = false, ) private val validator = ProcessValidator.default( diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala index 236f517aa07..46bbf521dea 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/GenericTransformationValidationSpec.scala @@ -68,7 +68,7 @@ class GenericTransformationValidationSpec extends AnyFunSuite with Matchers with ), ModelDefinitionBuilder.emptyExpressionConfig, ClassExtractionSettings.Default, - ModelSettings.Default, + allowEndingScenarioWithoutSink = false, ) private val validator = ProcessValidator.default( From 388f79585e6d1a03aad2afa109073b24e006754a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Mon, 5 May 2025 15:53:29 +0200 Subject: [PATCH 09/13] optional next node - changes in compiler --- .../registrar/FlinkProcessRegistrar.scala | 3 - .../engine/build/GraphBuilder.scala | 34 ++++---- .../engine/canonize/MaybeArtificial.scala | 13 +-- .../engine/canonize/MissingSinkHandler.scala | 32 +++++++ .../engine/canonize/ProcessCanonizer.scala | 83 +++++++------------ .../touk/nussknacker/engine/graph/node.scala | 12 ++- .../marshall/ProcessMarshallerSpec.scala | 5 +- .../touk/nussknacker/engine/Interpreter.scala | 37 +++++---- .../engine/compile/PartSubGraphCompiler.scala | 33 +++++--- .../engine/compile/ProcessCompiler.scala | 22 ++--- .../nodecompilation/NodeCompiler.scala | 8 +- .../CompiledNodesCollector.scala | 13 +-- .../engine/compiledgraph/node.scala | 22 +++-- .../engine/split/ProcessSplitter.scala | 51 ++++++++---- .../SplittedNodesCollector.scala | 4 +- .../engine/splittedgraph/splittednode.scala | 8 +- 16 files changed, 214 insertions(+), 166 deletions(-) create mode 100644 scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala index d086e07ba67..471cc9ba5f5 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala @@ -14,7 +14,6 @@ import pl.touk.nussknacker.engine.api.component.NodeComponentInfo import pl.touk.nussknacker.engine.api.context.{JoinContextTransformation, ValidationContext} import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.compile.ArtificialDeadEndSink import pl.touk.nussknacker.engine.compiledgraph.part._ import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.FlinkScenarioCompilationDependencies @@ -258,8 +257,6 @@ class FlinkProcessRegistrar( processPart match { case part @ SinkPart(sink: FlinkSink, _, contextBefore, _) => registerSinkPark(start, part, sink, contextBefore) - case part @ SinkPart(ArtificialDeadEndSink, _, contextBefore, _) => - registerSinkPark(start, part, EmptySink, contextBefore) case part: SinkPart => // TODO: fixme "part.obj" is not stringified well // (eg. Scenario can only use flink sinks, instead given: pl.touk.nussknacker.engine.management.sample.sink.LiteDeadEndSink$@21220fd7) diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala index 1aaecceffee..2c38b743428 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala @@ -20,34 +20,36 @@ trait GraphBuilder[R] { def build(inner: GraphBuilder.Creator[R]): GraphBuilder[R] def source(id: String, typ: String, params: (String, Expression)*): GraphBuilder[SourceNode] = - new SimpleGraphBuilder( - SourceNode(Source(id, SourceRef(typ, toNodeParameters(params))), _) - ) + new SimpleGraphBuilder(node => SourceNode(Source(id, SourceRef(typ, toNodeParameters(params))), Some(node))) def buildVariable(id: String, varName: String, fields: (String, Expression)*): GraphBuilder[R] = build(node => - creator(OneOutputSubsequentNode(VariableBuilder(id, varName, fields.map(f => Field(f._1, f._2)).toList), node)) + creator( + OneOutputSubsequentNode(VariableBuilder(id, varName, fields.map(f => Field(f._1, f._2)).toList), Some(node)) + ) ) def buildSimpleVariable(id: String, varName: String, value: Expression): GraphBuilder[R] = - build(node => creator(OneOutputSubsequentNode(Variable(id, varName, value), node))) + build(node => creator(OneOutputSubsequentNode(Variable(id, varName, value), Some(node)))) def enricher(id: String, output: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode(Enricher(id, ServiceRef(svcId, toNodeParameters(params)), output), node) + OneOutputSubsequentNode(Enricher(id, ServiceRef(svcId, toNodeParameters(params)), output), Some(node)) ) ) def processor(id: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = - build(node => creator(OneOutputSubsequentNode(Processor(id, ServiceRef(svcId, toNodeParameters(params))), node))) + build(node => + creator(OneOutputSubsequentNode(Processor(id, ServiceRef(svcId, toNodeParameters(params))), Some(node))) + ) def disabledProcessor(id: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( OneOutputSubsequentNode( Processor(id, ServiceRef(svcId, toNodeParameters(params)), isDisabled = Some(true)), - node + Some(node) ) ) ) @@ -98,24 +100,24 @@ trait GraphBuilder[R] { ) def fragmentInput(id: String, params: (String, Class[_])*): GraphBuilder[SourceNode] = - new SimpleGraphBuilder( + new SimpleGraphBuilder(node => SourceNode( FragmentInputDefinition( id = id, parameters = params.map(kv => FragmentParameter(ParameterName(kv._1), FragmentClazzRef(kv._2.getName))).toList ), - _ + Some(node) ) ) def fragmentInputWithRawParameters(id: String, params: FragmentParameter*): GraphBuilder[SourceNode] = - new SimpleGraphBuilder( + new SimpleGraphBuilder(node => SourceNode( FragmentInputDefinition( id = id, parameters = params.toList ), - _ + Some(node) ) ) @@ -172,7 +174,7 @@ trait GraphBuilder[R] { creator( OneOutputSubsequentNode( CustomNode(id, Some(outputVar), customNodeRef, toNodeParameters(params)), - node + Some(node) ) ) ) @@ -184,7 +186,7 @@ trait GraphBuilder[R] { def customNodeNoOutput(id: String, customNodeRef: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode(CustomNode(id, None, customNodeRef, toNodeParameters(params)), node) + OneOutputSubsequentNode(CustomNode(id, None, customNodeRef, toNodeParameters(params)), Some(node)) ) ) @@ -203,7 +205,9 @@ trait GraphBuilder[R] { val branchParameters = branchParams.map { case (branchId, bParams) => BranchParameters(branchId, toNodeParameters(bParams)) } - new SimpleGraphBuilder(SourceNode(node.Join(id, output, typ, toNodeParameters(params), branchParameters), _)) + new SimpleGraphBuilder(n => + SourceNode(node.Join(id, output, typ, toNodeParameters(params), branchParameters), Some(n)) + ) } def decisionTable( diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala index 0250a6e77ad..7e2650f0434 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MaybeArtificial.scala @@ -45,9 +45,9 @@ private[engine] object MaybeArtificial { // we need to make sure it's unique to prevent weird errors private def generateArtificialName() = s"$DummyObjectNamePrefix-${UUID.randomUUID()}" - def missingSinkError(errors: ProcessUncanonizationError*): MaybeArtificial[node.SubsequentNode] = + def missingSinkError(errors: ProcessUncanonizationError*): MaybeArtificial[Option[node.SubsequentNode]] = new MaybeArtificial( - node.EndingNode(node.Sink(generateArtificialName(), SinkRef(artificalSourceSinkRef, Nil), None)), + Some(node.EndingNode(node.Sink(generateArtificialName(), SinkRef(artificalSourceSinkRef, Nil), None))), errors.toList ) @@ -56,13 +56,4 @@ private[engine] object MaybeArtificial { node.SourceNode(node.Source(generateArtificialName(), SourceRef(artificalSourceSinkRef, Nil)), _) ) - def addedArtificialDeadEndSink(previousNodeId: String): MaybeArtificial[node.SubsequentNode] = - new MaybeArtificial( - node.EndingNode(new ArtificialDeadEndSink(previousNodeId)), - List.empty, - ) - - class ArtificialDeadEndSink(val previousNodeId: String) - extends node.Sink(s"artificialDeadEndSink-after-$previousNodeId", SinkRef(artificalSourceSinkRef, Nil), None) - } diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala new file mode 100644 index 00000000000..2d0fca7ddc4 --- /dev/null +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala @@ -0,0 +1,32 @@ +package pl.touk.nussknacker.engine.canonize + +import pl.touk.nussknacker.engine.graph.node + +// This trait and its 2 implementation encapsulate the logic of handling scenarios that do not end with sink +// - older behavior is that error is returned for missing sinks +// - new behavior, which we intend to become the default and only one in the future, is allowing missing sinks +sealed trait MissingSinkHandler { + def handleMissingSink(previousNodeId: String): MaybeArtificial[Option[node.SubsequentNode]] +} + +object MissingSinkHandler { + + object AllowMissingSinkHandler extends MissingSinkHandler { + + override def handleMissingSink( + previousNodeId: String + ): MaybeArtificial[Option[node.SubsequentNode]] = + new MaybeArtificial(None, Nil) + + } + + object DoNotAllowMissingSinkHandler extends MissingSinkHandler { + + override def handleMissingSink( + previousNodeId: String + ): MaybeArtificial[Option[node.SubsequentNode]] = + new MaybeArtificial(None, InvalidTailOfBranch(previousNodeId) :: Nil) + + } + +} diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala index 74fb3a7ae5c..214c180efbe 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala @@ -24,29 +24,27 @@ object ProcessCanonizer { def uncanonize( canonicalProcess: CanonicalProcess, - allowEndingScenarioWithoutSink: Boolean, + missingSinkHandler: MissingSinkHandler, ): ValidatedNel[ProcessUncanonizationError, EspProcess] = - uncanonizeArtificial(canonicalProcess, allowEndingScenarioWithoutSink).toValidNel + uncanonizeArtificial(canonicalProcess, missingSinkHandler).toValidNel def uncanonizeArtificial( canonicalProcess: CanonicalProcess, - allowEndingScenarioWithoutSink: Boolean, + missingSinkHandler: MissingSinkHandler, ): MaybeArtificial[EspProcess] = { val branches: MaybeArtificial[NonEmptyList[pl.touk.nussknacker.engine.graph.node.SourceNode]] = - canonicalProcess.allStartNodes.map(uncanonizeSource(_, allowEndingScenarioWithoutSink)).sequence + canonicalProcess.allStartNodes.map(uncanonizeSource(_)(missingSinkHandler)).sequence branches.map(bList => EspProcess(canonicalProcess.metaData, bList)) } private def uncanonizeSource( canonicalNode: List[canonicalnode.CanonicalNode], - allowEndingScenarioWithoutSink: Boolean, - ): MaybeArtificial[node.SourceNode] = + )(implicit missingSinkHandler: MissingSinkHandler): MaybeArtificial[node.SourceNode] = canonicalNode match { case (a @ canonicalnode.FlatNode(data: node.StartingNodeData)) :: tail => - uncanonize(a, tail, allowEndingScenarioWithoutSink).map(node.SourceNode(data, _)) - + uncanonize(a, tail).map(node.SourceNode(data, _)) case other :: _ => MaybeArtificial.artificialSource(InvalidRootNode(other.id)) @@ -57,83 +55,64 @@ object ProcessCanonizer { private def uncanonize( previous: canonicalnode.CanonicalNode, canonicalNode: List[canonicalnode.CanonicalNode], - allowEndingScenarioWithoutSink: Boolean, - ): MaybeArtificial[node.SubsequentNode] = + )(implicit missingSinkHandler: MissingSinkHandler): MaybeArtificial[Option[node.SubsequentNode]] = canonicalNode match { case canonicalnode.FlatNode(data: node.BranchEndData) :: Nil => - new MaybeArtificial(node.BranchEnd(data), Nil) + new MaybeArtificial(Some(node.BranchEnd(data)), Nil) case canonicalnode.FlatNode(data: node.EndingNodeData) :: Nil => - new MaybeArtificial(node.EndingNode(data), Nil) + new MaybeArtificial(Some(node.EndingNode(data)), Nil) case (a @ canonicalnode.FlatNode(data: node.OneOutputSubsequentNodeData)) :: tail => - uncanonize(a, tail, allowEndingScenarioWithoutSink).map(node.OneOutputSubsequentNode(data, _)) - + uncanonize(a, tail).map(node.OneOutputSubsequentNode(data, _)).map(Some(_): Option[node.SubsequentNode]) case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail if nextFalse.isEmpty => - uncanonize(a, tail, allowEndingScenarioWithoutSink).map(nextTrue => node.FilterNode(data, Some(nextTrue), None)) + uncanonize(a, tail).map(nextTrue => Some(node.FilterNode(data, nextTrue, None))) case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail if tail.isEmpty => - uncanonize(a, nextFalse, allowEndingScenarioWithoutSink).map { nextFalseV => - node.FilterNode(data, None, Some(nextFalseV)) - } + uncanonize(a, nextFalse).map { nextFalseV => Some(node.FilterNode(data, None, nextFalseV)) } case (a @ canonicalnode.FilterNode(data, nextFalse)) :: tail => - (uncanonize(a, tail, allowEndingScenarioWithoutSink), uncanonize(a, nextFalse, allowEndingScenarioWithoutSink)) - .mapN { (nextTrue, nextFalseV) => - node.FilterNode(data, Some(nextTrue), Some(nextFalseV)) - } + (uncanonize(a, tail), uncanonize(a, nextFalse)).mapN { (nextTrue, nextFalseV) => + Some(node.FilterNode(data, nextTrue, nextFalseV)) + } case (a @ canonicalnode.SwitchNode(data, Nil, defaultNext)) :: Nil => - uncanonize(a, defaultNext, allowEndingScenarioWithoutSink).map { defaultNextV => - node.SwitchNode(data, Nil, Some(defaultNextV)) - } + uncanonize(a, defaultNext).map(defaultNextV => Some(node.SwitchNode(data, Nil, defaultNextV))) case (a @ canonicalnode.SwitchNode(data, nexts, defaultNext)) :: Nil if defaultNext.isEmpty => nexts - .map { casee => - uncanonize(a, casee.nodes, allowEndingScenarioWithoutSink).map(node.Case(casee.expression, _)) - } + .map(casee => uncanonize(a, casee.nodes).map(node.Case(casee.expression, _))) .sequence[MaybeArtificial, node.Case] .map(node.SwitchNode(data, _, None)) + .map(Some(_)) case (a @ canonicalnode.SwitchNode(data, nexts, defaultNext)) :: Nil => val unFlattenNexts = nexts - .map { casee => - uncanonize(a, casee.nodes, allowEndingScenarioWithoutSink).map(node.Case(casee.expression, _)) - } + .map(casee => uncanonize(a, casee.nodes).map(node.Case(casee.expression, _))) .sequence[MaybeArtificial, node.Case] - (unFlattenNexts, uncanonize(a, defaultNext, allowEndingScenarioWithoutSink)).mapN { (nextsV, defaultNextV) => - node.SwitchNode(data, nextsV, Some(defaultNextV)) + (unFlattenNexts, uncanonize(a, defaultNext)).mapN { (nextsV, defaultNextV) => + Some(node.SwitchNode(data, nextsV, defaultNextV)) } case canonicalnode.SplitNode(bare, Nil) :: Nil => - handleMissingSink(bare.id, allowEndingScenarioWithoutSink) + missingSinkHandler.handleMissingSink(bare.id) case (a @ canonicalnode.SplitNode(bare, nexts)) :: Nil => - nexts.map(uncanonize(a, _, allowEndingScenarioWithoutSink)).sequence[MaybeArtificial, node.SubsequentNode].map { - uncanonized => - node.SplitNode(bare, uncanonized) - } + nexts + .map(uncanonize(a, _)) + .sequence[MaybeArtificial, Option[node.SubsequentNode]] + .map { uncanonized => + Some(node.SplitNode(bare, uncanonized.flatten)) + } case invalidHead :: _ => MaybeArtificial.missingSinkError(InvalidTailOfBranch(invalidHead.id)) case Nil => - handleMissingSink(previous.id, allowEndingScenarioWithoutSink) + missingSinkHandler.handleMissingSink(previous.id) } - private def handleMissingSink( - previousNodeId: String, - allowEndingScenarioWithoutSink: Boolean - ): MaybeArtificial[node.SubsequentNode] = { - if (allowEndingScenarioWithoutSink) { - MaybeArtificial.addedArtificialDeadEndSink(previousNodeId) - } else { - MaybeArtificial.missingSinkError(InvalidTailOfBranch(previousNodeId)) - } - } - } object NodeCanonizer { @@ -141,14 +120,14 @@ object NodeCanonizer { def canonize(n: node.Node): List[canonicalnode.CanonicalNode] = n match { case oneOut: node.OneOutputNode => - canonicalnode.FlatNode(oneOut.data) :: canonize(oneOut.next) + canonicalnode.FlatNode(oneOut.data) :: oneOut.next.map(canonize).getOrElse(Nil) case node.FilterNode(data, nextTrue, nextFalse) => canonicalnode.FilterNode(data, nextFalse.toList.flatMap(canonize)) :: nextTrue.toList.flatMap(canonize) case node.SwitchNode(data, nexts, defaultNext) => canonicalnode.SwitchNode( data = data, nexts = nexts.map { next => - canonicalnode.Case(next.expression, canonize(next.node)) + canonicalnode.Case(next.expression, next.node.map(canonize).getOrElse(Nil)) }, defaultNext = defaultNext.toList.flatMap(canonize) ) :: Nil diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala index 5572914ca27..7f80c47918e 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala @@ -38,14 +38,14 @@ object node { } sealed trait OneOutputNode extends NodeWithData { - def next: SubsequentNode + def next: Option[SubsequentNode] } - case class SourceNode(data: StartingNodeData, next: SubsequentNode) extends OneOutputNode + case class SourceNode(data: StartingNodeData, next: Option[SubsequentNode]) extends OneOutputNode sealed trait SubsequentNode extends Node - case class OneOutputSubsequentNode(data: OneOutputSubsequentNodeData, next: SubsequentNode) + case class OneOutputSubsequentNode(data: OneOutputSubsequentNodeData, next: Option[SubsequentNode]) extends OneOutputNode with SubsequentNode @@ -61,7 +61,11 @@ object node { case class SplitNode(data: Split, nextParts: List[SubsequentNode]) extends SubsequentNode - case class Case(expression: Expression, node: SubsequentNode) + case class Case(expression: Expression, node: Option[SubsequentNode]) + + object Case { + def apply(expression: Expression, node: SubsequentNode): Case = Case(expression, Some(node)) + } case class EndingNode(data: EndingNodeData) extends SubsequentNode diff --git a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala index 095ab65d2a5..429456bd564 100644 --- a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala +++ b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala @@ -16,6 +16,7 @@ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.{canonicalnode, CanonicalProcess} import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.{CanonicalNode, FlatNode} import pl.touk.nussknacker.engine.canonize.{MaybeArtificial, ProcessCanonizer} +import pl.touk.nussknacker.engine.canonize.MissingSinkHandler.DoNotAllowMissingSinkHandler import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.expression.Expression.Language import pl.touk.nussknacker.engine.graph.node @@ -48,7 +49,7 @@ class ProcessMarshallerSpec .customNode("b", "alamakota == 'true'", "someRef") .buildVariable("c", "fooVar", "f1" -> "expr1".spel, "f2" -> "expr2".spel) .enricher("d", "barVar", "dService", "p1" -> "expr3".spel) - .switch("f", "expr4".spel, "eVar", nestedGraph("e"), Case("e1".spel, GraphBuilder.emptySink("endE1", ""))) + .switch("f", "expr4".spel, "eVar", nestedGraph("e"), Case("e1".spel, Some(GraphBuilder.emptySink("endE1", "")))) val result = marshallAndUnmarshall(process) @@ -227,7 +228,7 @@ class ProcessMarshallerSpec inside( ProcessCanonizer.uncanonize( CanonicalProcess(MetaData("1", StreamMetaData()), nodes.toList, List.empty), - allowEndingScenarioWithoutSink = false + DoNotAllowMissingSinkHandler, ) ) { case Invalid(NonEmptyList(canonize.InvalidTailOfBranch(id), Nil)) => id shouldBe expectedBadNodeId diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/Interpreter.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/Interpreter.scala index c9a0270225c..ddc3b884b8d 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/Interpreter.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/Interpreter.scala @@ -76,16 +76,16 @@ private class InterpreterInternal[F[_]: Monad]( } node match { case Source(_, _, next) => - interpretNext(node, next, ctx) + interpretOptionalNext(node, next, ctx) case VariableBuilder(_, varName, Right(fields), next) => val variable = createOrUpdateVariable(ctx, varName, fields) - interpretNext(node, next, variable) + interpretOptionalNext(node, next, variable) case VariableBuilder(_, varName, Left(expression), next) => val valueWithModifiedContext = expressionEvaluator.evaluate[Any](expression, varName, node.id, ctx) - interpretNext(node, next, ctx.withVariable(varName, valueWithModifiedContext.value)) + interpretOptionalNext(node, next, ctx.withVariable(varName, valueWithModifiedContext.value)) case FragmentUsageStart(_, params, next) => val (newCtx, vars) = expressionEvaluator.evaluateParameters(params, ctx) - interpretNext( + interpretOptionalNext( node, next, newCtx.pushNewContext(vars.map { case (paramName, value) => (paramName.value, value) }) @@ -101,17 +101,17 @@ private class InterpreterInternal[F[_]: Monad]( parentContext.withVariable(varName, parsedFieldsMap) case None => parentContext } - interpretNext(node, next, newParentContext) + interpretOptionalNext(node, next, newParentContext) case Processor(_, ref, next, false) => invokeWrappedInInterpreterShape(ref, ctx).flatMap { // for Processor the result is null/BoxedUnit/Void etc. so we ignore it case Left(ValueWithContext(_, newCtx)) => - interpretNext(node, next, newCtx) + interpretOptionalNext(node, next, newCtx) case Right(exInfo) => Monad[F].pure(List(Right(exInfo))) } case Processor(_, _, next, true) => - interpretNext(node, next, ctx) + interpretOptionalNext(node, next, ctx) case EndingProcessor(id, ref, false) => listeners.foreach(_.endEncountered(id, ref.id, ctx, jobData.metaData)) invokeWrappedInInterpreterShape(ref, ctx).map { @@ -145,8 +145,9 @@ private class InterpreterInternal[F[_]: Monad]( ) case Enricher(_, ref, outName, next) => invokeWrappedInInterpreterShape(ref, ctx).flatMap { - case Left(ValueWithContext(out, newCtx)) => interpretNext(node, next, newCtx.withVariable(outName, out)) - case Right(exInfo) => Monad[F].pure(List(Right(exInfo))) + case Left(ValueWithContext(out, newCtx)) => + interpretOptionalNext(node, next, newCtx.withVariable(outName, out)) + case Right(exInfo) => Monad[F].pure(List(Right(exInfo))) } case Filter(_, expression, nextTrue, nextFalse, disabled) => val valueWithModifiedContext = @@ -169,7 +170,7 @@ private class InterpreterInternal[F[_]: Monad]( val valueWithModifiedContext = evaluateExpression[Boolean](casee.expression, accCtx, s"$expressionName-$i") if (valueWithModifiedContext.value) { - (valueWithModifiedContext.context, Some(casee.node)) + (valueWithModifiedContext.context, casee.next) } else { (valueWithModifiedContext.context, None) } @@ -189,7 +190,7 @@ private class InterpreterInternal[F[_]: Monad]( case BranchEnd(e) => Monad[F].pure(List(Left(InterpretationResult(e.joinReference, ctx)))) case CustomNode(_, _, next) => - interpretNext(node, next, ctx) + interpretOptionalNext(node, next, ctx) case EndingCustomNode(id, ref) => listeners.foreach(_.endEncountered(id, ref, ctx, jobData.metaData)) interpretationResult(node, EndReference(id), ctx) @@ -217,7 +218,13 @@ private class InterpreterInternal[F[_]: Monad]( case Some(next) => interpretNext(node, next, ctx) case None => - listeners.foreach(_.deadEndEncountered(node.id, ctx, jobData.metaData)) + // todo: perhaps we should refactor the way DeadEndingData is used, all nodes can be dead ends now + node match { + case Filter(_, _, _, _, _) | Switch(_, _, _, _) => + listeners.foreach(_.deadEndEncountered(node.id, ctx, jobData.metaData)) + case _ => + () + } interpretationResult(node, DeadEndReference(node.id), ctx) } } @@ -225,11 +232,11 @@ private class InterpreterInternal[F[_]: Monad]( private def interpretNext(node: Node, next: Next, ctx: Context): F[List[Result[InterpretationResult]]] = { onTransitionToNextNode(node, next, ctx) next match { - case NextNode(node) => interpret(node, ctx) - case pr @ PartRef(ref) => { + case NextNode(node) => + interpret(node, ctx) + case pr @ PartRef(ref) => listeners.foreach(_.nodeEntered(pr.id, ctx, jobData.metaData)) Monad[F].pure(List(Left(InterpretationResult(NextPartReference(ref), ctx)))) - } } } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/PartSubGraphCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/PartSubGraphCompiler.scala index 3b1dc757ed5..36e7fac46f2 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/PartSubGraphCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/PartSubGraphCompiler.scala @@ -6,7 +6,7 @@ import cats.data.Validated._ import cats.instances.list._ import cats.instances.option._ import pl.touk.nussknacker.engine.{compiledgraph, ScenarioCompilationDependencies} -import pl.touk.nussknacker.engine.api.{JobData, MetaData, NodeId} +import pl.touk.nussknacker.engine.api.NodeId import pl.touk.nussknacker.engine.api.context.{OutputVar, ProcessCompilationError, ValidationContext} import pl.touk.nussknacker.engine.api.context.ProcessCompilationError._ import pl.touk.nussknacker.engine.api.definition.Parameter @@ -52,7 +52,7 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { case splittednode.SplitNode(bareNode, nexts) => val compiledNexts = nexts.map(n => compile(n, ctx)).sequence compiledNexts.andThen(nx => - toCompilationResult(Valid(compiledgraph.node.SplitNode(bareNode.id, nx)), Map.empty) + toCompilationResult(Valid(compiledgraph.node.SplitNode(bareNode.id, nx.flatten)), Map.empty) ) case splittednode.FilterNode(f @ Filter(id, _, _, _), nextTrue, nextFalse) => @@ -67,8 +67,8 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { compiledgraph.node.Filter( id = id, expression = expr, - nextTrue = next, - nextFalse = nextFalse, + nextTrue = next.flatten, + nextFalse = nextFalse.flatten, isDisabled = f.isDisabled.contains(true) ) ) @@ -76,7 +76,7 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { case splittednode.SwitchNode(Switch(id, expression, varName, _), nexts, defaultNext) => val result = nodeCompiler.compileSwitch( Applicative[Option].product(varName, expression), - nexts.map(c => (c.node.id, c.expression)), + nexts.flatMap(c => c.node.map(next => (next.id, c.expression))), ctx ) val contextAfter = result.validationContext.getOrElse(ctx) @@ -88,14 +88,14 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { f3 = defaultNext.map(dn => compile(dn, contextAfter)).sequence ) { case (_, (expr, caseExpressions), cases, defaultNext) => val compiledCases = caseExpressions.zip(cases).map(k => compiledgraph.node.Case(k._1, k._2)) - compiledgraph.node.Switch(id, Applicative[Option].product(varName, expr), compiledCases, defaultNext) + compiledgraph.node.Switch(id, Applicative[Option].product(varName, expr), compiledCases, defaultNext.flatten) } case splittednode.EndingNode(data) => compileEndingNode(ctx, data) } } - private def handleSourceNode(nodeData: StartingNodeData, ctx: ValidationContext, next: splittednode.Next)( + private def handleSourceNode(nodeData: StartingNodeData, ctx: ValidationContext, next: Option[splittednode.Next])( implicit scenarioCompilationDependencies: ScenarioCompilationDependencies ): CompilationResult[node.Source] = { // just like in a custom node we can't add input context here because it contains output variable context (not input) @@ -168,7 +168,7 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { } } - private def compileSubsequent(ctx: ValidationContext, data: OneOutputSubsequentNodeData, next: Next)( + private def compileSubsequent(ctx: ValidationContext, data: OneOutputSubsequentNodeData, next: Option[Next])( implicit nodeId: NodeId, scenarioCompilationDependencies: ScenarioCompilationDependencies ): CompilationResult[Node] = { @@ -264,13 +264,26 @@ class PartSubGraphCompiler(nodeCompiler: NodeCompiler) { } } + private def compile(nextOpt: Option[splittednode.Next], ctx: ValidationContext)( + implicit scenarioCompilationDependencies: ScenarioCompilationDependencies + ): CompilationResult[Option[compiledgraph.node.Next]] = { + nextOpt match { + case Some(next) => compile(next, ctx) + case None => CompilationResult(Map.empty, Valid(None)) + } + } + private def compile(next: splittednode.Next, ctx: ValidationContext)( implicit scenarioCompilationDependencies: ScenarioCompilationDependencies - ): CompilationResult[compiledgraph.node.Next] = { + ): CompilationResult[Option[compiledgraph.node.Next]] = { next match { - case splittednode.NextNode(n) => compile(n, ctx).map(cn => compiledgraph.node.NextNode(cn)) + case splittednode.NextNode(n) => + compile(n, ctx) + .map(cn => compiledgraph.node.NextNode(cn)) + .map(Some(_)) case splittednode.PartRef(ref) => CompilationResult(Map(ref -> NodeTypingInfo(ctx, Map.empty, None)), Valid(compiledgraph.node.PartRef(ref))) + .map(Some(_)) } } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala index 0fb15a72380..6172d10243e 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala @@ -10,7 +10,7 @@ import pl.touk.nussknacker.engine.api.context._ import pl.touk.nussknacker.engine.api.context.ProcessCompilationError._ import pl.touk.nussknacker.engine.api.dict.DictRegistry import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.canonize.{MaybeArtificial, ProcessCanonizer} +import pl.touk.nussknacker.engine.canonize.ProcessCanonizer import pl.touk.nussknacker.engine.compile.FragmentValidator.validateUniqueFragmentOutputNames import pl.touk.nussknacker.engine.compile.nodecompilation.{LazyParameterCreationStrategy, NodeCompiler} import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult @@ -115,7 +115,7 @@ protected trait ProcessCompilerBase { ThreadUtils.withContextClassLoader(classLoader) { val compilationResultWithArtificial = ProcessCanonizer - .uncanonizeArtificial(process, nodeCompiler.allowEndingScenarioWithoutSink) + .uncanonizeArtificial(process, nodeCompiler.missingSinkHandler) .map(ProcessSplitter.split) .map(compile) compilationResultWithArtificial.extract @@ -191,22 +191,14 @@ protected trait ProcessCompilerBase { implicit scenarioCompilationDependencies: ScenarioCompilationDependencies ): CompilationResult[List[compiledgraph.part.SubsequentPart]] = { import CompilationResult._ - parts.map { - case SinkPart(node @ EndingNode(_: MaybeArtificial.ArtificialDeadEndSink)) => - CompilationResult.pure( - compiledgraph.part.SinkPart( - obj = ArtificialDeadEndSink, - node = node, - contextBefore = ValidationContext.empty, - validationContext = ValidationContext.empty - ) - ) - case p => + parts + .map(p => ctx .get(p.id) .map(compileSubsequentPart(p, _)) .getOrElse(CompilationResult(Invalid(NonEmptyList.of[ProcessCompilationError](MissingPart(p.id))))) - }.sequence + ) + .sequence } private def compileSubsequentPart(part: SubsequentPart, ctx: ValidationContext)( @@ -410,5 +402,3 @@ object ProcessValidator { } } - -object ArtificialDeadEndSink extends api.process.Sink diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala index 78babac8794..020f0bb0610 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/nodecompilation/NodeCompiler.scala @@ -15,6 +15,7 @@ import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process.Source import pl.touk.nussknacker.engine.api.typed.ReturningType import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} +import pl.touk.nussknacker.engine.canonize.MissingSinkHandler import pl.touk.nussknacker.engine.compile._ import pl.touk.nussknacker.engine.compile.nodecompilation.NodeCompiler.NodeCompilationResult import pl.touk.nussknacker.engine.compiledgraph.{CompiledParameter, TypedParameter} @@ -69,13 +70,14 @@ class NodeCompiler( nonServicesLazyParamStrategy: LazyParameterCreationStrategy, ) { - def allowEndingScenarioWithoutSink( + def missingSinkHandler( implicit scenarioCompilationDependencies: ScenarioCompilationDependencies - ): Boolean = { + ): MissingSinkHandler = { import scenarioCompilationDependencies._ lazy val allowEndingScenarioWithoutSink = definitions.allowEndingScenarioWithoutSink lazy val isFragment = metaData.typeSpecificData.isFragment - allowEndingScenarioWithoutSink && !isFragment + if (allowEndingScenarioWithoutSink && !isFragment) MissingSinkHandler.AllowMissingSinkHandler + else MissingSinkHandler.DoNotAllowMissingSinkHandler } def withLabelsDictTyper: NodeCompiler = { diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/CompiledNodesCollector.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/CompiledNodesCollector.scala index 6cfe3f5b4f8..743111338f9 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/CompiledNodesCollector.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/CompiledNodesCollector.scala @@ -11,15 +11,15 @@ object CompiledNodesCollector { case n: Processor => collectNodes(n.next) case n: Enricher => collectNodes(n.next) case n: Filter => - n.nextTrue.toList.flatMap(collectNodes) ::: n.nextFalse.toList.flatMap(collectNodes) + collectNodes(n.nextTrue) ::: collectNodes(n.nextFalse) case n: Switch => n.nexts.flatMap { case Case(_, ch) => collectNodes(ch) - } ::: n.defaultNext.toList.flatMap(collectNodes) + } ::: collectNodes(n.defaultNext) case n: CustomNode => collectNodes(n.next) case n: FragmentUsageStart => collectNodes(n.next) case n: FragmentUsageEnd => collectNodes(n.next) - case SplitNode(_, nextsWithParts) => nextsWithParts.flatMap(collectNodes) + case SplitNode(_, nextsWithParts) => nextsWithParts.flatMap(next => collectNodes(Some(next))) case _: Sink => List.empty case _: BranchEnd => List.empty case _: EndingCustomNode => List.empty @@ -29,10 +29,11 @@ object CompiledNodesCollector { node :: children } - private def collectNodes(next: Next): List[Node] = + private def collectNodes(next: Option[Next]): List[Node] = next match { - case NextNode(node) => collectAllNodes(node) - case _: PartRef => List.empty + case Some(NextNode(node)) => collectAllNodes(node) + case Some(_: PartRef) => List.empty + case None => List.empty } } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/node.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/node.scala index d1e1dfef964..0e54de50f0b 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/node.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compiledgraph/node.scala @@ -11,7 +11,7 @@ object node { def id: String } - case class Source(id: String, ref: Option[String], next: Next) extends Node + case class Source(id: String, ref: Option[String], next: Option[Next]) extends Node case class Sink(id: String, ref: String, isDisabled: Boolean) extends Node @@ -19,14 +19,18 @@ object node { override def id: String = definition.artificialNodeId } - case class VariableBuilder(id: String, varName: String, value: Either[CompiledExpression, List[Field]], next: Next) - extends Node + case class VariableBuilder( + id: String, + varName: String, + value: Either[CompiledExpression, List[Field]], + next: Option[Next] + ) extends Node - case class Processor(id: String, service: ServiceRef, next: Next, isDisabled: Boolean) extends Node + case class Processor(id: String, service: ServiceRef, next: Option[Next], isDisabled: Boolean) extends Node case class EndingProcessor(id: String, service: ServiceRef, isDisabled: Boolean) extends Node - case class Enricher(id: String, service: ServiceRef, output: String, next: Next) extends Node + case class Enricher(id: String, service: ServiceRef, output: String, next: Option[Next]) extends Node case class Filter( id: String, @@ -43,18 +47,18 @@ object node { defaultNext: Option[Next] ) extends Node - case class Case(expression: CompiledExpression, node: Next) + case class Case(expression: CompiledExpression, next: Option[Next]) - case class CustomNode(id: String, ref: String, next: Next) extends Node + case class CustomNode(id: String, ref: String, next: Option[Next]) extends Node case class EndingCustomNode(id: String, ref: String) extends Node case class FragmentOutput(id: String, fieldsWithExpression: Map[String, TypedExpression], isDisabled: Boolean) extends Node - case class FragmentUsageStart(id: String, params: List[CompiledParameter], next: Next) extends Node + case class FragmentUsageStart(id: String, params: List[CompiledParameter], next: Option[Next]) extends Node - case class FragmentUsageEnd(id: String, outputVarDefinition: Option[FragmentOutputVarDefinition], next: Next) + case class FragmentUsageEnd(id: String, outputVarDefinition: Option[FragmentOutputVarDefinition], next: Option[Next]) extends Node case class FragmentOutputVarDefinition(name: String, fields: List[Field]) diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala index 17ed878899e..3b5b392c0b6 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala @@ -15,7 +15,7 @@ object ProcessSplitter { } private def split(node: SourceNode): SourcePart = { - val nextWithParts = traverse(node.next) + val nextWithParts = traverse(node.id, node.next) SourcePart(splittednode.SourceNode(node.data, nextWithParts.next), nextWithParts.nextParts, nextWithParts.ends) } @@ -35,39 +35,49 @@ object ProcessSplitter { SinkPart(node) } + private def traverse(nodeId: String, nextNodeOpt: Option[SubsequentNode]): NextWithParts = { + nextNodeOpt match { + case Some(next) => + traverse(next) + case None => + NextWithParts.end(nodeId) + } + } + private def traverse(node: SubsequentNode): NextWithParts = node match { case FilterNode(data, nextTrue, nextFalse) => (nextTrue.map(traverse), nextFalse.map(traverse)) match { case (Some(nextTrueT), Some(nextFalseT)) => NextWithParts( - NextNode(splittednode.FilterNode(data, Some(nextTrueT.next), Some(nextFalseT.next))), + NextNode(splittednode.FilterNode(data, nextTrueT.next, nextFalseT.next)), nextTrueT.nextParts ::: nextFalseT.nextParts, nextTrueT.ends ::: nextFalseT.ends ) case (None, Some(nextFalseT)) => NextWithParts( - NextNode(splittednode.FilterNode(data, None, Some(nextFalseT.next))), + NextNode(splittednode.FilterNode(data, None, nextFalseT.next)), nextFalseT.nextParts, DeadEnd(data.id) :: nextFalseT.ends ) case (Some(nextTrueT), None) => NextWithParts( - NextNode(splittednode.FilterNode(data, Some(nextTrueT.next), None)), + NextNode(splittednode.FilterNode(data, nextTrueT.next, None)), nextTrueT.nextParts, DeadEnd(data.id) :: nextTrueT.ends ) - case (None, None) => throw new IllegalStateException("should not happen") + case (None, None) => + NextWithParts.end(data.id) } case SwitchNode(data, nexts, defaultNext) => val (nextsT, casesNextParts, casesEnds) = nexts.map { casee => - val nextWithParts = traverse(casee.node) + val nextWithParts = traverse(data.id, casee.node) (splittednode.Case(casee.expression, nextWithParts.next), nextWithParts.nextParts, nextWithParts.ends) }.unzip3 defaultNext.map(traverse) match { case Some(defaultNextT) => NextWithParts( - NextNode(splittednode.SwitchNode(data, nextsT, Some(defaultNextT.next))), + NextNode(splittednode.SwitchNode(data, nextsT, defaultNextT.next)), defaultNextT.nextParts ::: casesNextParts.flatten, defaultNextT.ends ::: casesEnds.flatten ) @@ -79,15 +89,20 @@ object ProcessSplitter { ) } case OneOutputSubsequentNode(custom: CustomNode, next) => - val part = split(custom, next) - NextWithParts(PartRef(part.id), List(part), List.empty) + next match { + case Some(next) => + val part = split(custom, next) + NextWithParts(PartRef(part.id), List(part), List.empty) + case None => + NextWithParts.end(custom.id) + } case split: SplitNode => val nextWithParts = split.nextParts.map(traverse) - val node = splittednode.SplitNode(split.data, nextWithParts.map(_.next)) + val node = splittednode.SplitNode(split.data, nextWithParts.flatMap(_.next)) NextWithParts(NextNode(node), nextWithParts.flatMap(_.nextParts), nextWithParts.flatMap(_.ends)) case OneOutputSubsequentNode(other, next) => - traverse(next).map { nextT => - NextNode(splittednode.OneOutputSubsequentNode(other, nextT)) + traverse(other.id, next).map { nextT => + Some(NextNode(splittednode.OneOutputSubsequentNode(other, nextT))) } case EndingNode(sink: Sink) => val part = split(sink) @@ -108,12 +123,20 @@ object ProcessSplitter { } - case class NextWithParts(next: splittednode.Next, nextParts: List[SubsequentPart], ends: List[End]) { + case class NextWithParts(next: Option[splittednode.Next], nextParts: List[SubsequentPart], ends: List[End]) { - def map(f: splittednode.Next => splittednode.Next): NextWithParts = { + def map(f: Option[splittednode.Next] => Option[splittednode.Next]): NextWithParts = { copy(next = f(next)) } } + object NextWithParts { + def apply(next: splittednode.Next, nextParts: List[SubsequentPart], ends: List[End]): NextWithParts = + NextWithParts(Some(next), nextParts, ends) + + def end(nodeId: String): NextWithParts = + NextWithParts(None, List.empty, List(NormalEnd(nodeId))) + } + } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/SplittedNodesCollector.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/SplittedNodesCollector.scala index c31ac9ebcdb..22a7eecc1d2 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/SplittedNodesCollector.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/SplittedNodesCollector.scala @@ -20,12 +20,12 @@ object SplittedNodesCollector { def collectNodes(node: SplittedNode[_ <: NodeData]): List[SplittedNode[_ <: NodeData]] = { val children = node match { case n: OneOutputNode[_] => - collectNodes(n.next) + n.next.toList.flatMap(collectNodes) case n: FilterNode => n.nextTrue.toList.flatMap(collectNodes) ::: n.nextFalse.toList.flatMap(collectNodes) case n: SwitchNode => n.nexts.flatMap { case Case(_, ch) => - collectNodes(ch) + ch.toList.flatMap(collectNodes) } ::: n.defaultNext.toList.flatMap(collectNodes) case SplitNode(_, nextsWithParts) => nextsWithParts.flatMap(collectNodes) diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/splittednode.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/splittednode.scala index a2f42c61798..356b4e201da 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/splittednode.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/splittedgraph/splittednode.scala @@ -11,14 +11,14 @@ object splittednode { } sealed trait OneOutputNode[+T <: NodeData] extends SplittedNode[T] { - def next: Next + def next: Option[Next] } - case class SourceNode[+T <: StartingNodeData](data: T, next: Next) extends OneOutputNode[T] + case class SourceNode[+T <: StartingNodeData](data: T, next: Option[Next]) extends OneOutputNode[T] sealed trait SubsequentNode[T <: NodeData] extends SplittedNode[T] - case class OneOutputSubsequentNode[T <: OneOutputSubsequentNodeData](data: T, next: Next) + case class OneOutputSubsequentNode[T <: OneOutputSubsequentNodeData](data: T, next: Option[Next]) extends OneOutputNode[T] with SubsequentNode[T] @@ -30,7 +30,7 @@ object splittednode { case class SwitchNode(data: Switch, nexts: List[Case], defaultNext: Option[Next] = None) extends SubsequentNode[Switch] - case class Case(expression: Expression, node: Next) + case class Case(expression: Expression, node: Option[Next]) case class EndingNode[T <: EndingNodeData](data: T) extends SubsequentNode[T] From dda9c79a12fcd8ddc7aec16a8532669299480de6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Mon, 5 May 2025 18:02:34 +0200 Subject: [PATCH 10/13] optional next node - changes in compiler --- .../test/utils/domain/ProcessTestData.scala | 4 +- .../flink/ResultCollectingListenerSpec.scala | 67 +++------ .../registrar/FlinkProcessRegistrar.scala | 1 - .../management/streaming/SampleProcess.scala | 2 +- .../engine/build/GraphBuilder.scala | 140 ++++++++++-------- .../engine/canonize/MissingSinkHandler.scala | 2 +- .../engine/canonize/ProcessCanonizer.scala | 2 +- .../touk/nussknacker/engine/graph/node.scala | 2 +- .../marshall/ProcessMarshallerSpec.scala | 2 +- .../engine/compile/ProcessCompiler.scala | 4 +- .../engine/split/ProcessSplitter.scala | 6 +- .../engine/compile/FragmentResolverSpec.scala | 58 ++++---- 12 files changed, 142 insertions(+), 148 deletions(-) diff --git a/designer/server/src/test/scala/pl/touk/nussknacker/test/utils/domain/ProcessTestData.scala b/designer/server/src/test/scala/pl/touk/nussknacker/test/utils/domain/ProcessTestData.scala index cb3e3b35937..e41b7f60843 100644 --- a/designer/server/src/test/scala/pl/touk/nussknacker/test/utils/domain/ProcessTestData.scala +++ b/designer/server/src/test/scala/pl/touk/nussknacker/test/utils/domain/ProcessTestData.scala @@ -207,7 +207,7 @@ object ProcessTestData { // TODO: merge with this below val sampleScenario: CanonicalProcess = { - def endWithMessage(idSuffix: String, message: String): SubsequentNode = { + def endWithMessage(idSuffix: String, message: String): Option[SubsequentNode] = { GraphBuilder .buildVariable("message" + idSuffix, "output", "message" -> s"'$message'".spel) .emptySink( @@ -493,7 +493,7 @@ object ProcessTestData { .to(endWithMessage) } - private def endWithMessage: SubsequentNode = { + private def endWithMessage: Option[SubsequentNode] = { val idSuffix = "suffix" val endMessage = "#test #{#input} #test \n#{\"abc\".toString + {1,2,3}.toString + \"abc\"}\n#test\n#{\"ab{}c\"}" diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala index 62f2fc7459d..fc65385d125 100644 --- a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala @@ -113,7 +113,7 @@ class ResultCollectingListenerSpec } test("union of two sources with additional variable in only one of the branches - without sinks") { - val scenario = removeSinks( + val scenario = ScenarioBuilder .streaming("sample-union") .sources( @@ -134,14 +134,13 @@ class ResultCollectingListenerSpec "bar" -> List("Output expression" -> "'bar source'".spel) ) ) - .emptySink("end", "dead-end") // sink must be created by the DSL, but is then removed + .endWithoutSink ) - ) withCollectingTestResults( scenario, testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-union", 8) + assertNumberOfSamplesThatFinishedInNode(testResults, "union", 8) transitionVariables(testResults, "start-foo", Some("union")) shouldBe Set( Map("input" -> 10), Map("input" -> 20), @@ -160,17 +159,7 @@ class ResultCollectingListenerSpec Map("input" -> 300, "customVariableInBarBranch" -> 150), Map("input" -> 400, "customVariableInBarBranch" -> 200), ) - transitionVariables(testResults, "union", Some("artificialDeadEndSink-after-union")) shouldBe Set( - Map("input" -> 10, "dataIsFrom" -> "foo source"), - Map("input" -> 20, "dataIsFrom" -> "foo source"), - Map("input" -> 30, "dataIsFrom" -> "foo source"), - Map("input" -> 40, "dataIsFrom" -> "foo source"), - Map("input" -> 100, "customVariableInBarBranch" -> 50, "dataIsFrom" -> "bar source"), - Map("input" -> 200, "customVariableInBarBranch" -> 100, "dataIsFrom" -> "bar source"), - Map("input" -> 300, "customVariableInBarBranch" -> 150, "dataIsFrom" -> "bar source"), - Map("input" -> 400, "customVariableInBarBranch" -> 200, "dataIsFrom" -> "bar source"), - ) - transitionVariables(testResults, "artificialDeadEndSink-after-union", None) shouldBe Set( + transitionVariables(testResults, "union", None) shouldBe Set( Map("input" -> 10, "dataIsFrom" -> "foo source"), Map("input" -> 20, "dataIsFrom" -> "foo source"), Map("input" -> 30, "dataIsFrom" -> "foo source"), @@ -292,7 +281,7 @@ class ResultCollectingListenerSpec } test("there is a split - fail to compile without sinks") { - val scenario = removeSinks( + val scenario = ScenarioBuilder .streaming("sample-split") .source("start-foo", "start1") @@ -300,12 +289,11 @@ class ResultCollectingListenerSpec "split", GraphBuilder .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) - .emptySink("end1", "dead-end"), + .endWithoutSink, GraphBuilder .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) - .emptySink("end2", "dead-end") + .endWithoutSink ) - ) catchExceptionMessage( withCollectingTestResults(scenario, _ => ()) @@ -314,7 +302,7 @@ class ResultCollectingListenerSpec } test("there is a split - without sinks") { - val scenario = removeSinks( + val scenario = ScenarioBuilder .streaming("sample-split") .source("start-foo", "start1") @@ -322,18 +310,17 @@ class ResultCollectingListenerSpec "split", GraphBuilder .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) - .emptySink("end1", "dead-end"), + .endWithoutSink, GraphBuilder .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) - .emptySink("end2", "dead-end") + .endWithoutSink ) - ) withCollectingTestResults( scenario, testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-bv1", 4) - assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-bv2", 4) + assertNumberOfSamplesThatFinishedInNode(testResults, "bv1", 4) + assertNumberOfSamplesThatFinishedInNode(testResults, "bv2", 4) transitionVariables(testResults, "start-foo", Some("split")) shouldBe Set( Map("input" -> 10), Map("input" -> 20), @@ -352,25 +339,13 @@ class ResultCollectingListenerSpec Map("input" -> 30), Map("input" -> 40), ) - transitionVariables(testResults, "bv1", Some("artificialDeadEndSink-after-bv1")) shouldBe Set( + transitionVariables(testResults, "bv1", None) shouldBe Set( Map("input" -> 10, "timesTwo" -> 20), Map("input" -> 20, "timesTwo" -> 40), Map("input" -> 30, "timesTwo" -> 60), Map("input" -> 40, "timesTwo" -> 80), ) - transitionVariables(testResults, "bv2", Some("artificialDeadEndSink-after-bv2")) shouldBe Set( - Map("input" -> 10, "timesFour" -> 40), - Map("input" -> 20, "timesFour" -> 80), - Map("input" -> 30, "timesFour" -> 120), - Map("input" -> 40, "timesFour" -> 160), - ) - transitionVariables(testResults, "artificialDeadEndSink-after-bv1", None) shouldBe Set( - Map("input" -> 10, "timesTwo" -> 20), - Map("input" -> 20, "timesTwo" -> 40), - Map("input" -> 30, "timesTwo" -> 60), - Map("input" -> 40, "timesTwo" -> 80), - ) - transitionVariables(testResults, "artificialDeadEndSink-after-bv2", None) shouldBe Set( + transitionVariables(testResults, "bv2", None) shouldBe Set( Map("input" -> 10, "timesFour" -> 40), Map("input" -> 20, "timesFour" -> 80), Map("input" -> 30, "timesFour" -> 120), @@ -447,7 +422,7 @@ class ResultCollectingListenerSpec } test("there is a fragment - without sinks") { - val scenarioWithFragment = removeSinks( + val scenarioWithFragment = ScenarioBuilder .streaming("sample-scenario-with-fragment") .source("source", "start1") @@ -456,9 +431,8 @@ class ResultCollectingListenerSpec "fragment1", List("fragment1_input" -> "#input".spel), Map("output" -> "fragmentResult"), - Map("output" -> GraphBuilder.emptySink("end", "dead-end")) + Map("output" -> None) ) - ) val fragment = ScenarioBuilder .fragment("fragment1", "fragment1_input" -> classOf[Int]) @@ -470,7 +444,7 @@ class ResultCollectingListenerSpec withCollectingTestResults( scenario, testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "artificialDeadEndSink-after-sub-fragmentEnd", 3) + assertNumberOfSamplesThatFinishedInNode(testResults, "sub-fragmentEnd", 3) assertNumberOfSamplesThatFinishedInNode(testResults, "sub-filter", 1) transitionVariables(testResults, "source", Some("sub")) shouldBe Set( Map("input" -> 10), @@ -502,17 +476,12 @@ class ResultCollectingListenerSpec transitionVariables( testResults, "sub-fragmentEnd", - Some("artificialDeadEndSink-after-sub-fragmentEnd") + None ) shouldBe Set( Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), ) - transitionVariables(testResults, "artificialDeadEndSink-after-sub-fragmentEnd", None) shouldBe Set( - Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), - Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), - Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), - ) }, allowEndingScenarioWithoutSink = true, ) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala index 471cc9ba5f5..89368e5e4bd 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/registrar/FlinkProcessRegistrar.scala @@ -21,7 +21,6 @@ import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters import pl.touk.nussknacker.engine.flink.api.compat.ExplicitUidInOperatorsSupport import pl.touk.nussknacker.engine.flink.api.process._ import pl.touk.nussknacker.engine.flink.api.typeinformation.TypeInformationDetection -import pl.touk.nussknacker.engine.flink.util.sink.EmptySink import pl.touk.nussknacker.engine.graph.node.{BranchEndDefinition, NodeData} import pl.touk.nussknacker.engine.node.NodeComponentInfoExtractor.fromScenarioNode import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkCompatibilityProvider, FlinkJobConfig} diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/SampleProcess.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/SampleProcess.scala index 5b8a9808c7d..3cd15b344cd 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/SampleProcess.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/SampleProcess.scala @@ -26,7 +26,7 @@ object SampleProcess { .emptySink("end", "kafka-string", "Topic" -> s"'output-$name'".spel, "Value" -> "#input".spel) } - private def endWithMessage(idSuffix: String, message: String): SubsequentNode = { + private def endWithMessage(idSuffix: String, message: String): Option[SubsequentNode] = { GraphBuilder .buildVariable("message" + idSuffix, "output", "message" -> s"'$message'".spel) .emptySink("end" + idSuffix, "monitor") diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala index 2c38b743428..08242befbbc 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/build/GraphBuilder.scala @@ -2,8 +2,7 @@ package pl.touk.nussknacker.engine.build import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.graph.{node, EdgeType} -import pl.touk.nussknacker.engine.graph.evaluatedparam.{Parameter => NodeParameter} -import pl.touk.nussknacker.engine.graph.evaluatedparam.BranchParameters +import pl.touk.nussknacker.engine.graph.evaluatedparam.{BranchParameters, Parameter => NodeParameter} import pl.touk.nussknacker.engine.graph.expression._ import pl.touk.nussknacker.engine.graph.fragment.FragmentRef import pl.touk.nussknacker.engine.graph.node._ @@ -20,36 +19,38 @@ trait GraphBuilder[R] { def build(inner: GraphBuilder.Creator[R]): GraphBuilder[R] def source(id: String, typ: String, params: (String, Expression)*): GraphBuilder[SourceNode] = - new SimpleGraphBuilder(node => SourceNode(Source(id, SourceRef(typ, toNodeParameters(params))), Some(node))) + new SimpleGraphBuilder(node => SourceNode(Source(id, SourceRef(typ, toNodeParameters(params))), node)) def buildVariable(id: String, varName: String, fields: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode(VariableBuilder(id, varName, fields.map(f => Field(f._1, f._2)).toList), Some(node)) + Some(OneOutputSubsequentNode(VariableBuilder(id, varName, fields.map(f => Field(f._1, f._2)).toList), node)) ) ) def buildSimpleVariable(id: String, varName: String, value: Expression): GraphBuilder[R] = - build(node => creator(OneOutputSubsequentNode(Variable(id, varName, value), Some(node)))) + build(node => creator(Some(OneOutputSubsequentNode(Variable(id, varName, value), node)))) def enricher(id: String, output: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode(Enricher(id, ServiceRef(svcId, toNodeParameters(params)), output), Some(node)) + Some(OneOutputSubsequentNode(Enricher(id, ServiceRef(svcId, toNodeParameters(params)), output), node)) ) ) def processor(id: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = build(node => - creator(OneOutputSubsequentNode(Processor(id, ServiceRef(svcId, toNodeParameters(params))), Some(node))) + creator(Some(OneOutputSubsequentNode(Processor(id, ServiceRef(svcId, toNodeParameters(params))), node))) ) def disabledProcessor(id: String, svcId: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode( - Processor(id, ServiceRef(svcId, toNodeParameters(params)), isDisabled = Some(true)), - Some(node) + Some( + OneOutputSubsequentNode( + Processor(id, ServiceRef(svcId, toNodeParameters(params)), isDisabled = Some(true)), + node + ) ) ) ) @@ -63,16 +64,18 @@ trait GraphBuilder[R] { ): GraphBuilder[R] = build(node => creator( - FragmentNode( - FragmentInput( - id, - FragmentRef( - fragmentId, - toNodeParameters(params), - Map(fragmentOutputDefinitionName -> outputParamName) - ) - ), - Map(fragmentOutputDefinitionName -> node) + Some( + FragmentNode( + FragmentInput( + id, + FragmentRef( + fragmentId, + toNodeParameters(params), + Map(fragmentOutputDefinitionName -> outputParamName) + ) + ), + Map(fragmentOutputDefinitionName -> node), + ) ) ) ) @@ -82,20 +85,24 @@ trait GraphBuilder[R] { fragmentId: String, params: List[(String, Expression)], outputParameters: Map[String, String], - outputs: Map[String, SubsequentNode] + outputs: Map[String, Option[SubsequentNode]] ): R = creator( - FragmentNode( - FragmentInput(id, FragmentRef(fragmentId, toNodeParameters(params), outputParameters)), - outputs + Some( + FragmentNode( + FragmentInput(id, FragmentRef(fragmentId, toNodeParameters(params), outputParameters)), + outputs + ) ) ) def fragmentEnd(id: String, fragmentId: String, params: (String, Expression)*): R = creator( - FragmentNode( - FragmentInput(id, FragmentRef(fragmentId, toNodeParameters(params), Map.empty)), - Map() + Some( + FragmentNode( + FragmentInput(id, FragmentRef(fragmentId, toNodeParameters(params), Map.empty)), + Map() + ) ) ) @@ -106,7 +113,7 @@ trait GraphBuilder[R] { id = id, parameters = params.map(kv => FragmentParameter(ParameterName(kv._1), FragmentClazzRef(kv._2.getName))).toList ), - Some(node) + node ) ) @@ -117,12 +124,12 @@ trait GraphBuilder[R] { id = id, parameters = params.toList ), - Some(node) + node ) ) def fragmentOutput(id: String, outputName: String, params: (String, Expression)*): R = - creator(EndingNode(FragmentOutputDefinition(id, outputName, params.map(kv => Field(kv._1, kv._2)).toList))) + creator(Some(EndingNode(FragmentOutputDefinition(id, outputName, params.map(kv => Field(kv._1, kv._2)).toList)))) def filter( id: String, @@ -132,67 +139,78 @@ trait GraphBuilder[R] { ): GraphBuilder[R] = build(node => creator( - FilterNode( - Filter(id, expression, disabled), - Some(node).filter(_ => edgeType == EdgeType.FilterTrue), - Some(node).filter(_ => edgeType == EdgeType.FilterFalse) + Some( + FilterNode( + Filter(id, expression, disabled), + node.filter(_ => edgeType == EdgeType.FilterTrue), + node.filter(_ => edgeType == EdgeType.FilterFalse) + ) ) ) ) - def filter(id: String, expression: Expression, nextFalse: SubsequentNode): GraphBuilder[R] = - build(node => creator(FilterNode(Filter(id, expression), nextTrue = Some(node), nextFalse = Some(nextFalse)))) + def filter(id: String, expression: Expression, nextFalse: Option[SubsequentNode]): GraphBuilder[R] = + build(node => creator(Some(FilterNode(Filter(id, expression), nextTrue = node, nextFalse = nextFalse)))) + + def endWithoutSink: R = + creator(None) def emptySink(id: String, typ: String, params: (String, Expression)*): R = - creator(EndingNode(Sink(id, SinkRef(typ, toNodeParameters(params))))) + creator(Some(EndingNode(Sink(id, SinkRef(typ, toNodeParameters(params)))))) def disabledSink(id: String, typ: String): R = - creator(EndingNode(Sink(id, SinkRef(typ, List()), isDisabled = Some(true)))) + creator(Some(EndingNode(Sink(id, SinkRef(typ, List()), isDisabled = Some(true))))) def processorEnd(id: String, svcId: String, params: (String, Expression)*): R = - creator(EndingNode(Processor(id, ServiceRef(svcId, toNodeParameters(params))))) + creator(Some(EndingNode(Processor(id, ServiceRef(svcId, toNodeParameters(params)))))) def disabledProcessorEnd(id: String, svcId: String, params: (String, Expression)*): R = creator( - EndingNode(Processor(id, ServiceRef(svcId, toNodeParameters(params)), isDisabled = Some(true))) + Some(EndingNode(Processor(id, ServiceRef(svcId, toNodeParameters(params)), isDisabled = Some(true)))) ) def branchEnd(branchId: String, joinId: String): R = - creator(BranchEnd(node.BranchEndData(BranchEndDefinition(branchId, joinId)))) + creator(Some(BranchEnd(node.BranchEndData(BranchEndDefinition(branchId, joinId))))) def switch(id: String, nexts: Case*): R = - creator(SwitchNode(Switch(id), nexts.toList, None)) + creator(Some(SwitchNode(Switch(id), nexts.toList, None))) def switch(id: String, expression: Expression, exprVal: String, nexts: Case*): R = - creator(SwitchNode(Switch(id, Some(expression), Some(exprVal)), nexts.toList, None)) + creator(Some(SwitchNode(Switch(id, Some(expression), Some(exprVal)), nexts.toList, None))) - def switch(id: String, expression: Expression, exprVal: String, defaultNext: SubsequentNode, nexts: Case*): R = - creator(SwitchNode(Switch(id, Some(expression), Some(exprVal)), nexts.toList, Some(defaultNext))) + def switch( + id: String, + expression: Expression, + exprVal: String, + defaultNext: Option[SubsequentNode], + nexts: Case* + ): R = + creator(Some(SwitchNode(Switch(id, Some(expression), Some(exprVal)), nexts.toList, defaultNext))) def customNode(id: String, outputVar: String, customNodeRef: String, params: (String, Expression)*): GraphBuilder[R] = build(node => creator( - OneOutputSubsequentNode( - CustomNode(id, Some(outputVar), customNodeRef, toNodeParameters(params)), - Some(node) + Some( + OneOutputSubsequentNode( + CustomNode(id, Some(outputVar), customNodeRef, toNodeParameters(params)), + node + ) ) ) ) // outputVar must be provided always when parameter with @OutputVariableName annotation is used - look into comment in @OutputVariableName def endingCustomNode(id: String, outputVar: Option[String], customNodeRef: String, params: (String, Expression)*): R = - creator(EndingNode(CustomNode(id, outputVar, customNodeRef, toNodeParameters(params)))) + creator(Some(EndingNode(CustomNode(id, outputVar, customNodeRef, toNodeParameters(params))))) def customNodeNoOutput(id: String, customNodeRef: String, params: (String, Expression)*): GraphBuilder[R] = build(node => - creator( - OneOutputSubsequentNode(CustomNode(id, None, customNodeRef, toNodeParameters(params)), Some(node)) - ) + creator(Some(OneOutputSubsequentNode(CustomNode(id, None, customNodeRef, toNodeParameters(params)), node))) ) - def split(id: String, nexts: SubsequentNode*): R = creator(SplitNode(Split(id), nexts.toList)) + def split(id: String, nexts: Option[SubsequentNode]*): R = creator(Some(SplitNode(Split(id), nexts.toList.flatten))) - def to(node: SubsequentNode): R = + def to(node: Option[SubsequentNode]): R = creator(node) def join( @@ -205,9 +223,7 @@ trait GraphBuilder[R] { val branchParameters = branchParams.map { case (branchId, bParams) => BranchParameters(branchId, toNodeParameters(bParams)) } - new SimpleGraphBuilder(n => - SourceNode(node.Join(id, output, typ, toNodeParameters(params), branchParameters), Some(n)) - ) + new SimpleGraphBuilder(n => SourceNode(node.Join(id, output, typ, toNodeParameters(params), branchParameters), n)) } def decisionTable( @@ -230,16 +246,16 @@ trait GraphBuilder[R] { } -private[build] class SimpleGraphBuilder[R <: Node](val creator: GraphBuilder.Creator[R]) extends GraphBuilder[R] { +private[build] class SimpleGraphBuilder[R](val creator: GraphBuilder.Creator[R]) extends GraphBuilder[R] { override def build(inner: GraphBuilder.Creator[R]) = new SimpleGraphBuilder(inner) } -object GraphBuilder extends GraphBuilder[SubsequentNode] { +object GraphBuilder extends GraphBuilder[Option[SubsequentNode]] { - type Creator[R] = SubsequentNode => R + type Creator[R] = Option[SubsequentNode] => R - override def creator: Creator[SubsequentNode] = identity[SubsequentNode] + override def creator: Creator[Option[SubsequentNode]] = identity[Option[SubsequentNode]] - override def build(inner: Creator[SubsequentNode]) = new SimpleGraphBuilder[SubsequentNode](inner) + override def build(inner: Creator[Option[SubsequentNode]]) = new SimpleGraphBuilder[Option[SubsequentNode]](inner) } diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala index 2d0fca7ddc4..f97e315b85a 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/MissingSinkHandler.scala @@ -4,7 +4,7 @@ import pl.touk.nussknacker.engine.graph.node // This trait and its 2 implementation encapsulate the logic of handling scenarios that do not end with sink // - older behavior is that error is returned for missing sinks -// - new behavior, which we intend to become the default and only one in the future, is allowing missing sinks +// - new behavior, which we intend to become the default and only one in the future, allows ending scenario without sink sealed trait MissingSinkHandler { def handleMissingSink(previousNodeId: String): MaybeArtificial[Option[node.SubsequentNode]] } diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala index 214c180efbe..15b36e93da6 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/canonize/ProcessCanonizer.scala @@ -136,7 +136,7 @@ object NodeCanonizer { case node.SplitNode(bare, nexts) => canonicalnode.SplitNode(bare, nexts.map(canonize)) :: Nil case node.FragmentNode(input, nexts) => - canonicalnode.Fragment(input, nexts.mapValuesNow(canonize)) :: Nil + canonicalnode.Fragment(input, nexts.mapValuesNow(_.map(canonize).getOrElse(List.empty))) :: Nil case BranchEnd(e: BranchEndData) => canonicalnode.FlatNode(e) :: Nil } diff --git a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala index 7f80c47918e..c0cd04a7ec1 100644 --- a/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala +++ b/scenario-api/src/main/scala/pl/touk/nussknacker/engine/graph/node.scala @@ -53,7 +53,7 @@ object node { extends SubsequentNode // this should never occur in process to be run (unresolved) - case class FragmentNode(data: FragmentInput, nexts: Map[String, SubsequentNode]) extends SubsequentNode + case class FragmentNode(data: FragmentInput, nexts: Map[String, Option[SubsequentNode]]) extends SubsequentNode // defaultNext is deprecated, will be removed in future versions case class SwitchNode(data: Switch, nexts: List[Case], defaultNext: Option[SubsequentNode] = None) diff --git a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala index 429456bd564..2343dce26bb 100644 --- a/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala +++ b/scenario-api/src/test/scala/pl/touk/nussknacker/engine/marshall/ProcessMarshallerSpec.scala @@ -49,7 +49,7 @@ class ProcessMarshallerSpec .customNode("b", "alamakota == 'true'", "someRef") .buildVariable("c", "fooVar", "f1" -> "expr1".spel, "f2" -> "expr2".spel) .enricher("d", "barVar", "dService", "p1" -> "expr3".spel) - .switch("f", "expr4".spel, "eVar", nestedGraph("e"), Case("e1".spel, Some(GraphBuilder.emptySink("endE1", "")))) + .switch("f", "expr4".spel, "eVar", nestedGraph("e"), Case("e1".spel, GraphBuilder.emptySink("endE1", ""))) val result = marshallAndUnmarshall(process) diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala index 6172d10243e..c59053afd29 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/compile/ProcessCompiler.scala @@ -241,7 +241,7 @@ protected trait ProcessCompilerBase { splittednode.SourceNode(sourceData, part.node.next), ctx, nextParts, - part.ends.map(e => TypedEnd(e, typesForParts(e.nodeId))) + part.ends.map(e => TypedEnd(e, typesForParts.getOrElse(e.nodeId, ValidationContext.empty))) ) } } @@ -324,7 +324,7 @@ protected trait ProcessCompilerBase { ctx.left.getOrElse(ValidationContext.empty), nextCtx, nextPartsCompiled, - part.ends.map(e => TypedEnd(e, typesForParts(e.nodeId))) + part.ends.map(e => TypedEnd(e, typesForParts.getOrElse(e.nodeId, ValidationContext.empty))) ) } .distinctErrors diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala index 3b5b392c0b6..dc9f2681cc6 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/split/ProcessSplitter.scala @@ -67,7 +67,11 @@ object ProcessSplitter { DeadEnd(data.id) :: nextTrueT.ends ) case (None, None) => - NextWithParts.end(data.id) + NextWithParts( + NextNode(splittednode.FilterNode(data, None, None)), + List.empty, + DeadEnd(data.id) :: Nil, + ) } case SwitchNode(data, nexts, defaultNext) => val (nextsT, casesNextParts, casesEnds) = nexts.map { casee => diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/FragmentResolverSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/FragmentResolverSpec.scala index 800313c8c5c..8615a11774c 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/FragmentResolverSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/compile/FragmentResolverSpec.scala @@ -399,16 +399,18 @@ class FragmentResolverSpec extends AnyFunSuite with Matchers with Inside { ): GraphBuilder[R] = build(node => builder.creator( - FragmentNode( - FragmentInput( - id, - FragmentRef( - fragmentId, - params.map { case (name, value) => NodeParameter(ParameterName(name), value) }.toList + Some( + FragmentNode( + FragmentInput( + id, + FragmentRef( + fragmentId, + params.map { case (name, value) => NodeParameter(ParameterName(name), value) }.toList + ), + isDisabled = Some(true) ), - isDisabled = Some(true) - ), - Map(output -> node) + Map(output -> node) + ) ) ) ) @@ -417,31 +419,35 @@ class FragmentResolverSpec extends AnyFunSuite with Matchers with Inside { id: String, fragmentId: String, params: List[(String, Expression)], - outputs: Map[String, SubsequentNode] + outputs: Map[String, Option[SubsequentNode]] ): R = creator( - FragmentNode( - FragmentInput( - id, - FragmentRef(fragmentId, params.map { case (name, value) => NodeParameter(ParameterName(name), value) }), - isDisabled = Some(true) - ), - outputs + Some( + FragmentNode( + FragmentInput( + id, + FragmentRef(fragmentId, params.map { case (name, value) => NodeParameter(ParameterName(name), value) }), + isDisabled = Some(true) + ), + outputs + ) ) ) def fragmentDisabledEnd(id: String, fragmentId: String, params: (String, Expression)*): R = creator( - FragmentNode( - FragmentInput( - id, - FragmentRef( - fragmentId, - params.map { case (name, value) => NodeParameter(ParameterName(name), value) }.toList + Some( + FragmentNode( + FragmentInput( + id, + FragmentRef( + fragmentId, + params.map { case (name, value) => NodeParameter(ParameterName(name), value) }.toList + ), + isDisabled = Some(true) ), - isDisabled = Some(true) - ), - Map() + Map() + ) ) ) From 0fa8ecf288f2f81d4f7eb8f6bbd81403566af65d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Tue, 6 May 2025 12:07:45 +0200 Subject: [PATCH 11/13] refactor tests --- .../flink/FlinkMiniClusterTestRunner.scala | 94 ++++++ .../flink/ResultCollectingListenerSpec.scala | 315 +----------------- .../flink/ScenarioWithoutSinksSpec.scala | 255 ++++++++++++++ 3 files changed, 355 insertions(+), 309 deletions(-) create mode 100644 engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/FlinkMiniClusterTestRunner.scala create mode 100644 engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ScenarioWithoutSinksSpec.scala diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/FlinkMiniClusterTestRunner.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/FlinkMiniClusterTestRunner.scala new file mode 100644 index 00000000000..6d5fc1c7247 --- /dev/null +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/FlinkMiniClusterTestRunner.scala @@ -0,0 +1,94 @@ +package pl.touk.nussknacker.engine.flink + +import com.typesafe.config.ConfigFactory +import pl.touk.nussknacker.engine.api.component.ComponentDefinition +import pl.touk.nussknacker.engine.api.process.SourceFactory +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.flink.test.FlinkSpec +import pl.touk.nussknacker.engine.flink.test.ScalatestMiniClusterJobStatusCheckingOps.miniClusterWithServicesToOps +import pl.touk.nussknacker.engine.flink.util.source.EmitWatermarkAfterEachElementCollectionSource +import pl.touk.nussknacker.engine.flink.util.transformer.FlinkBaseComponentProvider +import pl.touk.nussknacker.engine.flink.util.transformer.aggregate.AggregateWindowsConfig +import pl.touk.nussknacker.engine.process.helpers.ConfigCreatorWithCollectingListener +import pl.touk.nussknacker.engine.process.runner.FlinkScenarioUnitTestJob +import pl.touk.nussknacker.engine.testing.LocalModelData +import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListener, ResultsCollectingListenerHolder} +import pl.touk.nussknacker.engine.testmode.TestProcess.{NodeTransition, TestResults} +import pl.touk.nussknacker.engine.util.config.DocsConfig +import pl.touk.nussknacker.test.ProcessUtils.convertToAnyShouldWrapper + +import java.time.{Duration, Instant} +import scala.jdk.CollectionConverters._ +import scala.util.Try + +trait FlinkMiniClusterTestRunner { _: FlinkSpec => + + protected def sourcesWithMockedData: Map[String, List[Int]] + + protected def withCollectingTestResults( + canonicalProcess: CanonicalProcess, + assertions: TestResults[Any] => Unit, + allowEndingScenarioWithoutSink: Boolean = false, + ): Unit = { + ResultsCollectingListenerHolder.withListener { collectingListener => + val model = modelData(collectingListener, AggregateWindowsConfig.Default, allowEndingScenarioWithoutSink) + flinkMiniCluster.withDetachedStreamExecutionEnvironment { env => + val executionResult = new FlinkScenarioUnitTestJob(model).run(canonicalProcess, env) + flinkMiniCluster.waitForJobIsFinished(executionResult.getJobID) + assertions(collectingListener.results) + } + } + } + + private def modelData( + collectingListener: => ResultsCollectingListener[Any], + aggregateWindowsConfig: AggregateWindowsConfig, + allowEndingScenarioWithoutSink: Boolean, + ): LocalModelData = { + def sourceComponent(data: List[Int]) = SourceFactory.noParamUnboundedStreamFactory[Int]( + EmitWatermarkAfterEachElementCollectionSource + .create[Int](data, _ => Instant.now.toEpochMilli, Duration.ofHours(1)) + ) + val config = + if (allowEndingScenarioWithoutSink) { + ConfigFactory.parseString("""allowEndingScenarioWithoutSink: true""") + } else { + ConfigFactory.empty() + } + LocalModelData( + config, + sourcesWithMockedData.toList.map { case (name, data) => + ComponentDefinition(name, sourceComponent(data)) + } ::: + FlinkBaseUnboundedComponentProvider.create( + DocsConfig.Default, + aggregateWindowsConfig + ) ::: FlinkBaseComponentProvider.Components, + configCreator = new ConfigCreatorWithCollectingListener(collectingListener), + ) + } + + protected def transitionVariables( + testResults: TestResults[Any], + fromNodeId: String, + toNodeId: Option[String] + ): Set[Map[String, Any]] = + testResults + .nodeTransitionResults(NodeTransition(fromNodeId, toNodeId)) + .map(_.variables) + .toSet[Map[String, Any]] + .map(_.map { case (key, value) => (key, scalaMap(value)) }) + + private def scalaMap(value: Any): Any = { + value match { + case hashMap: java.util.HashMap[_, _] => hashMap.asScala.toMap + case other => other + } + } + + protected def assertNumberOfSamplesThatFinishedInNode(testResults: TestResults[Any], sinkId: String, expected: Int) = + testResults.nodeTransitionResults.get(NodeTransition(sinkId, None)).map(_.length) shouldBe Some(expected) + + protected def catchExceptionMessage(f: => Any): String = Try(f).failed.get.getMessage + +} diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala index fc65385d125..236c8a67513 100644 --- a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ResultCollectingListenerSpec.scala @@ -1,46 +1,29 @@ package pl.touk.nussknacker.engine.flink -import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.LazyLogging import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers -import pl.touk.nussknacker.engine.api.component.ComponentDefinition -import pl.touk.nussknacker.engine.api.process._ import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode._ import pl.touk.nussknacker.engine.compile.FragmentResolver import pl.touk.nussknacker.engine.flink.test.FlinkSpec -import pl.touk.nussknacker.engine.flink.test.ScalatestMiniClusterJobStatusCheckingOps.miniClusterWithServicesToOps -import pl.touk.nussknacker.engine.flink.util.source.EmitWatermarkAfterEachElementCollectionSource -import pl.touk.nussknacker.engine.flink.util.transformer.FlinkBaseComponentProvider -import pl.touk.nussknacker.engine.flink.util.transformer.aggregate.AggregateWindowsConfig -import pl.touk.nussknacker.engine.graph.node -import pl.touk.nussknacker.engine.process.helpers.ConfigCreatorWithCollectingListener -import pl.touk.nussknacker.engine.process.runner.FlinkScenarioUnitTestJob -import pl.touk.nussknacker.engine.testing.LocalModelData -import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListener, ResultsCollectingListenerHolder} -import pl.touk.nussknacker.engine.testmode.TestProcess.{NodeTransition, TestResults} -import pl.touk.nussknacker.engine.util.config.DocsConfig import pl.touk.nussknacker.test.VeryPatientScalaFutures -import java.time.{Duration, Instant} -import scala.jdk.CollectionConverters._ -import scala.util.Try - class ResultCollectingListenerSpec extends AnyFunSuite with BeforeAndAfterAll with Matchers with LazyLogging with VeryPatientScalaFutures - with FlinkSpec { + with FlinkSpec + with FlinkMiniClusterTestRunner { import pl.touk.nussknacker.engine.spel.SpelExtension._ - private val data1 = List(10, 20, 30, 40) - private val data2 = List(100, 200, 300, 400) + override protected def sourcesWithMockedData: Map[String, List[Int]] = Map( + "start1" -> List(10, 20, 30, 40), + "start2" -> List(100, 200, 300, 400), + ) test("union of two sources with additional variable in only one of the branches") { val scenario = ScenarioBuilder @@ -112,68 +95,6 @@ class ResultCollectingListenerSpec ) } - test("union of two sources with additional variable in only one of the branches - without sinks") { - val scenario = - ScenarioBuilder - .streaming("sample-union") - .sources( - GraphBuilder - .source("start-foo", "start1") - .branchEnd("foo", "union"), - GraphBuilder - .source("start-bar", "start2") - .buildSimpleVariable("bv1", "customVariableInBarBranch", "#input/2".spel) - .branchEnd("bar", "union"), - GraphBuilder - .join( - "union", - "union", - Some("dataIsFrom"), - List( - "foo" -> List("Output expression" -> "'foo source'".spel), - "bar" -> List("Output expression" -> "'bar source'".spel) - ) - ) - .endWithoutSink - ) - - withCollectingTestResults( - scenario, - testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "union", 8) - transitionVariables(testResults, "start-foo", Some("union")) shouldBe Set( - Map("input" -> 10), - Map("input" -> 20), - Map("input" -> 30), - Map("input" -> 40), - ) - transitionVariables(testResults, "start-bar", Some("bv1")) shouldBe Set( - Map("input" -> 100), - Map("input" -> 200), - Map("input" -> 300), - Map("input" -> 400), - ) - transitionVariables(testResults, "bv1", Some("union")) shouldBe Set( - Map("input" -> 100, "customVariableInBarBranch" -> 50), - Map("input" -> 200, "customVariableInBarBranch" -> 100), - Map("input" -> 300, "customVariableInBarBranch" -> 150), - Map("input" -> 400, "customVariableInBarBranch" -> 200), - ) - transitionVariables(testResults, "union", None) shouldBe Set( - Map("input" -> 10, "dataIsFrom" -> "foo source"), - Map("input" -> 20, "dataIsFrom" -> "foo source"), - Map("input" -> 30, "dataIsFrom" -> "foo source"), - Map("input" -> 40, "dataIsFrom" -> "foo source"), - Map("input" -> 100, "customVariableInBarBranch" -> 50, "dataIsFrom" -> "bar source"), - Map("input" -> 200, "customVariableInBarBranch" -> 100, "dataIsFrom" -> "bar source"), - Map("input" -> 300, "customVariableInBarBranch" -> 150, "dataIsFrom" -> "bar source"), - Map("input" -> 400, "customVariableInBarBranch" -> 200, "dataIsFrom" -> "bar source"), - ) - }, - allowEndingScenarioWithoutSink = true, - ) - } - test("there is for-each node") { val scenario = ScenarioBuilder .streaming("sample-for-each") @@ -280,82 +201,6 @@ class ResultCollectingListenerSpec ) } - test("there is a split - fail to compile without sinks") { - val scenario = - ScenarioBuilder - .streaming("sample-split") - .source("start-foo", "start1") - .split( - "split", - GraphBuilder - .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) - .endWithoutSink, - GraphBuilder - .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) - .endWithoutSink - ) - - catchExceptionMessage( - withCollectingTestResults(scenario, _ => ()) - ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv2, bv1))" - - } - - test("there is a split - without sinks") { - val scenario = - ScenarioBuilder - .streaming("sample-split") - .source("start-foo", "start1") - .split( - "split", - GraphBuilder - .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) - .endWithoutSink, - GraphBuilder - .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) - .endWithoutSink - ) - - withCollectingTestResults( - scenario, - testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "bv1", 4) - assertNumberOfSamplesThatFinishedInNode(testResults, "bv2", 4) - transitionVariables(testResults, "start-foo", Some("split")) shouldBe Set( - Map("input" -> 10), - Map("input" -> 20), - Map("input" -> 30), - Map("input" -> 40), - ) - transitionVariables(testResults, "split", Some("bv1")) shouldBe Set( - Map("input" -> 10), - Map("input" -> 20), - Map("input" -> 30), - Map("input" -> 40), - ) - transitionVariables(testResults, "split", Some("bv2")) shouldBe Set( - Map("input" -> 10), - Map("input" -> 20), - Map("input" -> 30), - Map("input" -> 40), - ) - transitionVariables(testResults, "bv1", None) shouldBe Set( - Map("input" -> 10, "timesTwo" -> 20), - Map("input" -> 20, "timesTwo" -> 40), - Map("input" -> 30, "timesTwo" -> 60), - Map("input" -> 40, "timesTwo" -> 80), - ) - transitionVariables(testResults, "bv2", None) shouldBe Set( - Map("input" -> 10, "timesFour" -> 40), - Map("input" -> 20, "timesFour" -> 80), - Map("input" -> 30, "timesFour" -> 120), - Map("input" -> 40, "timesFour" -> 160), - ) - }, - allowEndingScenarioWithoutSink = true, - ) - } - test("there is a fragment") { val scenarioWithFragment = ScenarioBuilder .streaming("sample-scenario-with-fragment") @@ -421,152 +266,4 @@ class ResultCollectingListenerSpec ) } - test("there is a fragment - without sinks") { - val scenarioWithFragment = - ScenarioBuilder - .streaming("sample-scenario-with-fragment") - .source("source", "start1") - .fragment( - "sub", - "fragment1", - List("fragment1_input" -> "#input".spel), - Map("output" -> "fragmentResult"), - Map("output" -> None) - ) - - val fragment = ScenarioBuilder - .fragment("fragment1", "fragment1_input" -> classOf[Int]) - .filter("filter", "#fragment1_input != 10".spel) - .fragmentOutput("fragmentEnd", "output", "output" -> "#fragment1_input".spel) - - val scenario = FragmentResolver(List(fragment)).resolve(scenarioWithFragment).toOption.get - - withCollectingTestResults( - scenario, - testResults => { - assertNumberOfSamplesThatFinishedInNode(testResults, "sub-fragmentEnd", 3) - assertNumberOfSamplesThatFinishedInNode(testResults, "sub-filter", 1) - transitionVariables(testResults, "source", Some("sub")) shouldBe Set( - Map("input" -> 10), - Map("input" -> 20), - Map("input" -> 30), - Map("input" -> 40), - ) - transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( - Map("fragment1_input" -> 10), - Map("fragment1_input" -> 20), - Map("fragment1_input" -> 30), - Map("fragment1_input" -> 40), - ) - transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( - Map("fragment1_input" -> 10), - Map("fragment1_input" -> 20), - Map("fragment1_input" -> 30), - Map("fragment1_input" -> 40), - ) - // This sample is filtered out and does not proceed further - transitionVariables(testResults, "sub-filter", None) shouldBe Set( - Map("fragment1_input" -> 10), - ) - transitionVariables(testResults, "sub-filter", Some("sub-fragmentEnd")) shouldBe Set( - Map("fragment1_input" -> 20), - Map("fragment1_input" -> 30), - Map("fragment1_input" -> 40), - ) - transitionVariables( - testResults, - "sub-fragmentEnd", - None - ) shouldBe Set( - Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), - Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), - Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), - ) - }, - allowEndingScenarioWithoutSink = true, - ) - } - - private def transitionVariables(testResults: TestResults[Any], fromNodeId: String, toNodeId: Option[String]) = - testResults - .nodeTransitionResults(NodeTransition(fromNodeId, toNodeId)) - .map(_.variables) - .toSet[Map[String, Any]] - .map(_.map { case (key, value) => (key, scalaMap(value)) }) - - private def scalaMap(value: Any): Any = { - value match { - case hashMap: java.util.HashMap[_, _] => hashMap.asScala.toMap - case other => other - } - } - - private def assertNumberOfSamplesThatFinishedInNode(testResults: TestResults[Any], sinkId: String, expected: Int) = - testResults.nodeTransitionResults.get(NodeTransition(sinkId, None)).map(_.length) shouldBe Some(expected) - - private def withCollectingTestResults( - canonicalProcess: CanonicalProcess, - assertions: TestResults[Any] => Unit, - allowEndingScenarioWithoutSink: Boolean = false, - ): Unit = { - ResultsCollectingListenerHolder.withListener { collectingListener => - val model = modelData(collectingListener, AggregateWindowsConfig.Default, allowEndingScenarioWithoutSink) - flinkMiniCluster.withDetachedStreamExecutionEnvironment { env => - val executionResult = new FlinkScenarioUnitTestJob(model).run(canonicalProcess, env) - flinkMiniCluster.waitForJobIsFinished(executionResult.getJobID) - assertions(collectingListener.results) - } - } - } - - private def modelData( - collectingListener: => ResultsCollectingListener[Any], - aggregateWindowsConfig: AggregateWindowsConfig, - allowEndingScenarioWithoutSink: Boolean, - ): LocalModelData = { - def sourceComponent(data: List[Int]) = SourceFactory.noParamUnboundedStreamFactory[Int]( - EmitWatermarkAfterEachElementCollectionSource - .create[Int](data, _ => Instant.now.toEpochMilli, Duration.ofHours(1)) - ) - val config = - if (allowEndingScenarioWithoutSink) { - ConfigFactory.parseString("""allowEndingScenarioWithoutSink: true""") - } else { - ConfigFactory.empty() - } - LocalModelData( - config, - ComponentDefinition("start1", sourceComponent(data1)) :: - ComponentDefinition("start2", sourceComponent(data2)) :: - FlinkBaseUnboundedComponentProvider.create( - DocsConfig.Default, - aggregateWindowsConfig - ) ::: FlinkBaseComponentProvider.Components, - configCreator = new ConfigCreatorWithCollectingListener(collectingListener), - ) - } - - private def removeSinks(canonicalProcess: CanonicalProcess): CanonicalProcess = { - canonicalProcess.mapAllNodes(removeSinks) - } - - private def removeSinks(nodes: List[CanonicalNode]): List[CanonicalNode] = { - nodes.flatMap { - case FlatNode(_: node.Sink) => - None - case FlatNode(other) => - Some(FlatNode(other)) - case SplitNode(data, nexts) => - Some(SplitNode(data, nexts.map(removeSinks))) - case FilterNode(data, nextFalse) => - Some(FilterNode(data, removeSinks(nextFalse))) - case SwitchNode(data, cases, defaultNext) => - Some(SwitchNode(data, cases.map(c => Case(c.expression, removeSinks(c.nodes))), removeSinks(defaultNext))) - case Fragment(data, outputs) => - Some(Fragment(data, outputs.toList.map { case (key, value) => (key, removeSinks(value)) }.toMap)) - } - } - - private def catchExceptionMessage(f: => Any): String = Try(f).failed.get.getMessage - } diff --git a/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ScenarioWithoutSinksSpec.scala b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ScenarioWithoutSinksSpec.scala new file mode 100644 index 00000000000..1a986ecf936 --- /dev/null +++ b/engine/flink/components/base-tests/src/test/scala/pl/touk/nussknacker/engine/flink/ScenarioWithoutSinksSpec.scala @@ -0,0 +1,255 @@ +package pl.touk.nussknacker.engine.flink + +import com.typesafe.scalalogging.LazyLogging +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite +import org.scalatest.matchers.should.Matchers +import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} +import pl.touk.nussknacker.engine.compile.FragmentResolver +import pl.touk.nussknacker.engine.flink.test.FlinkSpec +import pl.touk.nussknacker.test.VeryPatientScalaFutures + +class ScenarioWithoutSinksSpec + extends AnyFunSuite + with BeforeAndAfterAll + with Matchers + with LazyLogging + with VeryPatientScalaFutures + with FlinkSpec + with FlinkMiniClusterTestRunner { + + import pl.touk.nussknacker.engine.spel.SpelExtension._ + + override protected def sourcesWithMockedData: Map[String, List[Int]] = Map( + "start1" -> List(10, 20, 30, 40), + "start2" -> List(100, 200, 300, 400), + ) + + test("ending without sink is not allowed - there is a split - fail to compile without sinks in both branches") { + val scenario = + ScenarioBuilder + .streaming("sample-split") + .source("start-foo", "start1") + .split( + "split", + GraphBuilder + .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) + .endWithoutSink, + GraphBuilder + .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) + .endWithoutSink + ) + + catchExceptionMessage( + withCollectingTestResults(scenario, _ => ()) + ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv2, bv1))" + + } + + test("ending without sink is not allowed - there is a split - fail to compile without sink in one of the branches") { + val scenario = + ScenarioBuilder + .streaming("sample-split") + .source("start-foo", "start1") + .split( + "split", + GraphBuilder + .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) + .endWithoutSink, + GraphBuilder + .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) + .emptySink("end", "dead-end") + ) + + catchExceptionMessage( + withCollectingTestResults(scenario, _ => ()) + ) shouldBe "Compilation errors: InvalidTailOfBranch(Set(bv1))" + + } + + test( + "ending without sink is allowed - union of two sources with additional variable in only one of the branches without sink" + ) { + val scenario = + ScenarioBuilder + .streaming("sample-union") + .sources( + GraphBuilder + .source("start-foo", "start1") + .branchEnd("foo", "union"), + GraphBuilder + .source("start-bar", "start2") + .buildSimpleVariable("bv1", "customVariableInBarBranch", "#input/2".spel) + .branchEnd("bar", "union"), + GraphBuilder + .join( + "union", + "union", + Some("dataIsFrom"), + List( + "foo" -> List("Output expression" -> "'foo source'".spel), + "bar" -> List("Output expression" -> "'bar source'".spel) + ) + ) + .endWithoutSink + ) + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "union", 8) + transitionVariables(testResults, "start-foo", Some("union")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "start-bar", Some("bv1")) shouldBe Set( + Map("input" -> 100), + Map("input" -> 200), + Map("input" -> 300), + Map("input" -> 400), + ) + transitionVariables(testResults, "bv1", Some("union")) shouldBe Set( + Map("input" -> 100, "customVariableInBarBranch" -> 50), + Map("input" -> 200, "customVariableInBarBranch" -> 100), + Map("input" -> 300, "customVariableInBarBranch" -> 150), + Map("input" -> 400, "customVariableInBarBranch" -> 200), + ) + transitionVariables(testResults, "union", None) shouldBe Set( + Map("input" -> 10, "dataIsFrom" -> "foo source"), + Map("input" -> 20, "dataIsFrom" -> "foo source"), + Map("input" -> 30, "dataIsFrom" -> "foo source"), + Map("input" -> 40, "dataIsFrom" -> "foo source"), + Map("input" -> 100, "customVariableInBarBranch" -> 50, "dataIsFrom" -> "bar source"), + Map("input" -> 200, "customVariableInBarBranch" -> 100, "dataIsFrom" -> "bar source"), + Map("input" -> 300, "customVariableInBarBranch" -> 150, "dataIsFrom" -> "bar source"), + Map("input" -> 400, "customVariableInBarBranch" -> 200, "dataIsFrom" -> "bar source"), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + + test("ending without sink is allowed - there is a split - without sinks") { + val scenario = + ScenarioBuilder + .streaming("sample-split") + .source("start-foo", "start1") + .split( + "split", + GraphBuilder + .buildSimpleVariable("bv1", "timesTwo", "#input*2".spel) + .endWithoutSink, + GraphBuilder + .buildSimpleVariable("bv2", "timesFour", "#input*4".spel) + .endWithoutSink + ) + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "bv1", 4) + assertNumberOfSamplesThatFinishedInNode(testResults, "bv2", 4) + transitionVariables(testResults, "start-foo", Some("split")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "split", Some("bv1")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "split", Some("bv2")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "bv1", None) shouldBe Set( + Map("input" -> 10, "timesTwo" -> 20), + Map("input" -> 20, "timesTwo" -> 40), + Map("input" -> 30, "timesTwo" -> 60), + Map("input" -> 40, "timesTwo" -> 80), + ) + transitionVariables(testResults, "bv2", None) shouldBe Set( + Map("input" -> 10, "timesFour" -> 40), + Map("input" -> 20, "timesFour" -> 80), + Map("input" -> 30, "timesFour" -> 120), + Map("input" -> 40, "timesFour" -> 160), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + + test("ending without sink is allowed - there is a fragment - without sink") { + val scenarioWithFragment = + ScenarioBuilder + .streaming("sample-scenario-with-fragment") + .source("source", "start1") + .fragment( + "sub", + "fragment1", + List("fragment1_input" -> "#input".spel), + Map("output" -> "fragmentResult"), + Map("output" -> None) + ) + + val fragment = ScenarioBuilder + .fragment("fragment1", "fragment1_input" -> classOf[Int]) + .filter("filter", "#fragment1_input != 10".spel) + .fragmentOutput("fragmentEnd", "output", "output" -> "#fragment1_input".spel) + + val scenario = FragmentResolver(List(fragment)).resolve(scenarioWithFragment).toOption.get + + withCollectingTestResults( + scenario, + testResults => { + assertNumberOfSamplesThatFinishedInNode(testResults, "sub-fragmentEnd", 3) + assertNumberOfSamplesThatFinishedInNode(testResults, "sub-filter", 1) + transitionVariables(testResults, "source", Some("sub")) shouldBe Set( + Map("input" -> 10), + Map("input" -> 20), + Map("input" -> 30), + Map("input" -> 40), + ) + transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( + Map("fragment1_input" -> 10), + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + transitionVariables(testResults, "sub", Some("sub-filter")) shouldBe Set( + Map("fragment1_input" -> 10), + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + // This sample is filtered out and does not proceed further + transitionVariables(testResults, "sub-filter", None) shouldBe Set( + Map("fragment1_input" -> 10), + ) + transitionVariables(testResults, "sub-filter", Some("sub-fragmentEnd")) shouldBe Set( + Map("fragment1_input" -> 20), + Map("fragment1_input" -> 30), + Map("fragment1_input" -> 40), + ) + transitionVariables( + testResults, + "sub-fragmentEnd", + None + ) shouldBe Set( + Map("input" -> 20, "fragmentResult" -> Map("output" -> 20)), + Map("input" -> 30, "fragmentResult" -> Map("output" -> 30)), + Map("input" -> 40, "fragmentResult" -> Map("output" -> 40)), + ) + }, + allowEndingScenarioWithoutSink = true, + ) + } + +} From 2403821958317f0679dd5d45e4f092d10f82c587 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Wed, 7 May 2025 19:29:46 +0200 Subject: [PATCH 12/13] review changes --- .../AdditionalInfoProviders.scala | 4 ++-- .../ui/definition/DefinitionsService.scala | 2 +- .../processingtype/ProcessingTypeData.scala | 4 ++-- .../ui/config/ConfigurationTest.scala | 2 +- .../ui/definition/DefinitionsServiceSpec.scala | 2 +- .../engine/flink/api/NkGlobalParameters.scala | 6 +++--- .../process/ExecutionConfigPreparer.scala | 6 +++--- .../engine/process/FlinkJobConfig.scala | 6 +++--- .../FlinkProcessCompilerDataFactory.scala | 12 +++++++----- ...tubbedFlinkProcessCompilerDataFactory.scala | 4 ++-- .../engine/process/util/Serializers.scala | 2 +- ...KafkaTransactionalScenarioInterpreter.scala | 2 +- .../lite/ScenarioInterpreterFactory.scala | 2 +- .../nussknacker/engine/BaseModelData.scala | 6 ++---- .../pl/touk/nussknacker/engine/ModelData.scala | 18 ++++++++++++------ .../engine/DefaultModelConfigLoaderTest.scala | 8 ++++---- ...CompilerDataFactoryWithTestComponents.scala | 4 ++-- 17 files changed, 48 insertions(+), 42 deletions(-) diff --git a/designer/server/src/main/scala/pl/touk/nussknacker/ui/additionalInfo/AdditionalInfoProviders.scala b/designer/server/src/main/scala/pl/touk/nussknacker/ui/additionalInfo/AdditionalInfoProviders.scala index 4ab75ddce82..89ee5be7a19 100644 --- a/designer/server/src/main/scala/pl/touk/nussknacker/ui/additionalInfo/AdditionalInfoProviders.scala +++ b/designer/server/src/main/scala/pl/touk/nussknacker/ui/additionalInfo/AdditionalInfoProviders.scala @@ -23,7 +23,7 @@ class AdditionalInfoProviders(typeToConfig: ProcessingTypeDataProvider[ModelData ScalaServiceLoader .load[AdditionalInfoProvider](pt.modelClassLoader) .headOption - .map(_.nodeAdditionalInfo(pt.modelConfig)) + .map(_.nodeAdditionalInfo(pt.modelConfig.underlyingConfig)) ) private val propertiesProviders: ProcessingTypeDataProvider[Option[MetaData => Future[Option[AdditionalInfo]]], _] = @@ -31,7 +31,7 @@ class AdditionalInfoProviders(typeToConfig: ProcessingTypeDataProvider[ModelData ScalaServiceLoader .load[AdditionalInfoProvider](pt.modelClassLoader) .headOption - .map(_.propertiesAdditionalInfo(pt.modelConfig)) + .map(_.propertiesAdditionalInfo(pt.modelConfig.underlyingConfig)) ) def prepareAdditionalInfoForNode(nodeData: NodeData, processingType: ProcessingType)( diff --git a/designer/server/src/main/scala/pl/touk/nussknacker/ui/definition/DefinitionsService.scala b/designer/server/src/main/scala/pl/touk/nussknacker/ui/definition/DefinitionsService.scala index 2d05d80a9b3..9660c71bd16 100644 --- a/designer/server/src/main/scala/pl/touk/nussknacker/ui/definition/DefinitionsService.scala +++ b/designer/server/src/main/scala/pl/touk/nussknacker/ui/definition/DefinitionsService.scala @@ -84,7 +84,7 @@ class DefinitionsService( } import net.ceedubs.ficus.Ficus._ - val scenarioPropertiesDocsUrl = modelData.modelConfig.getAs[String]("scenarioPropertiesDocsUrl") + val scenarioPropertiesDocsUrl = modelData.modelConfig.underlyingConfig.getAs[String]("scenarioPropertiesDocsUrl") prepareUIDefinitions( withStaticDefinition, diff --git a/designer/server/src/main/scala/pl/touk/nussknacker/ui/process/processingtype/ProcessingTypeData.scala b/designer/server/src/main/scala/pl/touk/nussknacker/ui/process/processingtype/ProcessingTypeData.scala index d99b6a16bd2..051b7683fa9 100644 --- a/designer/server/src/main/scala/pl/touk/nussknacker/ui/process/processingtype/ProcessingTypeData.scala +++ b/designer/server/src/main/scala/pl/touk/nussknacker/ui/process/processingtype/ProcessingTypeData.scala @@ -76,12 +76,12 @@ object ProcessingTypeData { componentDefinitionExtractionMode: ComponentDefinitionExtractionMode ) = { // TODO: consider using ParameterName for property names instead of String (for scenario and fragment properties) - val scenarioProperties = deploymentData.deploymentScenarioPropertiesConfig ++ modelData.modelConfig + val scenarioProperties = deploymentData.deploymentScenarioPropertiesConfig ++ modelData.modelConfig.underlyingConfig .getOrElse[Map[String, ScenarioPropertyConfig]]( "scenarioPropertiesConfig", Map.empty ) - val fragmentProperties = modelData.modelConfig + val fragmentProperties = modelData.modelConfig.underlyingConfig .getOrElse[Map[String, ScenarioPropertyConfig]]("fragmentPropertiesConfig", Map.empty) val staticDefinitionForDynamicComponents = diff --git a/designer/server/src/test/scala/pl/touk/nussknacker/ui/config/ConfigurationTest.scala b/designer/server/src/test/scala/pl/touk/nussknacker/ui/config/ConfigurationTest.scala index a761ca43da4..ce586d4b92f 100644 --- a/designer/server/src/test/scala/pl/touk/nussknacker/ui/config/ConfigurationTest.scala +++ b/designer/server/src/test/scala/pl/touk/nussknacker/ui/config/ConfigurationTest.scala @@ -28,7 +28,7 @@ class ConfigurationTest extends AnyFunSuite with WithTestDeploymentManagerClassL ) } - private lazy val modelDataConfig = modelData.modelConfig + private lazy val modelDataConfig = modelData.modelConfig.underlyingConfig private def classLoader = { getClass.getClassLoader diff --git a/designer/server/src/test/scala/pl/touk/nussknacker/ui/definition/DefinitionsServiceSpec.scala b/designer/server/src/test/scala/pl/touk/nussknacker/ui/definition/DefinitionsServiceSpec.scala index 3b752c600c0..24861c3ae25 100644 --- a/designer/server/src/test/scala/pl/touk/nussknacker/ui/definition/DefinitionsServiceSpec.scala +++ b/designer/server/src/test/scala/pl/touk/nussknacker/ui/definition/DefinitionsServiceSpec.scala @@ -340,7 +340,7 @@ class DefinitionsServiceSpec extends AnyFunSuite with Matchers with PatientScala val processingType = Streaming val alignedComponentsDefinitionProvider = new AlignedComponentsDefinitionProvider( - new BuiltInComponentsDefinitionsPreparer(ComponentsUiConfigParser.parse(model.modelConfig)), + new BuiltInComponentsDefinitionsPreparer(ComponentsUiConfigParser.parse(model.modelConfig.underlyingConfig)), new FragmentComponentDefinitionExtractor( getClass.getClassLoader, model.modelDefinitionWithClasses.classDefinitions, diff --git a/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala b/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala index 4b58e593e41..4359907176c 100644 --- a/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala +++ b/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala @@ -1,7 +1,6 @@ package pl.touk.nussknacker.engine.flink.api import _root_.java.util -import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging import io.circe.Encoder import net.ceedubs.ficus.Ficus._ @@ -10,6 +9,7 @@ import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo import pl.touk.nussknacker.engine.api.process.{ProcessId, ProcessName, VersionId} +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters.NkGlobalParametersToMapEncoder import scala.jdk.CollectionConverters._ @@ -50,10 +50,10 @@ object NkGlobalParameters extends LazyLogging { modelInfo: ModelInfo, deploymentId: String, // TODO: Pass here DeploymentId? processVersion: ProcessVersion, - modelConfig: Config, + modelConfig: ModelConfig, additionalInformation: Map[String, String] ): NkGlobalParameters = { - val configGlobalParameters = modelConfig.getAs[ConfigGlobalParameters]("globalParameters") + val configGlobalParameters = modelConfig.underlyingConfig.getAs[ConfigGlobalParameters]("globalParameters") NkGlobalParameters( modelInfo, deploymentId, diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala index 0084a3a6f8e..bd6fab6a635 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala @@ -1,12 +1,12 @@ package pl.touk.nussknacker.engine.process -import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging import net.ceedubs.ficus.Ficus._ import org.apache.flink.api.common.ExecutionConfig import pl.touk.nussknacker.engine.ModelData import pl.touk.nussknacker.engine.api.JobData import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters import pl.touk.nussknacker.engine.flink.api.typeinformation.FlinkTypeInfoRegistrar @@ -46,7 +46,7 @@ object ExecutionConfigPreparer extends LazyLogging { } } - class ProcessSettingsPreparer(modelConfig: Config, modelInfo: ModelInfo) extends ExecutionConfigPreparer { + class ProcessSettingsPreparer(modelConfig: ModelConfig, modelInfo: ModelInfo) extends ExecutionConfigPreparer { override def prepareExecutionConfig( config: ExecutionConfig @@ -80,7 +80,7 @@ object ExecutionConfigPreparer extends LazyLogging { class SerializationPreparer(modelData: ModelData) extends ExecutionConfigPreparer { protected def enableObjectReuse: Boolean = - modelData.modelConfig.getOrElse[Boolean]("enableObjectReuse", true) + modelData.modelConfig.underlyingConfig.getOrElse[Boolean]("enableObjectReuse", true) override def prepareExecutionConfig( config: ExecutionConfig diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala index af8e9773008..ba02ed67eca 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala @@ -1,6 +1,6 @@ package pl.touk.nussknacker.engine.process -import com.typesafe.config.Config +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.process.FlinkJobConfig.ExecutionMode.ExecutionMode import pl.touk.nussknacker.engine.process.util.StateConfiguration.RocksDBStateBackendConfig @@ -16,8 +16,8 @@ object FlinkJobConfig { import net.ceedubs.ficus.readers.ArbitraryTypeReader._ import net.ceedubs.ficus.readers.EnumerationReader._ - def parse(modelConfig: Config): FlinkJobConfig = { - modelConfig.as[FlinkJobConfig] + def parse(modelConfig: ModelConfig): FlinkJobConfig = { + modelConfig.underlyingConfig.as[FlinkJobConfig] } object ExecutionMode extends Enumeration { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala index e454ecef208..7ad8ea71beb 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala @@ -18,6 +18,7 @@ import pl.touk.nussknacker.engine.api.component.{ import pl.touk.nussknacker.engine.api.definition.EngineScenarioCompilationDependencies import pl.touk.nussknacker.engine.api.dict.EngineDictRegistry import pl.touk.nussknacker.engine.api.process.{ProcessConfigCreator, ProcessObjectDependencies} +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.compile._ import pl.touk.nussknacker.engine.compile.nodecompilation.LazyParameterCreationStrategy import pl.touk.nussknacker.engine.definition.clazz.ClassDefinitionSet @@ -44,7 +45,7 @@ import scala.concurrent.duration.FiniteDuration class FlinkProcessCompilerDataFactory( creator: ProcessConfigCreator, extractModelDefinition: ExtractDefinitionFun, - modelConfig: Config, + modelConfig: ModelConfig, runtimeMode: RuntimeMode, configsFromProviderWithDictionaryEditor: Map[DesignerWideComponentId, ComponentAdditionalConfig], nodesData: NodesDeploymentData, @@ -74,23 +75,24 @@ class FlinkProcessCompilerDataFactory( usedNodes: UsedNodes, userCodeClassLoader: ClassLoader ): FlinkProcessCompilerData = { - val modelDependencies = ProcessObjectDependencies.withConfig(modelConfig) + val modelDependencies = ProcessObjectDependencies.withConfig(modelConfig.underlyingConfig) // TODO: this should be somewhere else? - val timeout = modelConfig.as[FiniteDuration]("timeout") + val timeout = modelConfig.underlyingConfig.as[FiniteDuration]("timeout") // TODO: should this be the default? val asyncExecutionContextPreparer = creator .asyncExecutionContextPreparer(modelDependencies) .getOrElse( - modelConfig.as[DefaultServiceExecutionContextPreparer]("asyncExecutionConfig") + modelConfig.underlyingConfig.as[DefaultServiceExecutionContextPreparer]("asyncExecutionConfig") ) val defaultListeners = prepareDefaultListeners(usedNodes) ++ creator.listeners(modelDependencies) val listenersToUse = adjustListeners(defaultListeners, modelDependencies) val (definitionWithTypes, dictRegistry) = definitions(modelDependencies, userCodeClassLoader) - val customProcessValidator = CustomProcessValidatorLoader.loadProcessValidators(userCodeClassLoader, modelConfig) + val customProcessValidator = + CustomProcessValidatorLoader.loadProcessValidators(userCodeClassLoader, modelConfig.underlyingConfig) val compilerData = ProcessCompilerData.prepare( JobData(metaData, processVersion), diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala index 56cb5c01b5e..ce5d807a44a 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala @@ -1,6 +1,5 @@ package pl.touk.nussknacker.engine.process.compiler -import com.typesafe.config.Config import pl.touk.nussknacker.engine.ModelData.ExtractDefinitionFun import pl.touk.nussknacker.engine.RuntimeMode import pl.touk.nussknacker.engine.api.{NodeId, Params} @@ -12,6 +11,7 @@ import pl.touk.nussknacker.engine.api.component.{ } import pl.touk.nussknacker.engine.api.context.ContextTransformation import pl.touk.nussknacker.engine.api.process.ProcessConfigCreator +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.api.typed.ReturningType import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess @@ -33,7 +33,7 @@ abstract class StubbedFlinkProcessCompilerDataFactory( process: CanonicalProcess, creator: ProcessConfigCreator, extractModelDefinition: ExtractDefinitionFun, - modelConfig: Config, + modelConfig: ModelConfig, runtimeMode: RuntimeMode, configsFromProviderWithDictionaryEditor: Map[DesignerWideComponentId, ComponentAdditionalConfig], nodesDeploymentData: NodesDeploymentData, diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/util/Serializers.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/util/Serializers.scala index e37b5fb3fae..d0a03ff6f28 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/util/Serializers.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/util/Serializers.scala @@ -30,7 +30,7 @@ object Serializers extends LazyLogging { implicitly[SerializerRegistrar[SpelMapHack]] :: Nil).foreach(_.registerIn(config)) ScalaServiceLoader .load[SerializersRegistrar](getClass.getClassLoader) - .foreach(_.register(modelData.modelConfig, config)) + .foreach(_.register(modelData.modelConfig.underlyingConfig, config)) TimeSerializers.addDefaultSerializers(config) } diff --git a/engine/lite/kafka/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/kafka/KafkaTransactionalScenarioInterpreter.scala b/engine/lite/kafka/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/kafka/KafkaTransactionalScenarioInterpreter.scala index 66c0aab8656..05b6c6afe8b 100644 --- a/engine/lite/kafka/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/kafka/KafkaTransactionalScenarioInterpreter.scala +++ b/engine/lite/kafka/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/kafka/KafkaTransactionalScenarioInterpreter.scala @@ -117,7 +117,7 @@ class KafkaTransactionalScenarioInterpreter private[kafka] ( private val sourceMetrics = new SourceMetrics(interpreter.sources.keys) - private val interpreterConfig = modelData.modelConfig.as[KafkaInterpreterConfig] + private val interpreterConfig = modelData.modelConfig.underlyingConfig.as[KafkaInterpreterConfig] private val taskRunner: TaskRunner = new TaskRunner( scenario.name.value, diff --git a/engine/lite/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/ScenarioInterpreterFactory.scala b/engine/lite/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/ScenarioInterpreterFactory.scala index d15c463698a..534a5f553df 100644 --- a/engine/lite/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/ScenarioInterpreterFactory.scala +++ b/engine/lite/runtime/src/main/scala/pl/touk/nussknacker/engine/lite/ScenarioInterpreterFactory.scala @@ -69,7 +69,7 @@ object ScenarioInterpreterFactory { modelData.withModelClassloaderAsContextClassLoader { val creator = modelData.configCreator - val modelDependencies = ProcessObjectDependencies.withConfig(modelData.modelConfig) + val modelDependencies = ProcessObjectDependencies.withConfig(modelData.modelConfig.underlyingConfig) val allNodes = process.collectAllNodes val countingListeners = List( diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala index 7209aab2dcb..8ca3eb0d183 100644 --- a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala +++ b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala @@ -1,12 +1,10 @@ package pl.touk.nussknacker.engine -import com.typesafe.config.Config import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.classloader.ModelClassLoader import pl.touk.nussknacker.engine.modelconfig.InputConfigDuringExecution -import java.net.URLClassLoader - // TODO: Replace ModelData -> BasedModelData inheritance with composition. Thanks to that it won't be needed to downcast // to ModelData in case of interpreter invocation trait BaseModelData { @@ -19,7 +17,7 @@ trait BaseModelData { def inputConfigDuringExecution: InputConfigDuringExecution - def modelConfig: Config + def modelConfig: ModelConfig } diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala index f9a00cd0f2c..af449d78116 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala @@ -9,6 +9,7 @@ import pl.touk.nussknacker.engine.api.dict.{DictServicesFactory, EngineDictRegis import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy import pl.touk.nussknacker.engine.api.process.{ProcessConfigCreator, ProcessObjectDependencies} +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.classloader.ModelClassLoader import pl.touk.nussknacker.engine.definition.component.Components.ComponentDefinitionExtractionMode import pl.touk.nussknacker.engine.definition.model.{ @@ -163,7 +164,7 @@ case class ClassLoaderModelData private ( } } - override val namingStrategy: NamingStrategy = NamingStrategy.fromConfig(modelConfig) + override val namingStrategy: NamingStrategy = NamingStrategy.fromConfig(modelConfig.underlyingConfig) override val extractModelDefinitionFun: ExtractDefinitionFun = new ExtractDefinitionFunImpl( @@ -245,14 +246,17 @@ trait ModelData extends BaseModelData with AutoCloseable { DictServicesFactoryLoader.justOne(modelClassLoader) final lazy val designerDictServices: UiDictServices = - dictServicesFactory.createUiDictServices(modelDefinition.expressionConfig.dictionaries, modelConfig) + dictServicesFactory.createUiDictServices( + modelDefinition.expressionConfig.dictionaries, + modelConfig.underlyingConfig + ) final lazy val engineDictRegistry: EngineDictRegistry = dictServicesFactory.createEngineDictRegistry(modelDefinition.expressionConfig.dictionaries) // TODO: remove it, see notice in CustomProcessValidatorFactory final def customProcessValidator: CustomProcessValidator = { - CustomProcessValidatorLoader.loadProcessValidators(modelClassLoader, modelConfig) + CustomProcessValidatorLoader.loadProcessValidators(modelClassLoader, modelConfig.underlyingConfig) } final def withModelClassloaderAsContextClassLoader[T](block: => T): T = { @@ -265,10 +269,12 @@ trait ModelData extends BaseModelData with AutoCloseable { def modelConfigLoader: ModelConfigLoader - final override lazy val modelConfig: Config = - modelConfigLoader.resolveConfig(inputConfigDuringExecution, modelClassLoader) + final override lazy val modelConfig: ModelConfig = + ProcessObjectDependencies.parseModelConfig( + modelConfigLoader.resolveConfig(inputConfigDuringExecution, modelClassLoader) + ) - final lazy val componentsUiConfig: ComponentsUiConfig = ComponentsUiConfigParser.parse(modelConfig) + final lazy val componentsUiConfig: ComponentsUiConfig = ComponentsUiConfigParser.parse(modelConfig.underlyingConfig) final def close(): Unit = { designerDictServices.close() diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/DefaultModelConfigLoaderTest.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/DefaultModelConfigLoaderTest.scala index 74b087d59d3..b30a37cf930 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/DefaultModelConfigLoaderTest.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/DefaultModelConfigLoaderTest.scala @@ -26,7 +26,7 @@ class DefaultModelConfigLoaderTest extends AnyFunSuite with Matchers { } test("should load model.conf and override with given") { - val config = LocalModelData(inputConfig, List.empty).modelConfig + val config = LocalModelData(inputConfig, List.empty).modelConfig.underlyingConfig config.getString("property1") shouldBe "value1" config.getString("property2") shouldBe "value1Suffix" @@ -45,7 +45,7 @@ class DefaultModelConfigLoaderTest extends AnyFunSuite with Matchers { } test("should resolve environment variables") { - val config = LocalModelData(ConfigFactory.empty(), List.empty).modelConfig + val config = LocalModelData(ConfigFactory.empty(), List.empty).modelConfig.underlyingConfig val envPath = System.getenv("PATH") envPath shouldNot be(null) @@ -53,13 +53,13 @@ class DefaultModelConfigLoaderTest extends AnyFunSuite with Matchers { } test("should not load application.conf") { - val config = LocalModelData(inputConfig, List.empty).modelConfig + val config = LocalModelData(inputConfig, List.empty).modelConfig.underlyingConfig config.hasPath("shouldNotLoad") shouldBe false } test("should not contain java.class.path") { - val config = LocalModelData(inputConfig, List.empty).modelConfig + val config = LocalModelData(inputConfig, List.empty).modelConfig.underlyingConfig // classpath can grow very long and there's a 65 KB limit on a single String value in Configuration // that we already hit in CI, see: https://github.com/lightbend/config/issues/627 diff --git a/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala b/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala index dda143b546f..4f1456f28ef 100644 --- a/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala +++ b/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala @@ -1,6 +1,5 @@ package pl.touk.nussknacker.engine.flink.util.test -import com.typesafe.config.Config import pl.touk.nussknacker.engine.{ModelData, RuntimeMode} import pl.touk.nussknacker.engine.ModelData.ExtractDefinitionFun import pl.touk.nussknacker.engine.api._ @@ -10,6 +9,7 @@ import pl.touk.nussknacker.engine.api.component.{ NodesDeploymentData } import pl.touk.nussknacker.engine.api.process._ +import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.definition.clazz.ClassDefinitionSet import pl.touk.nussknacker.engine.definition.component.ComponentDefinitionWithImplementation import pl.touk.nussknacker.engine.definition.globalvariables.GlobalVariableDefinitionWithImplementation @@ -47,7 +47,7 @@ object FlinkProcessCompilerDataFactoryWithTestComponents { def apply( creator: ProcessConfigCreator, extractModelDefinition: ExtractDefinitionFun, - modelConfig: Config, + modelConfig: ModelConfig, runtimeMode: RuntimeMode, testExtensionsHolder: TestExtensionsHolder, resultsCollectingListener: ResultsCollectingListener[Any], From 23d13f01e2a6914e10103ea5bbaea998baa9114f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Goworko?= Date: Wed, 7 May 2025 23:28:40 +0200 Subject: [PATCH 13/13] review changes --- .../touk/nussknacker/engine/ModelConfig.scala | 23 +++++++++++++++++++ .../process/ProcessObjectDependencies.scala | 22 +++--------------- .../engine/flink/api/NkGlobalParameters.scala | 2 +- .../process/ExecutionConfigPreparer.scala | 3 +-- .../engine/process/FlinkJobConfig.scala | 2 +- .../FlinkProcessCompilerDataFactory.scala | 11 +-------- ...ubbedFlinkProcessCompilerDataFactory.scala | 3 +-- .../nussknacker/engine/BaseModelData.scala | 1 - .../touk/nussknacker/engine/ModelData.scala | 5 +--- .../nussknacker/engine/InterpreterSpec.scala | 1 - ...ompilerDataFactoryWithTestComponents.scala | 3 +-- 11 files changed, 33 insertions(+), 43 deletions(-) create mode 100644 components-api/src/main/scala/pl/touk/nussknacker/engine/ModelConfig.scala diff --git a/components-api/src/main/scala/pl/touk/nussknacker/engine/ModelConfig.scala b/components-api/src/main/scala/pl/touk/nussknacker/engine/ModelConfig.scala new file mode 100644 index 00000000000..9643ed9c2b9 --- /dev/null +++ b/components-api/src/main/scala/pl/touk/nussknacker/engine/ModelConfig.scala @@ -0,0 +1,23 @@ +package pl.touk.nussknacker.engine + +import com.typesafe.config.Config +import net.ceedubs.ficus.Ficus.toFicusConfig +import net.ceedubs.ficus.readers.AnyValReaders._ +import net.ceedubs.ficus.readers.OptionReader._ + +final case class ModelConfig( + allowEndingScenarioWithoutSink: Boolean, + // TODO: we should parse this underlying config as ModelConfig class fields instead of passing raw config + underlyingConfig: Config, +) + +object ModelConfig { + + def parse(modelConfig: Config): ModelConfig = { + ModelConfig( + allowEndingScenarioWithoutSink = modelConfig.getOrElse[Boolean]("allowEndingScenarioWithoutSink", false), + underlyingConfig = modelConfig, + ) + } + +} diff --git a/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala b/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala index ad5f79c737d..b7f3c95b60c 100644 --- a/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala +++ b/components-api/src/main/scala/pl/touk/nussknacker/engine/api/process/ProcessObjectDependencies.scala @@ -1,11 +1,8 @@ package pl.touk.nussknacker.engine.api.process import com.typesafe.config.Config -import net.ceedubs.ficus.Ficus.toFicusConfig -import net.ceedubs.ficus.readers.AnyValReaders._ -import net.ceedubs.ficus.readers.OptionReader._ +import pl.touk.nussknacker.engine.ModelConfig import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig // TODO: Rename to ModelDependencies + rename config to modelConfig final case class ProcessObjectDependencies private (modelConfig: ModelConfig, namingStrategy: NamingStrategy) @@ -16,24 +13,11 @@ final case class ProcessObjectDependencies private (modelConfig: ModelConfig, na object ProcessObjectDependencies { def apply(underlyingConfig: Config, namingStrategy: NamingStrategy): ProcessObjectDependencies = { - ProcessObjectDependencies(parseModelConfig(underlyingConfig), namingStrategy) + new ProcessObjectDependencies(ModelConfig.parse(underlyingConfig), namingStrategy) } def withConfig(config: Config): ProcessObjectDependencies = { - ProcessObjectDependencies(parseModelConfig(config), NamingStrategy.fromConfig(config)) + ProcessObjectDependencies(ModelConfig.parse(config), NamingStrategy.fromConfig(config)) } - def parseModelConfig(modelConfig: Config): ModelConfig = { - ModelConfig( - allowEndingScenarioWithoutSink = modelConfig.getOrElse[Boolean]("allowEndingScenarioWithoutSink", false), - underlyingConfig = modelConfig, - ) - } - - final case class ModelConfig( - allowEndingScenarioWithoutSink: Boolean, - // TODO: we should parse this underlying config as ModelConfig class fields instead of passing raw config - underlyingConfig: Config, - ) - } diff --git a/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala b/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala index 4359907176c..0ece86d89a0 100644 --- a/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala +++ b/engine/flink/components-api/src/main/scala/pl/touk/nussknacker/engine/flink/api/NkGlobalParameters.scala @@ -6,10 +6,10 @@ import io.circe.Encoder import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ArbitraryTypeReader._ import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters +import pl.touk.nussknacker.engine.ModelConfig import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo import pl.touk.nussknacker.engine.api.process.{ProcessId, ProcessName, VersionId} -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters.NkGlobalParametersToMapEncoder import scala.jdk.CollectionConverters._ diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala index bd6fab6a635..69e5afa4fb9 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/ExecutionConfigPreparer.scala @@ -3,10 +3,9 @@ package pl.touk.nussknacker.engine.process import com.typesafe.scalalogging.LazyLogging import net.ceedubs.ficus.Ficus._ import org.apache.flink.api.common.ExecutionConfig -import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.{ModelConfig, ModelData} import pl.touk.nussknacker.engine.api.JobData import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.api.NkGlobalParameters import pl.touk.nussknacker.engine.flink.api.typeinformation.FlinkTypeInfoRegistrar diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala index ba02ed67eca..5b9e61a70dc 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/FlinkJobConfig.scala @@ -1,6 +1,6 @@ package pl.touk.nussknacker.engine.process -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig +import pl.touk.nussknacker.engine.ModelConfig import pl.touk.nussknacker.engine.process.FlinkJobConfig.ExecutionMode.ExecutionMode import pl.touk.nussknacker.engine.process.util.StateConfiguration.RocksDBStateBackendConfig diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala index 7ad8ea71beb..7317053ec82 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/FlinkProcessCompilerDataFactory.scala @@ -1,13 +1,6 @@ package pl.touk.nussknacker.engine.process.compiler -import com.typesafe.config.Config -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.{ - CustomProcessValidatorLoader, - ModelData, - RuntimeMode, - ScenarioCompilationDependencies -} +import pl.touk.nussknacker.engine.{CustomProcessValidatorLoader, ModelConfig, ModelData, RuntimeMode} import pl.touk.nussknacker.engine.ModelData.ExtractDefinitionFun import pl.touk.nussknacker.engine.api.{JobData, MetaData, ProcessListener, ProcessVersion} import pl.touk.nussknacker.engine.api.component.{ @@ -15,10 +8,8 @@ import pl.touk.nussknacker.engine.api.component.{ DesignerWideComponentId, NodesDeploymentData } -import pl.touk.nussknacker.engine.api.definition.EngineScenarioCompilationDependencies import pl.touk.nussknacker.engine.api.dict.EngineDictRegistry import pl.touk.nussknacker.engine.api.process.{ProcessConfigCreator, ProcessObjectDependencies} -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.compile._ import pl.touk.nussknacker.engine.compile.nodecompilation.LazyParameterCreationStrategy import pl.touk.nussknacker.engine.definition.clazz.ClassDefinitionSet diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala index ce5d807a44a..a8744203f02 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/compiler/StubbedFlinkProcessCompilerDataFactory.scala @@ -1,7 +1,7 @@ package pl.touk.nussknacker.engine.process.compiler +import pl.touk.nussknacker.engine.{ModelConfig, RuntimeMode} import pl.touk.nussknacker.engine.ModelData.ExtractDefinitionFun -import pl.touk.nussknacker.engine.RuntimeMode import pl.touk.nussknacker.engine.api.{NodeId, Params} import pl.touk.nussknacker.engine.api.component.{ ComponentAdditionalConfig, @@ -11,7 +11,6 @@ import pl.touk.nussknacker.engine.api.component.{ } import pl.touk.nussknacker.engine.api.context.ContextTransformation import pl.touk.nussknacker.engine.api.process.ProcessConfigCreator -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.api.typed.ReturningType import pl.touk.nussknacker.engine.api.typed.typing.{TypingResult, Unknown} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess diff --git a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala index 8ca3eb0d183..92199263d0a 100644 --- a/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala +++ b/extensions-api/src/main/scala/pl/touk/nussknacker/engine/BaseModelData.scala @@ -1,7 +1,6 @@ package pl.touk.nussknacker.engine import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.classloader.ModelClassLoader import pl.touk.nussknacker.engine.modelconfig.InputConfigDuringExecution diff --git a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala index af449d78116..f1f28116493 100644 --- a/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala +++ b/scenario-compiler/src/main/scala/pl/touk/nussknacker/engine/ModelData.scala @@ -9,7 +9,6 @@ import pl.touk.nussknacker.engine.api.dict.{DictServicesFactory, EngineDictRegis import pl.touk.nussknacker.engine.api.modelinfo.ModelInfo import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy import pl.touk.nussknacker.engine.api.process.{ProcessConfigCreator, ProcessObjectDependencies} -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.classloader.ModelClassLoader import pl.touk.nussknacker.engine.definition.component.Components.ComponentDefinitionExtractionMode import pl.touk.nussknacker.engine.definition.model.{ @@ -270,9 +269,7 @@ trait ModelData extends BaseModelData with AutoCloseable { def modelConfigLoader: ModelConfigLoader final override lazy val modelConfig: ModelConfig = - ProcessObjectDependencies.parseModelConfig( - modelConfigLoader.resolveConfig(inputConfigDuringExecution, modelClassLoader) - ) + ModelConfig.parse(modelConfigLoader.resolveConfig(inputConfigDuringExecution, modelClassLoader)) final lazy val componentsUiConfig: ComponentsUiConfig = ComponentsUiConfigParser.parse(modelConfig.underlyingConfig) diff --git a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala index e079bfb4560..fbca4ace9af 100644 --- a/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala +++ b/scenario-compiler/src/test/scala/pl/touk/nussknacker/engine/InterpreterSpec.scala @@ -29,7 +29,6 @@ import pl.touk.nussknacker.engine.api.exception.NuExceptionInfo import pl.touk.nussknacker.engine.api.expression._ import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process._ -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.api.test.InvocationCollectors import pl.touk.nussknacker.engine.api.test.InvocationCollectors.ServiceInvocationCollector import pl.touk.nussknacker.engine.api.typed.typing diff --git a/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala b/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala index 4f1456f28ef..59cb06331c1 100644 --- a/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala +++ b/utils/flink-components-testkit/src/main/scala/pl/touk/nussknacker/engine/flink/util/test/FlinkProcessCompilerDataFactoryWithTestComponents.scala @@ -1,6 +1,6 @@ package pl.touk.nussknacker.engine.flink.util.test -import pl.touk.nussknacker.engine.{ModelData, RuntimeMode} +import pl.touk.nussknacker.engine.{ModelConfig, ModelData, RuntimeMode} import pl.touk.nussknacker.engine.ModelData.ExtractDefinitionFun import pl.touk.nussknacker.engine.api._ import pl.touk.nussknacker.engine.api.component.{ @@ -9,7 +9,6 @@ import pl.touk.nussknacker.engine.api.component.{ NodesDeploymentData } import pl.touk.nussknacker.engine.api.process._ -import pl.touk.nussknacker.engine.api.process.ProcessObjectDependencies.ModelConfig import pl.touk.nussknacker.engine.definition.clazz.ClassDefinitionSet import pl.touk.nussknacker.engine.definition.component.ComponentDefinitionWithImplementation import pl.touk.nussknacker.engine.definition.globalvariables.GlobalVariableDefinitionWithImplementation