From 98ace545dfec93f7d1bd7d1ff3e88eb319c02e9e Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 12:11:07 +0100 Subject: [PATCH 01/48] [NU-1962] Flink test mechanism refactoring: inheritance replaced with composition --- .../process/runner/FlinkStubbedRunner.scala | 12 +++------ .../engine/process/runner/FlinkTestMain.scala | 8 +++--- .../runner/FlinkVerificationMain.scala | 8 +++--- .../management/FlinkProcessTestRunner.scala | 18 ++++++------- .../management/FlinkProcessVerifier.scala | 25 ++++++++++++------- ...er.scala => ReflectiveMethodInvoker.scala} | 6 ++--- 6 files changed, 41 insertions(+), 36 deletions(-) rename utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/{StaticMethodRunner.scala => ReflectiveMethodInvoker.scala} (73%) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index a51db93cf39..c30464fa795 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -21,15 +21,9 @@ import java.net.{MalformedURLException, URL} import scala.jdk.CollectionConverters._ import scala.util.Using -trait FlinkStubbedRunner { +final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, configuration: Configuration) { - protected def modelData: ModelData - - protected def process: CanonicalProcess - - protected def configuration: Configuration - - protected def createEnv: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment( + def createEnv: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment( MetaDataExtractor .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) .parallelism @@ -38,7 +32,7 @@ trait FlinkStubbedRunner { ) // we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... - protected def execute[T]( + def execute[T]( env: StreamExecutionEnvironment, savepointRestoreSettings: SavepointRestoreSettings ): Unit = { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala index 62e2efce685..75ba6546e70 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala @@ -53,17 +53,19 @@ class FlinkTestMain( processVersion: ProcessVersion, deploymentData: DeploymentData, val configuration: Configuration -) extends FlinkStubbedRunner { +) { + + private val stubbedRunner = new FlinkStubbedRunner(modelData, process, configuration) def runTest: TestResults[Json] = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener try { val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar(collectingListener, scenarioTestData) - val env = createEnv + val env = stubbedRunner.createEnv registrar.register(env, process, processVersion, deploymentData, resultCollector) - execute(env, SavepointRestoreSettings.none()) + stubbedRunner.execute(env, SavepointRestoreSettings.none()) collectingListener.results } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala index 3463bed6546..b5795736780 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala @@ -33,16 +33,18 @@ class FlinkVerificationMain( deploymentData: DeploymentData, savepointPath: String, val configuration: Configuration -) extends FlinkStubbedRunner { +) { + + private val stubbedRunner = new FlinkStubbedRunner(modelData, process, configuration) def runTest(): Unit = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar() - val env = createEnv + val env = stubbedRunner.createEnv registrar.register(env, process, processVersion, deploymentData, resultCollector) - execute(env, SavepointRestoreSettings.forPath(savepointPath, true)) + stubbedRunner.execute(env, SavepointRestoreSettings.forPath(savepointPath, true)) } protected def prepareRegistrar(): FlinkProcessRegistrar = { diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala index 853187e57d4..f1522ecdfe6 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala @@ -6,24 +6,24 @@ import pl.touk.nussknacker.engine.ModelData import pl.touk.nussknacker.engine.api.test.ScenarioTestData import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.util.StaticMethodRunner +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker import scala.concurrent.{ExecutionContext, Future} -class FlinkProcessTestRunner(modelData: ModelData) - extends StaticMethodRunner( - modelData.modelClassLoader.classLoader, - "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", - "run" - ) { +class FlinkProcessTestRunner(modelData: ModelData) { + + private val methodInvoker = new ReflectiveMethodInvoker[TestResults[Json]]( + modelData.modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", + "run" + ) // NU-1455: We encode variable on the engine, because of classLoader's problems def test(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData)( implicit ec: ExecutionContext ): Future[TestResults[Json]] = Future { - tryToInvoke(modelData, canonicalProcess, scenarioTestData, new Configuration()) - .asInstanceOf[TestResults[Json]] + methodInvoker.invokeStaticMethod(modelData, canonicalProcess, scenarioTestData, new Configuration()) } } diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala index 29b9226343a..82691385dc1 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala @@ -6,18 +6,18 @@ import pl.touk.nussknacker.engine.ModelData import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData -import pl.touk.nussknacker.engine.util.StaticMethodRunner +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker import scala.concurrent.Future import scala.util.control.NonFatal -class FlinkProcessVerifier(modelData: ModelData) - extends StaticMethodRunner( - modelData.modelClassLoader.classLoader, - "pl.touk.nussknacker.engine.process.runner.FlinkVerificationMain", - "run" - ) - with LazyLogging { +class FlinkProcessVerifier(modelData: ModelData) extends LazyLogging { + + private val methodInvoker = new ReflectiveMethodInvoker[Unit]( + modelData.modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.runner.FlinkVerificationMain", + "run" + ) def verify( processVersion: ProcessVersion, @@ -27,7 +27,14 @@ class FlinkProcessVerifier(modelData: ModelData) val processId = processVersion.processName try { logger.info(s"Starting to verify $processId") - tryToInvoke(modelData, canonicalProcess, processVersion, DeploymentData.empty, savepointPath, new Configuration()) + methodInvoker.invokeStaticMethod( + modelData, + canonicalProcess, + processVersion, + DeploymentData.empty, + savepointPath, + new Configuration() + ) logger.info(s"Verification of $processId successful") Future.successful(()) } catch { diff --git a/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/StaticMethodRunner.scala b/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/ReflectiveMethodInvoker.scala similarity index 73% rename from utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/StaticMethodRunner.scala rename to utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/ReflectiveMethodInvoker.scala index 5311746a49d..0e013712222 100644 --- a/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/StaticMethodRunner.scala +++ b/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/ReflectiveMethodInvoker.scala @@ -2,7 +2,7 @@ package pl.touk.nussknacker.engine.util import java.lang.reflect.InvocationTargetException -abstract class StaticMethodRunner(classLoader: ClassLoader, className: String, methodName: String) { +final class ReflectiveMethodInvoker[Result](classLoader: ClassLoader, className: String, methodName: String) { import scala.reflect.runtime.{universe => ru} @@ -17,9 +17,9 @@ abstract class StaticMethodRunner(classLoader: ClassLoader, className: String, m } // we have to use context loader, as in UI we have don't have e.g. nussknacker-process or user model on classpath... - def tryToInvoke(args: Any*): Any = ThreadUtils.withThisAsContextClassLoader(classLoader) { + def invokeStaticMethod(args: Any*): Result = ThreadUtils.withThisAsContextClassLoader(classLoader) { try { - invoker(args: _*) + invoker(args: _*).asInstanceOf[Result] } catch { case e: InvocationTargetException => throw e.getTargetException } From 107d3a240cc5b94c7ed8fe317a62f699bab80035 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 12:23:07 +0100 Subject: [PATCH 02/48] [NU-1962] Flink test mechanism refactoring: less parameters passing --- .../process/runner/FlinkStubbedRunner.scala | 53 ++++++++----------- .../engine/process/runner/FlinkTestMain.scala | 13 +++-- .../runner/FlinkVerificationMain.scala | 13 +++-- 3 files changed, 41 insertions(+), 38 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index c30464fa795..43007d5a543 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -1,39 +1,40 @@ package pl.touk.nussknacker.engine.process.runner -import org.apache.flink.configuration.{ - ConfigUtils, - Configuration, - CoreOptions, - PipelineOptions, - RestOptions, - TaskManagerOptions -} +import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.StreamMetaData -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.util.MetaDataExtractor +import pl.touk.nussknacker.engine.api.process.ProcessName +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import java.net.{MalformedURLException, URL} import scala.jdk.CollectionConverters._ import scala.util.Using -final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, configuration: Configuration) { +final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration: Configuration) { - def createEnv: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment( - MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) - .parallelism - .getOrElse(1), + def createEnv(parallelism: Int): StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment( + parallelism, configuration ) + private def createMiniCluster[T](env: StreamExecutionEnvironment, configuration: Configuration) = { + val miniCluster = new MiniCluster( + new MiniClusterConfiguration.Builder() + .setNumSlotsPerTaskManager(env.getParallelism) + .setConfiguration(configuration) + .build() + ) + miniCluster.start() + miniCluster + } + // we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... def execute[T]( env: StreamExecutionEnvironment, + parallelism: Int, + scenarioName: ProcessName, savepointRestoreSettings: SavepointRestoreSettings ): Unit = { // Checkpoints are disabled to prevent waiting for checkpoint to happen @@ -41,7 +42,7 @@ final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, env.getCheckpointConfig.disableCheckpointing() val streamGraph = env.getStreamGraph - streamGraph.setJobName(process.name.value) + streamGraph.setJobName(scenarioName.value) val jobGraph = streamGraph.getJobGraph() jobGraph.setClasspaths(classpathsFromModelWithFallbackToConfiguration) @@ -49,7 +50,7 @@ final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, val configuration: Configuration = new Configuration configuration.addAll(jobGraph.getJobConfiguration) - configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, env.getParallelism) + configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, parallelism) configuration.set[Integer](RestOptions.PORT, 0) // FIXME: reversing flink default order @@ -58,15 +59,7 @@ final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, // it is required for proper working of HadoopFileSystem FileSystem.initialize(configuration, null) - Using.resource( - new MiniCluster( - new MiniClusterConfiguration.Builder() - .setNumSlotsPerTaskManager(env.getParallelism) - .setConfiguration(configuration) - .build() - ) - ) { exec => - exec.start() + Using.resource(createMiniCluster(env, configuration)) { exec => val id = exec.submitJob(jobGraph).get().getJobID exec.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) } @@ -76,7 +69,7 @@ final class FlinkStubbedRunner(modelData: ModelData, process: CanonicalProcess, // The class is also used in some scala tests // and this fallback is to work with a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 // see details in pl.touk.nussknacker.engine.flink.test.MiniClusterExecutionEnvironment#execute - modelData.modelClassLoaderUrls match { + modelClassLoader.urls match { case Nil => ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( configuration, diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala index 75ba6546e70..183931a697f 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala @@ -4,7 +4,7 @@ import io.circe.Json import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion} +import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.api.test.ScenarioTestData import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess @@ -18,6 +18,7 @@ import pl.touk.nussknacker.engine.testmode.{ ResultsCollectingListenerHolder, TestServiceInvocationCollector } +import pl.touk.nussknacker.engine.util.MetaDataExtractor import scala.util.Using @@ -55,17 +56,21 @@ class FlinkTestMain( val configuration: Configuration ) { - private val stubbedRunner = new FlinkStubbedRunner(modelData, process, configuration) + private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) def runTest: TestResults[Json] = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener try { val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar(collectingListener, scenarioTestData) - val env = stubbedRunner.createEnv + val parallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + val env = stubbedRunner.createEnv(parallelism) registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute(env, SavepointRestoreSettings.none()) + stubbedRunner.execute(env, parallelism, process.name, SavepointRestoreSettings.none()) collectingListener.results } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala index b5795736780..49a73900af6 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala @@ -3,13 +3,14 @@ package pl.touk.nussknacker.engine.process.runner import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.ProcessVersion +import pl.touk.nussknacker.engine.api.{ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.process.compiler.VerificationFlinkProcessCompilerDataFactory import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestRunId, TestServiceInvocationCollector} +import pl.touk.nussknacker.engine.util.MetaDataExtractor object FlinkVerificationMain extends FlinkRunner { @@ -35,16 +36,20 @@ class FlinkVerificationMain( val configuration: Configuration ) { - private val stubbedRunner = new FlinkStubbedRunner(modelData, process, configuration) + private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) def runTest(): Unit = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar() - val env = stubbedRunner.createEnv + val parallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + val env = stubbedRunner.createEnv(parallelism) registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute(env, SavepointRestoreSettings.forPath(savepointPath, true)) + stubbedRunner.execute(env, parallelism, process.name, SavepointRestoreSettings.forPath(savepointPath, true)) } protected def prepareRegistrar(): FlinkProcessRegistrar = { From 65044813d9275111d08493094a9d5ed5828f28cb Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 12:48:03 +0100 Subject: [PATCH 03/48] [NU-1962] Flink test mechanism refactoring: execute method splitted into logical parts --- .../process/runner/FlinkStubbedRunner.scala | 64 ++++++++++++------- 1 file changed, 40 insertions(+), 24 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index 43007d5a543..733a5ae2626 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -2,9 +2,10 @@ package pl.touk.nussknacker.engine.process.runner import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.graph.StreamGraph import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.util.loader.ModelClassLoader @@ -14,10 +15,16 @@ import scala.util.Using final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration: Configuration) { - def createEnv(parallelism: Int): StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment( - parallelism, - configuration - ) + def createEnv(parallelism: Int): StreamExecutionEnvironment = { + val env = StreamExecutionEnvironment.createLocalEnvironment( + parallelism, + configuration + ) + // Checkpoints are disabled to prevent waiting for checkpoint to happen + // before finishing execution. + env.getCheckpointConfig.disableCheckpointing() + env + } private def createMiniCluster[T](env: StreamExecutionEnvironment, configuration: Configuration) = { val miniCluster = new MiniCluster( @@ -37,34 +44,32 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration scenarioName: ProcessName, savepointRestoreSettings: SavepointRestoreSettings ): Unit = { - // Checkpoints are disabled to prevent waiting for checkpoint to happen - // before finishing execution. - env.getCheckpointConfig.disableCheckpointing() - val streamGraph = env.getStreamGraph - streamGraph.setJobName(scenarioName.value) - - val jobGraph = streamGraph.getJobGraph() - jobGraph.setClasspaths(classpathsFromModelWithFallbackToConfiguration) - jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) + setupStreamGraph(streamGraph, scenarioName, savepointRestoreSettings) - val configuration: Configuration = new Configuration - configuration.addAll(jobGraph.getJobConfiguration) - configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, parallelism) - configuration.set[Integer](RestOptions.PORT, 0) - - // FIXME: reversing flink default order - configuration.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") + val miniClusterConfiguration = prepareMiniClusterConfiguration(parallelism, streamGraph) // it is required for proper working of HadoopFileSystem - FileSystem.initialize(configuration, null) + FileSystem.initialize(miniClusterConfiguration, null) - Using.resource(createMiniCluster(env, configuration)) { exec => - val id = exec.submitJob(jobGraph).get().getJobID + Using.resource(createMiniCluster(env, miniClusterConfiguration)) { exec => + val id = exec.submitJob(streamGraph.getJobGraph).get().getJobID exec.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) } } + private def setupStreamGraph( + streamGraph: StreamGraph, + scenarioName: ProcessName, + savepointRestoreSettings: SavepointRestoreSettings + ): Unit = { + streamGraph.setJobName(scenarioName.value) + + val jobGraph = streamGraph.getJobGraph() + jobGraph.setClasspaths(classpathsFromModelWithFallbackToConfiguration) + jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) + } + private def classpathsFromModelWithFallbackToConfiguration = { // The class is also used in some scala tests // and this fallback is to work with a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 @@ -80,4 +85,15 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration } } + private def prepareMiniClusterConfiguration[T](parallelism: Int, streamGraph: StreamGraph) = { + val configuration: Configuration = new Configuration + configuration.addAll(streamGraph.getJobGraph.getJobConfiguration) + configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, parallelism) + configuration.set[Integer](RestOptions.PORT, 0) + + // FIXME: reversing flink default order + configuration.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") + configuration + } + } From e2cb08a2ecdf89b1da1df36f381b9437f60b9cee Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 14:54:02 +0100 Subject: [PATCH 04/48] [NU-1962] Flink test mechanism refactoring: passing the same jobgraph as mutated --- .../process/runner/FlinkStubbedRunner.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index 733a5ae2626..d1f46ede633 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -45,27 +45,25 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration savepointRestoreSettings: SavepointRestoreSettings ): Unit = { val streamGraph = env.getStreamGraph - setupStreamGraph(streamGraph, scenarioName, savepointRestoreSettings) + streamGraph.setJobName(scenarioName.value) + val jobGraph = streamGraph.getJobGraph + setupJobGraph(jobGraph, savepointRestoreSettings) - val miniClusterConfiguration = prepareMiniClusterConfiguration(parallelism, streamGraph) + val miniClusterConfiguration = prepareMiniClusterConfiguration(parallelism, jobGraph) // it is required for proper working of HadoopFileSystem FileSystem.initialize(miniClusterConfiguration, null) Using.resource(createMiniCluster(env, miniClusterConfiguration)) { exec => - val id = exec.submitJob(streamGraph.getJobGraph).get().getJobID + val id = exec.submitJob(jobGraph).get().getJobID exec.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) } } - private def setupStreamGraph( - streamGraph: StreamGraph, - scenarioName: ProcessName, + private def setupJobGraph( + jobGraph: JobGraph, savepointRestoreSettings: SavepointRestoreSettings ): Unit = { - streamGraph.setJobName(scenarioName.value) - - val jobGraph = streamGraph.getJobGraph() jobGraph.setClasspaths(classpathsFromModelWithFallbackToConfiguration) jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) } @@ -85,9 +83,9 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration } } - private def prepareMiniClusterConfiguration[T](parallelism: Int, streamGraph: StreamGraph) = { + private def prepareMiniClusterConfiguration[T](parallelism: Int, jobGraph: JobGraph) = { val configuration: Configuration = new Configuration - configuration.addAll(streamGraph.getJobGraph.getJobConfiguration) + configuration.addAll(jobGraph.getJobConfiguration) configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, parallelism) configuration.set[Integer](RestOptions.PORT, 0) From 42e2698d7b24a43647c3e7182e910ffa6541d234 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 15:03:47 +0100 Subject: [PATCH 05/48] [NU-1962] Flink test mechanism refactoring: temporary removed fallback for empty classpath --- .../engine/process/runner/FlinkStubbedRunner.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index d1f46ede633..f939a9da9e7 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -73,12 +73,13 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration // and this fallback is to work with a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 // see details in pl.touk.nussknacker.engine.flink.test.MiniClusterExecutionEnvironment#execute modelClassLoader.urls match { - case Nil => - ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( - configuration, - PipelineOptions.CLASSPATHS, - new URL(_) - ) + // FIXME abr: is it necessary? +// case Nil => +// ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( +// configuration, +// PipelineOptions.CLASSPATHS, +// new URL(_) +// ) case list => list.asJava } } From dc816961f98f754dd371430f1ebd8268c763f8b8 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 15:19:51 +0100 Subject: [PATCH 06/48] [NU-1962] Flink test mechanism refactoring: more cleanups --- .../process/runner/FlinkStubbedRunner.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index f939a9da9e7..5a1556e2ff6 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -5,11 +5,9 @@ import org.apache.flink.core.fs.FileSystem import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import org.apache.flink.streaming.api.graph.StreamGraph import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.util.loader.ModelClassLoader -import java.net.{MalformedURLException, URL} import scala.jdk.CollectionConverters._ import scala.util.Using @@ -26,10 +24,10 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration env } - private def createMiniCluster[T](env: StreamExecutionEnvironment, configuration: Configuration) = { + private def createMiniCluster(configuration: Configuration, numSlotsPerTaskManager: Int) = { val miniCluster = new MiniCluster( new MiniClusterConfiguration.Builder() - .setNumSlotsPerTaskManager(env.getParallelism) + .setNumSlotsPerTaskManager(numSlotsPerTaskManager) .setConfiguration(configuration) .build() ) @@ -38,7 +36,7 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration } // we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... - def execute[T]( + def execute( env: StreamExecutionEnvironment, parallelism: Int, scenarioName: ProcessName, @@ -49,14 +47,16 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration val jobGraph = streamGraph.getJobGraph setupJobGraph(jobGraph, savepointRestoreSettings) - val miniClusterConfiguration = prepareMiniClusterConfiguration(parallelism, jobGraph) + val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = parallelism, jobGraph) // it is required for proper working of HadoopFileSystem FileSystem.initialize(miniClusterConfiguration, null) - Using.resource(createMiniCluster(env, miniClusterConfiguration)) { exec => - val id = exec.submitJob(jobGraph).get().getJobID - exec.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) + val miniCluster = createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = parallelism) + + Using.resource(miniCluster) { miniCluster => + val id = miniCluster.submitJob(jobGraph).get().getJobID + miniCluster.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) } } @@ -84,10 +84,10 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration } } - private def prepareMiniClusterConfiguration[T](parallelism: Int, jobGraph: JobGraph) = { + private def prepareMiniClusterConfiguration(numTaskSlots: Int, jobGraph: JobGraph) = { val configuration: Configuration = new Configuration configuration.addAll(jobGraph.getJobConfiguration) - configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, parallelism) + configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, numTaskSlots) configuration.set[Integer](RestOptions.PORT, 0) // FIXME: reversing flink default order From 9e14ab95ba3dca86a1a4fe612e0d4353dcab0e1d Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 15:29:03 +0100 Subject: [PATCH 07/48] [NU-1962] Flink test mechanism refactoring: temporary removed passing jobGraph.getJobConfiguration to mini cluster --- .../nussknacker/engine/process/runner/FlinkStubbedRunner.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index 5a1556e2ff6..59f8b3056f7 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -86,7 +86,8 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration private def prepareMiniClusterConfiguration(numTaskSlots: Int, jobGraph: JobGraph) = { val configuration: Configuration = new Configuration - configuration.addAll(jobGraph.getJobConfiguration) + // FIXME abr: is it necessary? +// configuration.addAll(jobGraph.getJobConfiguration) configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, numTaskSlots) configuration.set[Integer](RestOptions.PORT, 0) From 726c789d86e85bdc3d5304d81c25e132b0293d5c Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 16 Jan 2025 15:54:21 +0100 Subject: [PATCH 08/48] [NU-1962] Flink test mechanism refactoring: classpath trick moved from production code to test code --- .../engine/process/runner/FlinkTestMain.scala | 10 ++++------ .../engine/process/runner/FlinkVerificationMain.scala | 6 +++--- .../engine/kafka/source/flink/TestFromFileSpec.scala | 10 +++++++++- .../engine/flink/test/FlinkTestConfiguration.scala | 7 ------- .../engine/util/loader/ModelClassLoader.scala | 2 +- 5 files changed, 17 insertions(+), 18 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala index 183931a697f..93acbb61921 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala @@ -4,9 +4,9 @@ import io.circe.Json import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.api.test.ScenarioTestData +import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{AdditionalModelConfigs, DeploymentData} import pl.touk.nussknacker.engine.process.compiler.TestFlinkProcessCompilerDataFactory @@ -20,8 +20,6 @@ import pl.touk.nussknacker.engine.testmode.{ } import pl.touk.nussknacker.engine.util.MetaDataExtractor -import scala.util.Using - object FlinkTestMain extends FlinkRunner { def run( @@ -48,12 +46,12 @@ object FlinkTestMain extends FlinkRunner { } class FlinkTestMain( - val modelData: ModelData, - val process: CanonicalProcess, + modelData: ModelData, + process: CanonicalProcess, scenarioTestData: ScenarioTestData, processVersion: ProcessVersion, deploymentData: DeploymentData, - val configuration: Configuration + configuration: Configuration ) { private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala index 49a73900af6..f1dfc19fcff 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala @@ -28,12 +28,12 @@ object FlinkVerificationMain extends FlinkRunner { } class FlinkVerificationMain( - val modelData: ModelData, - val process: CanonicalProcess, + modelData: ModelData, + process: CanonicalProcess, processVersion: ProcessVersion, deploymentData: DeploymentData, savepointPath: String, - val configuration: Configuration + configuration: Configuration ) { private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) diff --git a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala b/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala index 136e4601b73..615d0000a44 100644 --- a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala +++ b/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala @@ -24,8 +24,10 @@ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.util.json.ToJsonEncoder +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} +import java.net.URL import java.util.Collections class TestFromFileSpec @@ -48,7 +50,13 @@ class TestFromFileSpec LocalModelData( inputConfig = config, components = List.empty, - configCreator = new KafkaSourceFactoryProcessConfigCreator(() => TestFromFileSpec.resultsHolders) + configCreator = new KafkaSourceFactoryProcessConfigCreator(() => TestFromFileSpec.resultsHolders), + // This is a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 + // Flink overwrite user classloader by the AppClassLoader if classpaths parameter is empty + // (implementation in org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager) + // which holds all needed jars/classes in case of running from Scala plugin in IDE. + // but in case of running from sbt it contains only sbt-launcher.jar + modelClassLoader = new ModelClassLoader(getClass.getClassLoader, List(new URL("http://dummy-classpath.invalid"))) ) test("Should pass correct timestamp from test data") { diff --git a/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala b/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala index 4335d07293e..48b05889f6d 100644 --- a/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala +++ b/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala @@ -16,13 +16,6 @@ object FlinkTestConfiguration { config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("16m")) config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("16m")) - // This is a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 - // Flink overwrite user classloader by the AppClassLoader if classpaths parameter is empty - // (implementation in org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager) - // which holds all needed jars/classes in case of running from Scala plugin in IDE. - // but in case of running from sbt it contains only sbt-launcher.jar - config.set(PipelineOptions.CLASSPATHS, List("http://dummy-classpath.invalid").asJava) - // This is to prevent memory problem in tests with mutliple Table API based aggregations. An IllegalArgExceptionon // is thrown with message "The minBucketMemorySize is not valid!" in // org.apache.flink.table.runtime.util.collections.binary.AbstractBytesHashMap.java:121 where memorySize is set diff --git a/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/loader/ModelClassLoader.scala b/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/loader/ModelClassLoader.scala index 470d9190cac..9f8902685db 100644 --- a/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/loader/ModelClassLoader.scala +++ b/utils/utils-internal/src/main/scala/pl/touk/nussknacker/engine/util/loader/ModelClassLoader.scala @@ -6,7 +6,7 @@ import java.io.File import java.net.{URI, URL, URLClassLoader} import java.nio.file.Path -case class ModelClassLoader private (classLoader: ClassLoader, urls: List[URL]) { +case class ModelClassLoader(classLoader: ClassLoader, urls: List[URL]) { override def toString: String = s"ModelClassLoader(${toString(classLoader)})" From 7d7836c18d2567920f712cd7828c9bcbd5193187 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 17 Jan 2025 10:46:23 +0100 Subject: [PATCH 09/48] tests fix attempt --- .../engine/process/runner/FlinkStubbedRunner.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index 59f8b3056f7..295d0a70daf 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -8,6 +8,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.util.loader.ModelClassLoader +import java.net.{MalformedURLException, URL} import scala.jdk.CollectionConverters._ import scala.util.Using @@ -74,12 +75,12 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration // see details in pl.touk.nussknacker.engine.flink.test.MiniClusterExecutionEnvironment#execute modelClassLoader.urls match { // FIXME abr: is it necessary? -// case Nil => -// ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( -// configuration, -// PipelineOptions.CLASSPATHS, -// new URL(_) -// ) + case Nil => + ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( + configuration, + PipelineOptions.CLASSPATHS, + new URL(_) + ) case list => list.asJava } } From 3ac7e6e741af20eb213ec3a5eb33cadcc8dddeac Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 17 Jan 2025 11:27:27 +0100 Subject: [PATCH 10/48] referted classpath fix in test classes --- .../process/runner/FlinkStubbedRunner.scala | 10 +++++----- .../process/runner/FlinkTestMainSpec.scala | 3 ++- .../kafka/source/flink/TestFromFileSpec.scala | 10 +++------- .../source/flink/TestWithTestDataSpec.scala | 12 +++++++++-- .../flink/test/FlinkTestConfiguration.scala | 20 ++++++++++++++++++- 5 files changed, 39 insertions(+), 16 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index 295d0a70daf..a68ba94f844 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -76,11 +76,11 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration modelClassLoader.urls match { // FIXME abr: is it necessary? case Nil => - ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( - configuration, - PipelineOptions.CLASSPATHS, - new URL(_) - ) +// ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( +// configuration, +// PipelineOptions.CLASSPATHS, +// new URL(_) +// ) case list => list.asJava } } diff --git a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala index 13db1d66a46..187ba78fa46 100644 --- a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala +++ b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala @@ -2,6 +2,7 @@ package pl.touk.nussknacker.engine.process.runner import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} import io.circe.Json +import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.client.JobExecutionException import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec @@ -770,7 +771,7 @@ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with Befor ModelConfigs(config, AdditionalModelConfigs(additionalConfigsFromProvider)) ) ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - FlinkTestMain.run(modelData, process, scenarioTestData, FlinkTestConfiguration.configuration()) + FlinkTestMain.run(modelData, process, scenarioTestData, FlinkTestConfiguration.setupMemory(new Configuration)) } } diff --git a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala b/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala index 615d0000a44..19ffbbfacc7 100644 --- a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala +++ b/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala @@ -5,6 +5,7 @@ import com.typesafe.config.ConfigValueFactory.fromAnyRef import com.typesafe.scalalogging.LazyLogging import io.circe.Json import io.circe.Json.{Null, fromString, obj} +import org.apache.flink.configuration.Configuration import org.apache.kafka.common.record.TimestampType import org.scalatest.OptionValues import org.scalatest.funsuite.AnyFunSuite @@ -51,12 +52,7 @@ class TestFromFileSpec inputConfig = config, components = List.empty, configCreator = new KafkaSourceFactoryProcessConfigCreator(() => TestFromFileSpec.resultsHolders), - // This is a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 - // Flink overwrite user classloader by the AppClassLoader if classpaths parameter is empty - // (implementation in org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager) - // which holds all needed jars/classes in case of running from Scala plugin in IDE. - // but in case of running from sbt it contains only sbt-launcher.jar - modelClassLoader = new ModelClassLoader(getClass.getClassLoader, List(new URL("http://dummy-classpath.invalid"))) + modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) ) test("Should pass correct timestamp from test data") { @@ -141,7 +137,7 @@ class TestFromFileSpec modelData, process, scenarioTestData, - FlinkTestConfiguration.configuration(), + FlinkTestConfiguration.setupMemory(new Configuration), ) } } diff --git a/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala b/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala index 9ca30564187..0e9f4f160cb 100644 --- a/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala +++ b/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala @@ -6,6 +6,7 @@ import com.typesafe.scalalogging.LazyLogging import io.circe.Json import io.circe.Json._ import org.apache.avro.Schema +import org.apache.flink.configuration.Configuration import org.apache.kafka.common.record.TimestampType import org.scalatest.{LoneElement, OptionValues} import org.scalatest.funsuite.AnyFunSuite @@ -37,8 +38,10 @@ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.testmode.TestProcess._ import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.util.json.ToJsonEncoder +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} +import java.net.URL import java.util.Collections class TestWithTestDataSpec @@ -200,10 +203,15 @@ class TestWithTestDataSpec private def run(process: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { FlinkTestMain.run( - LocalModelData(config, List.empty, configCreator = creator), + LocalModelData( + config, + List.empty, + configCreator = creator, + modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) + ), process, scenarioTestData, - FlinkTestConfiguration.configuration(), + FlinkTestConfiguration.setupMemory(new Configuration), ) } } diff --git a/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala b/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala index 48b05889f6d..6c92d6e70fd 100644 --- a/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala +++ b/engine/flink/test-utils/src/main/scala/pl/touk/nussknacker/engine/flink/test/FlinkTestConfiguration.scala @@ -3,6 +3,8 @@ package pl.touk.nussknacker.engine.flink.test import com.github.ghik.silencer.silent import org.apache.flink.configuration._ +import java.net.URL + object FlinkTestConfiguration { // better to create each time because is mutable @@ -12,10 +14,26 @@ object FlinkTestConfiguration { val config = new Configuration config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, taskManagersCount) config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, taskSlotsCount) + + config.set(PipelineOptions.CLASSPATHS, classpathWorkaround.map(_.toString).asJava) + + setupMemory(config) + } + + // FIXME: better describe which classpath is used in this case + // This is a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 + // Flink overwrite user classloader by the AppClassLoader if classpaths parameter is empty + // (implementation in org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager) + // which holds all needed jars/classes in case of running from Scala plugin in IDE. + // but in case of running from sbt it contains only sbt-launcher.jar + def classpathWorkaround: List[URL] = { + List(new URL("http://dummy-classpath.invalid")) + } + + def setupMemory(config: Configuration): Configuration = { // to prevent OutOfMemoryError: Could not allocate enough memory segments for NetworkBufferPool on low memory env (like Travis) config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("16m")) config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("16m")) - // This is to prevent memory problem in tests with mutliple Table API based aggregations. An IllegalArgExceptionon // is thrown with message "The minBucketMemorySize is not valid!" in // org.apache.flink.table.runtime.util.collections.binary.AbstractBytesHashMap.java:121 where memorySize is set From a2f1899df90e9027347463dda09f9bd91d212b21 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 17 Jan 2025 11:37:23 +0100 Subject: [PATCH 11/48] wip --- .../engine/process/runner/FlinkStubbedRunner.scala | 2 +- .../engine/process/runner/FlinkTestMainSpec.scala | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala index a68ba94f844..77b0e19e2af 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala @@ -75,7 +75,7 @@ final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration // see details in pl.touk.nussknacker.engine.flink.test.MiniClusterExecutionEnvironment#execute modelClassLoader.urls match { // FIXME abr: is it necessary? - case Nil => +// case Nil => // ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( // configuration, // PipelineOptions.CLASSPATHS, diff --git a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala index 187ba78fa46..fc4f607876f 100644 --- a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala +++ b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala @@ -41,11 +41,14 @@ import pl.touk.nussknacker.engine.testmode.TestProcess._ import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} import pl.touk.nussknacker.engine.deployment.AdditionalModelConfigs +import pl.touk.nussknacker.engine.testing.LocalModelData +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import java.util.{Date, UUID} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.concurrent.{Await, Future} +import scala.jdk.CollectionConverters._ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with BeforeAndAfterEach with OptionValues { @@ -767,8 +770,10 @@ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with Befor .withValue("globalParameters.useIOMonadInInterpreter", ConfigValueFactory.fromAnyRef(useIOMonadInInterpreter)) // We need to set context loader to avoid forking in sbt - val modelData = ModelData.duringFlinkExecution( - ModelConfigs(config, AdditionalModelConfigs(additionalConfigsFromProvider)) + val modelData = ModelData.duringExecution( + ModelConfigs(config, AdditionalModelConfigs(additionalConfigsFromProvider)), + ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround), + resolveConfigs = false ) ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { FlinkTestMain.run(modelData, process, scenarioTestData, FlinkTestConfiguration.setupMemory(new Configuration)) From 8118f735bf8c1ca083bff3c86ee2a8456132bbec Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 17 Jan 2025 18:12:52 +0100 Subject: [PATCH 12/48] StreamExecutionEnvironment and MiniCluster created once --- build.sbt | 31 +++--- ...DevelopmentDeploymentManagerProvider.scala | 12 ++- .../MockableDeploymentManagerProvider.scala | 11 +- .../process/runner/FlinkStubbedRunner.scala | 100 ------------------ .../engine/process/runner/FlinkTestMain.scala | 31 +++--- .../runner/FlinkVerificationMain.scala | 43 +++++--- .../testmechanism/FlinkStubbedRunner.scala | 38 +++++++ .../FlinkStreamingProcessMainSpec.scala | 56 ---------- .../runner/SimpleProcessConfigCreator.scala | 59 +++++++++++ .../management/FlinkDeploymentManager.scala | 7 +- .../management/FlinkProcessTestRunner.scala | 29 ----- .../FlinkProcessTestRunner.scala | 45 ++++++++ .../FlinkProcessVerifier.scala | 38 +++++-- .../TestsMechanismMiniClusterFactory.scala | 39 +++++++ ...ismStreamExecutionEnvironmentFactory.scala | 19 ++++ ...er.engine.api.process.ProcessConfigCreator | 1 + .../FlinkProcessTestRunnerSpec.scala} | 53 ++++------ .../SimpleProcessConfigCreator.scala | 59 +++++++++++ .../kafka}/TestFromFileSpec.scala | 15 ++- .../schemedkafka}/TestWithTestDataSpec.scala | 28 +++-- 20 files changed, 409 insertions(+), 305 deletions(-) delete mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala create mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala create mode 100644 engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/SimpleProcessConfigCreator.scala delete mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala rename engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/{ => testsmechanism}/FlinkProcessVerifier.scala (52%) create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala create mode 100644 engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator rename engine/flink/{executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala => management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala} (96%) create mode 100644 engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala rename engine/flink/{kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink => tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka}/TestFromFileSpec.scala (92%) rename engine/flink/{schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink => tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka}/TestWithTestDataSpec.scala (91%) diff --git a/build.sbt b/build.sbt index 388e6e75faa..69f705a93b4 100644 --- a/build.sbt +++ b/build.sbt @@ -632,7 +632,8 @@ lazy val flinkDeploymentManager = (project in flink("management")) componentsApi % Provided, httpUtils % Provided, flinkScalaUtils % Provided, - flinkTestUtils % IntegrationTest, + flinkExecutor % Test, + flinkTestUtils % "it,test", kafkaTestUtils % "it,test" ) @@ -742,18 +743,22 @@ lazy val flinkTests = (project in flink("tests")) } ) .dependsOn( - defaultModel % Test, - flinkExecutor % Test, - flinkKafkaComponents % Test, - flinkBaseComponents % Test, - flinkBaseUnboundedComponents % Test, - flinkTableApiComponents % Test, - flinkTestUtils % Test, - kafkaTestUtils % Test, - flinkComponentsTestkit % Test, + defaultModel % Test, + flinkExecutor % Test, + flinkKafkaComponents % Test, + flinkBaseComponents % Test, + flinkBaseUnboundedComponents % Test, + flinkTableApiComponents % Test, + flinkTestUtils % Test, + kafkaTestUtils % Test, + flinkComponentsTestkit % Test, + flinkDeploymentManager % Test, + // TODO: cleanup kafka testsmechanism tests in order to remove test->test dependency + flinkKafkaComponentsUtils % "test->test", + flinkSchemedKafkaComponentsUtils % "test->test", // for local development - designer % Test, - deploymentManagerApi % Test + designer % Test, + deploymentManagerApi % Test ) lazy val defaultModel = (project in (file("defaultModel"))) @@ -997,7 +1002,7 @@ lazy val flinkSchemedKafkaComponentsUtils = (project in flink("schemed-kafka-com componentsUtils % Provided, kafkaTestUtils % Test, flinkTestUtils % Test, - flinkExecutor % Test + flinkExecutor % Test, ) lazy val flinkKafkaComponentsUtils = (project in flink("kafka-components-utils")) diff --git a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala index 7a247f11d78..e7fe3ac2173 100644 --- a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala +++ b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala @@ -4,6 +4,7 @@ import akka.actor.ActorSystem import cats.data.{Validated, ValidatedNel} import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging +import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.development.manager.DevelopmentStateStatus._ import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api.ProcessVersion @@ -19,7 +20,8 @@ import pl.touk.nussknacker.engine.api.deployment.simple.{SimpleProcessStateDefin import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment._ -import pl.touk.nussknacker.engine.management.{FlinkProcessTestRunner, FlinkStreamingPropertiesConfig} +import pl.touk.nussknacker.engine.management.FlinkStreamingPropertiesConfig +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner import java.net.URI import java.util.UUID @@ -48,7 +50,8 @@ class DevelopmentDeploymentManager(actorSystem: ActorSystem, modelData: BaseMode private val memory: TrieMap[ProcessName, StatusDetails] = TrieMap[ProcessName, StatusDetails]() private val random = new scala.util.Random() - private lazy val flinkTestRunner = new FlinkProcessTestRunner(modelData.asInvokableModelData) + private lazy val flinkTestRunner = + new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration()) implicit private class ProcessStateExpandable(processState: StatusDetails) { @@ -83,7 +86,10 @@ class DevelopmentDeploymentManager(actorSystem: ActorSystem, modelData: BaseMode case command: DMRunOffScheduleCommand => runOffSchedule(command) case _: DMMakeScenarioSavepointCommand => Future.successful(SavepointResult("")) case DMTestScenarioCommand(_, canonicalProcess, scenarioTestData) => - flinkTestRunner.test(canonicalProcess, scenarioTestData) // it's just for streaming e2e tests from file purposes + flinkTestRunner.runTestsAsync( + canonicalProcess, + scenarioTestData + ) // it's just for streaming e2e tests from file purposes } private def description(canonicalProcess: CanonicalProcess) = { diff --git a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala index 7627c005fb7..7110bd056df 100644 --- a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala +++ b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala @@ -4,16 +4,17 @@ import cats.data.Validated.valid import cats.data.ValidatedNel import com.typesafe.config.Config import io.circe.Json +import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.development.manager.MockableDeploymentManagerProvider.MockableDeploymentManager import pl.touk.nussknacker.engine.ModelData.BaseModelDataExt import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api.component.ScenarioPropertyConfig -import pl.touk.nussknacker.engine.api.definition.{NotBlankParameterValidator, StringParameterEditor} import pl.touk.nussknacker.engine.api.deployment._ import pl.touk.nussknacker.engine.api.deployment.simple.{SimpleProcessStateDefinitionManager, SimpleStateStatus} import pl.touk.nussknacker.engine.api.process.{ProcessIdWithName, ProcessName, VersionId} import pl.touk.nussknacker.engine.deployment.ExternalDeploymentId -import pl.touk.nussknacker.engine.management.{FlinkProcessTestRunner, FlinkStreamingPropertiesConfig} +import pl.touk.nussknacker.engine.management.FlinkStreamingPropertiesConfig +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner import pl.touk.nussknacker.engine.newdeployment.DeploymentId import pl.touk.nussknacker.engine.testing.StubbingCommands import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults @@ -59,7 +60,9 @@ object MockableDeploymentManagerProvider { with StubbingCommands { private lazy val testRunnerOpt = - modelDataOpt.map(modelData => new FlinkProcessTestRunner(modelData.asInvokableModelData)) + modelDataOpt.map(modelData => + new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration()) + ) override def resolve( idWithName: ProcessIdWithName, @@ -102,7 +105,7 @@ object MockableDeploymentManagerProvider { .get() .get(processVersion.processName.value) .map(Future.successful) - .orElse(testRunnerOpt.map(_.test(scenario, testData))) + .orElse(testRunnerOpt.map(_.runTestsAsync(scenario, testData))) .getOrElse( throw new IllegalArgumentException( s"Tests results not mocked for scenario [${processVersion.processName.value}] and no model data provided" diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala deleted file mode 100644 index 77b0e19e2af..00000000000 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkStubbedRunner.scala +++ /dev/null @@ -1,100 +0,0 @@ -package pl.touk.nussknacker.engine.process.runner - -import org.apache.flink.configuration._ -import org.apache.flink.core.fs.FileSystem -import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} -import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.api.process.ProcessName -import pl.touk.nussknacker.engine.util.loader.ModelClassLoader - -import java.net.{MalformedURLException, URL} -import scala.jdk.CollectionConverters._ -import scala.util.Using - -final class FlinkStubbedRunner(modelClassLoader: ModelClassLoader, configuration: Configuration) { - - def createEnv(parallelism: Int): StreamExecutionEnvironment = { - val env = StreamExecutionEnvironment.createLocalEnvironment( - parallelism, - configuration - ) - // Checkpoints are disabled to prevent waiting for checkpoint to happen - // before finishing execution. - env.getCheckpointConfig.disableCheckpointing() - env - } - - private def createMiniCluster(configuration: Configuration, numSlotsPerTaskManager: Int) = { - val miniCluster = new MiniCluster( - new MiniClusterConfiguration.Builder() - .setNumSlotsPerTaskManager(numSlotsPerTaskManager) - .setConfiguration(configuration) - .build() - ) - miniCluster.start() - miniCluster - } - - // we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... - def execute( - env: StreamExecutionEnvironment, - parallelism: Int, - scenarioName: ProcessName, - savepointRestoreSettings: SavepointRestoreSettings - ): Unit = { - val streamGraph = env.getStreamGraph - streamGraph.setJobName(scenarioName.value) - val jobGraph = streamGraph.getJobGraph - setupJobGraph(jobGraph, savepointRestoreSettings) - - val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = parallelism, jobGraph) - - // it is required for proper working of HadoopFileSystem - FileSystem.initialize(miniClusterConfiguration, null) - - val miniCluster = createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = parallelism) - - Using.resource(miniCluster) { miniCluster => - val id = miniCluster.submitJob(jobGraph).get().getJobID - miniCluster.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) - } - } - - private def setupJobGraph( - jobGraph: JobGraph, - savepointRestoreSettings: SavepointRestoreSettings - ): Unit = { - jobGraph.setClasspaths(classpathsFromModelWithFallbackToConfiguration) - jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) - } - - private def classpathsFromModelWithFallbackToConfiguration = { - // The class is also used in some scala tests - // and this fallback is to work with a work around for a behaviour added in https://issues.apache.org/jira/browse/FLINK-32265 - // see details in pl.touk.nussknacker.engine.flink.test.MiniClusterExecutionEnvironment#execute - modelClassLoader.urls match { - // FIXME abr: is it necessary? -// case Nil => -// ConfigUtils.decodeListFromConfig[String, URL, MalformedURLException]( -// configuration, -// PipelineOptions.CLASSPATHS, -// new URL(_) -// ) - case list => list.asJava - } - } - - private def prepareMiniClusterConfiguration(numTaskSlots: Int, jobGraph: JobGraph) = { - val configuration: Configuration = new Configuration - // FIXME abr: is it necessary? -// configuration.addAll(jobGraph.getJobConfiguration) - configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, numTaskSlots) - configuration.set[Integer](RestOptions.PORT, 0) - - // FIXME: reversing flink default order - configuration.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") - configuration - } - -} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala index 93acbb61921..eee9b7afb2e 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala @@ -1,16 +1,18 @@ package pl.touk.nussknacker.engine.process.runner import io.circe.Json -import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import org.apache.flink.runtime.minicluster.MiniCluster +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import pl.touk.nussknacker.engine.ModelData import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.api.test.ScenarioTestData -import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion, StreamMetaData} +import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{AdditionalModelConfigs, DeploymentData} import pl.touk.nussknacker.engine.process.compiler.TestFlinkProcessCompilerDataFactory import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar +import pl.touk.nussknacker.engine.process.testmechanism.FlinkStubbedRunner import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults import pl.touk.nussknacker.engine.testmode.{ @@ -18,57 +20,54 @@ import pl.touk.nussknacker.engine.testmode.{ ResultsCollectingListenerHolder, TestServiceInvocationCollector } -import pl.touk.nussknacker.engine.util.MetaDataExtractor object FlinkTestMain extends FlinkRunner { def run( + miniCluster: MiniCluster, + env: StreamExecutionEnvironment, modelData: ModelData, process: CanonicalProcess, - scenarioTestData: ScenarioTestData, - configuration: Configuration, + scenarioTestData: ScenarioTestData ): TestResults[Json] = { val processVersion = ProcessVersion.empty.copy(processName = ProcessName("snapshot version") ) // testing process may be unreleased, so it has no version new FlinkTestMain( + miniCluster, + env, modelData, process, scenarioTestData, processVersion, DeploymentData.empty.copy(additionalModelConfigs = AdditionalModelConfigs(modelData.additionalConfigsFromProvider) - ), - configuration + ) ).runTest } } class FlinkTestMain( + miniCluster: MiniCluster, + env: StreamExecutionEnvironment, modelData: ModelData, process: CanonicalProcess, scenarioTestData: ScenarioTestData, processVersion: ProcessVersion, - deploymentData: DeploymentData, - configuration: Configuration + deploymentData: DeploymentData ) { - private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) + private val stubbedRunner = new FlinkStubbedRunner(miniCluster, env) def runTest: TestResults[Json] = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener try { val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar(collectingListener, scenarioTestData) - val parallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - val env = stubbedRunner.createEnv(parallelism) registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute(env, parallelism, process.name, SavepointRestoreSettings.none()) + stubbedRunner.execute(process.name, SavepointRestoreSettings.none(), modelData.modelClassLoader) collectingListener.results } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala index f1dfc19fcff..86147380cda 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala @@ -1,55 +1,64 @@ package pl.touk.nussknacker.engine.process.runner -import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import org.apache.flink.runtime.minicluster.MiniCluster +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.{ProcessVersion, StreamMetaData} +import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.process.compiler.VerificationFlinkProcessCompilerDataFactory import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar +import pl.touk.nussknacker.engine.process.testmechanism.FlinkStubbedRunner import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} -import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestRunId, TestServiceInvocationCollector} -import pl.touk.nussknacker.engine.util.MetaDataExtractor +import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestServiceInvocationCollector} object FlinkVerificationMain extends FlinkRunner { def run( + miniCluster: MiniCluster, + env: StreamExecutionEnvironment, modelData: ModelData, process: CanonicalProcess, processVersion: ProcessVersion, deploymentData: DeploymentData, - savepointPath: String, - configuration: Configuration + savepointPath: String ): Unit = - new FlinkVerificationMain(modelData, process, processVersion, deploymentData, savepointPath, configuration) - .runTest() + new FlinkVerificationMain( + miniCluster, + env, + modelData, + process, + processVersion, + deploymentData, + savepointPath + ).runTest() } class FlinkVerificationMain( + miniCluster: MiniCluster, + env: StreamExecutionEnvironment, modelData: ModelData, process: CanonicalProcess, processVersion: ProcessVersion, deploymentData: DeploymentData, - savepointPath: String, - configuration: Configuration + savepointPath: String ) { - private val stubbedRunner = new FlinkStubbedRunner(modelData.modelClassLoader, configuration) + private val stubbedRunner = new FlinkStubbedRunner(miniCluster, env) def runTest(): Unit = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener val resultCollector = new TestServiceInvocationCollector(collectingListener) val registrar = prepareRegistrar() - val parallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - val env = stubbedRunner.createEnv(parallelism) registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute(env, parallelism, process.name, SavepointRestoreSettings.forPath(savepointPath, true)) + stubbedRunner.execute( + process.name, + SavepointRestoreSettings.forPath(savepointPath, true), + modelData.modelClassLoader + ) } protected def prepareRegistrar(): FlinkProcessRegistrar = { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala new file mode 100644 index 00000000000..446a182879d --- /dev/null +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala @@ -0,0 +1,38 @@ +package pl.touk.nussknacker.engine.process.testmechanism + +import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} +import org.apache.flink.runtime.minicluster.MiniCluster +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import pl.touk.nussknacker.engine.api.process.ProcessName +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader + +import scala.jdk.CollectionConverters._ + +// FIXME abr: rename +// we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... +final class FlinkStubbedRunner(miniCluster: MiniCluster, env: StreamExecutionEnvironment) { + + def execute( + scenarioName: ProcessName, + savepointRestoreSettings: SavepointRestoreSettings, + modelClassLoader: ModelClassLoader + ): Unit = { + val streamGraph = env.getStreamGraph + streamGraph.setJobName(scenarioName.value) + val jobGraph = streamGraph.getJobGraph + setupJobGraph(jobGraph, savepointRestoreSettings, modelClassLoader) + + val id = miniCluster.submitJob(jobGraph).get().getJobID + miniCluster.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) + } + + private def setupJobGraph( + jobGraph: JobGraph, + savepointRestoreSettings: SavepointRestoreSettings, + modelClassLoader: ModelClassLoader + ): Unit = { + jobGraph.setClasspaths(modelClassLoader.urls.asJava) + jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) + } + +} diff --git a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkStreamingProcessMainSpec.scala b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkStreamingProcessMainSpec.scala index cdbdb6b652e..8689d06c41e 100644 --- a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkStreamingProcessMainSpec.scala +++ b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkStreamingProcessMainSpec.scala @@ -7,18 +7,9 @@ import org.scalatest.Inside import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import pl.touk.nussknacker.engine.api._ -import pl.touk.nussknacker.engine.api.process._ import pl.touk.nussknacker.engine.build.ScenarioBuilder import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.flink.test.FlinkTestConfiguration -import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ -import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder -import pl.touk.nussknacker.engine.process.runner.SimpleProcessConfigCreator.{ - sinkForIntsResultsHolder, - valueMonitorResultsHolder -} - -import java.net.ConnectException class FlinkStreamingProcessMainSpec extends AnyFlatSpec with Matchers with Inside { @@ -51,50 +42,3 @@ class FlinkStreamingProcessMainSpec extends AnyFlatSpec with Matchers with Insid } } - -class SimpleProcessConfigCreator extends EmptyProcessConfigCreator { - - override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = - Map( - "logService" -> WithCategories(LogService, "c1"), - "throwingService" -> WithCategories(new ThrowingService(new RuntimeException("Thrown as expected")), "c1"), - "throwingTransientService" -> WithCategories(new ThrowingService(new ConnectException()), "c1"), - "returningDependentTypeService" -> WithCategories(ReturningDependentTypeService, "c1"), - "collectingEager" -> WithCategories(CollectingEagerService, "c1"), - "returningComponentUseCaseService" -> WithCategories(ReturningComponentUseCaseService, "c1") - ) - - override def sinkFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SinkFactory]] = Map( - "monitor" -> WithCategories(SinkFactory.noParam(MonitorEmptySink), "c2"), - "valueMonitor" -> WithCategories(SinkForAny(valueMonitorResultsHolder), "c2"), - "sinkForInts" -> WithCategories.anyCategory(SinkForInts(sinkForIntsResultsHolder)) - ) - - override def customStreamTransformers( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[CustomStreamTransformer]] = Map( - "stateCustom" -> WithCategories.anyCategory(StateCustomNode), - "transformWithTime" -> WithCategories.anyCategory(TransformerWithTime), - "joinBranchExpression" -> WithCategories.anyCategory(CustomJoinUsingBranchExpressions), - "transformerAddingComponentUseCase" -> WithCategories.anyCategory(TransformerAddingComponentUseCase) - ) - - override def sourceFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SourceFactory]] = Map( - "input" -> WithCategories(simpleRecordSource(Nil), "cat2"), - "jsonInput" -> WithCategories(jsonSource, "cat2"), - "typedJsonInput" -> WithCategories(TypedJsonSource, "cat2"), - "genericSourceWithCustomVariables" -> WithCategories.anyCategory(GenericSourceWithCustomVariables) - ) - -} - -object SimpleProcessConfigCreator extends Serializable { - - val valueMonitorResultsHolder = new TestResultsHolder[AnyRef] - val sinkForIntsResultsHolder = new TestResultsHolder[java.lang.Integer] - -} diff --git a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/SimpleProcessConfigCreator.scala b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/SimpleProcessConfigCreator.scala new file mode 100644 index 00000000000..dc4206f898c --- /dev/null +++ b/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/SimpleProcessConfigCreator.scala @@ -0,0 +1,59 @@ +package pl.touk.nussknacker.engine.process.runner + +import pl.touk.nussknacker.engine.api.process._ +import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, Service} +import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ +import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder +import pl.touk.nussknacker.engine.process.runner.SimpleProcessConfigCreator.{ + sinkForIntsResultsHolder, + valueMonitorResultsHolder +} + +import java.net.ConnectException + +class SimpleProcessConfigCreator extends EmptyProcessConfigCreator { + + override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = + Map( + "logService" -> WithCategories(LogService, "c1"), + "throwingService" -> WithCategories(new ThrowingService(new RuntimeException("Thrown as expected")), "c1"), + "throwingTransientService" -> WithCategories(new ThrowingService(new ConnectException()), "c1"), + "returningDependentTypeService" -> WithCategories(ReturningDependentTypeService, "c1"), + "collectingEager" -> WithCategories(CollectingEagerService, "c1"), + "returningComponentUseCaseService" -> WithCategories(ReturningComponentUseCaseService, "c1") + ) + + override def sinkFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SinkFactory]] = Map( + "monitor" -> WithCategories(SinkFactory.noParam(MonitorEmptySink), "c2"), + "valueMonitor" -> WithCategories(SinkForAny(valueMonitorResultsHolder), "c2"), + "sinkForInts" -> WithCategories.anyCategory(SinkForInts(sinkForIntsResultsHolder)) + ) + + override def customStreamTransformers( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[CustomStreamTransformer]] = Map( + "stateCustom" -> WithCategories.anyCategory(StateCustomNode), + "transformWithTime" -> WithCategories.anyCategory(TransformerWithTime), + "joinBranchExpression" -> WithCategories.anyCategory(CustomJoinUsingBranchExpressions), + "transformerAddingComponentUseCase" -> WithCategories.anyCategory(TransformerAddingComponentUseCase) + ) + + override def sourceFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SourceFactory]] = Map( + "input" -> WithCategories(simpleRecordSource(Nil), "cat2"), + "jsonInput" -> WithCategories(jsonSource, "cat2"), + "typedJsonInput" -> WithCategories(TypedJsonSource, "cat2"), + "genericSourceWithCustomVariables" -> WithCategories.anyCategory(GenericSourceWithCustomVariables) + ) + +} + +object SimpleProcessConfigCreator extends Serializable { + + val valueMonitorResultsHolder = new TestResultsHolder[AnyRef] + val sinkForIntsResultsHolder = new TestResultsHolder[java.lang.Integer] + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala index b6f768ee7c1..9b995c45791 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala @@ -3,6 +3,7 @@ package pl.touk.nussknacker.engine.management import cats.implicits._ import com.typesafe.scalalogging.LazyLogging import io.circe.syntax.EncoderOps +import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.engine.ModelData._ import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.api.deployment.DeploymentUpdateStrategy.StateRestoringStrategy @@ -13,6 +14,7 @@ import pl.touk.nussknacker.engine.api.process.{ProcessIdWithName, ProcessName, V import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{DeploymentData, ExternalDeploymentId} import pl.touk.nussknacker.engine.management.FlinkDeploymentManager.prepareProgramArgs +import pl.touk.nussknacker.engine.management.testsmechanism.{FlinkProcessTestRunner, FlinkProcessVerifier} import pl.touk.nussknacker.engine.{BaseModelData, DeploymentManagerDependencies, newdeployment} import scala.concurrent.Future @@ -27,7 +29,8 @@ abstract class FlinkDeploymentManager( import dependencies._ - private lazy val testRunner = new FlinkProcessTestRunner(modelData.asInvokableModelData) + private lazy val testRunner = + new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration) private lazy val verification = new FlinkProcessVerifier(modelData.asInvokableModelData) @@ -115,7 +118,7 @@ abstract class FlinkDeploymentManager( makeSavepoint(_, savepointDir) } case DMTestScenarioCommand(_, canonicalProcess, scenarioTestData) => - testRunner.test(canonicalProcess, scenarioTestData) + testRunner.runTestsAsync(canonicalProcess, scenarioTestData) case _: DMRunOffScheduleCommand => notImplemented } diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala deleted file mode 100644 index f1522ecdfe6..00000000000 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessTestRunner.scala +++ /dev/null @@ -1,29 +0,0 @@ -package pl.touk.nussknacker.engine.management - -import io.circe.Json -import org.apache.flink.configuration.Configuration -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.test.ScenarioTestData -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker - -import scala.concurrent.{ExecutionContext, Future} - -class FlinkProcessTestRunner(modelData: ModelData) { - - private val methodInvoker = new ReflectiveMethodInvoker[TestResults[Json]]( - modelData.modelClassLoader.classLoader, - "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", - "run" - ) - - // NU-1455: We encode variable on the engine, because of classLoader's problems - def test(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData)( - implicit ec: ExecutionContext - ): Future[TestResults[Json]] = - Future { - methodInvoker.invokeStaticMethod(modelData, canonicalProcess, scenarioTestData, new Configuration()) - } - -} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala new file mode 100644 index 00000000000..1bc413a1909 --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala @@ -0,0 +1,45 @@ +package pl.touk.nussknacker.engine.management.testsmechanism + +import io.circe.Json +import org.apache.flink.configuration.Configuration +import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.test.ScenarioTestData +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker + +import scala.concurrent.{ExecutionContext, Future} + +class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecutionConfig: Configuration) { + + private val streamExecutionEnvironment = + TestsMechanismStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment(parallelism, streamExecutionConfig) + + private val miniCluster = TestsMechanismMiniClusterFactory.createConfiguredMiniCluster(parallelism) + + // We use reflection to avoid bundling of flinkExecutor.jar inside flinkDeploymentManager assembly jar + // TODO: use provided dependency instead + private val methodInvoker = new ReflectiveMethodInvoker[TestResults[Json]]( + modelData.modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", + "run" + ) + + def runTestsAsync(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData)( + implicit ec: ExecutionContext + ): Future[TestResults[Json]] = + Future { + runTests(canonicalProcess, scenarioTestData) + } + + // NU-1455: We encode variable on the engine, because of classLoader's problems + def runTests(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = + methodInvoker.invokeStaticMethod( + miniCluster, + streamExecutionEnvironment, + modelData, + canonicalProcess, + scenarioTestData + ) + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala similarity index 52% rename from engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala rename to engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala index 82691385dc1..2df63253ac1 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkProcessVerifier.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala @@ -1,18 +1,21 @@ -package pl.touk.nussknacker.engine.management +package pl.touk.nussknacker.engine.management.testsmechanism import com.typesafe.scalalogging.LazyLogging import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.ProcessVersion +import pl.touk.nussknacker.engine.api.{ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData -import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker +import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ReflectiveMethodInvoker} import scala.concurrent.Future +import scala.util.Using import scala.util.control.NonFatal class FlinkProcessVerifier(modelData: ModelData) extends LazyLogging { + // We use reflection to avoid bundling of flinkExecutor.jar inside flinkDeploymentManager assembly jar + // TODO: use provided dependency instead private val methodInvoker = new ReflectiveMethodInvoker[Unit]( modelData.modelClassLoader.classLoader, "pl.touk.nussknacker.engine.process.runner.FlinkVerificationMain", @@ -24,17 +27,30 @@ class FlinkProcessVerifier(modelData: ModelData) extends LazyLogging { canonicalProcess: CanonicalProcess, savepointPath: String ): Future[Unit] = { + val parallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](canonicalProcess.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) val processId = processVersion.processName try { logger.info(s"Starting to verify $processId") - methodInvoker.invokeStaticMethod( - modelData, - canonicalProcess, - processVersion, - DeploymentData.empty, - savepointPath, - new Configuration() - ) + // TODO: reuse a single mini cluster between each verifications + Using.resource(TestsMechanismMiniClusterFactory.createConfiguredMiniCluster(parallelism)) { miniCluster => + val env = TestsMechanismStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment( + parallelism, + new Configuration() + ) + + methodInvoker.invokeStaticMethod( + miniCluster, + env, + modelData, + canonicalProcess, + processVersion, + DeploymentData.empty, + savepointPath + ) + } logger.info(s"Verification of $processId successful") Future.successful(()) } catch { diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala new file mode 100644 index 00000000000..f0a4aed9437 --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala @@ -0,0 +1,39 @@ +package pl.touk.nussknacker.engine.management.testsmechanism + +import org.apache.flink.configuration.{Configuration, CoreOptions, RestOptions, TaskManagerOptions} +import org.apache.flink.core.fs.FileSystem +import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} + +object TestsMechanismMiniClusterFactory { + + def createConfiguredMiniCluster(parallelism: Int): MiniCluster = { + val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = parallelism) + + // it is required for proper working of HadoopFileSystem + FileSystem.initialize(miniClusterConfiguration, null) + + createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = parallelism) + } + + private def prepareMiniClusterConfiguration(numTaskSlots: Int) = { + val configuration: Configuration = new Configuration + configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, numTaskSlots) + configuration.set[Integer](RestOptions.PORT, 0) + + // FIXME: reversing flink default order + configuration.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") + configuration + } + + private def createMiniCluster(configuration: Configuration, numSlotsPerTaskManager: Int) = { + val miniCluster = new MiniCluster( + new MiniClusterConfiguration.Builder() + .setNumSlotsPerTaskManager(numSlotsPerTaskManager) + .setConfiguration(configuration) + .build() + ) + miniCluster.start() + miniCluster + } + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala new file mode 100644 index 00000000000..172ff53f26d --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala @@ -0,0 +1,19 @@ +package pl.touk.nussknacker.engine.management.testsmechanism + +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment + +object TestsMechanismStreamExecutionEnvironmentFactory { + + def createStreamExecutionEnvironment(parallelism: Int, configuration: Configuration): StreamExecutionEnvironment = { + val env = StreamExecutionEnvironment.createLocalEnvironment( + parallelism, + configuration + ) + // Checkpoints are disabled to prevent waiting for checkpoint to happen + // before finishing execution. + env.getCheckpointConfig.disableCheckpointing() + env + } + +} diff --git a/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator b/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator new file mode 100644 index 00000000000..dc12680791f --- /dev/null +++ b/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator @@ -0,0 +1 @@ +pl.touk.nussknacker.engine.management.testsmechanism.SimpleProcessConfigCreator diff --git a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala similarity index 96% rename from engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala rename to engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala index fc4f607876f..5e53e6e7223 100644 --- a/engine/flink/executor/src/test/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMainSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.engine.process.runner +package pl.touk.nussknacker.engine.management.testsmechanism import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} import io.circe.Json @@ -7,50 +7,38 @@ import org.apache.flink.runtime.client.JobExecutionException import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.scalatest.{BeforeAndAfterEach, Inside, OptionValues} -import pl.touk.nussknacker.engine.api.component.{ - ComponentAdditionalConfig, - DesignerWideComponentId, - ParameterAdditionalUIConfig -} -import pl.touk.nussknacker.engine.api.parameter.{ - ParameterName, - ParameterValueCompileTimeValidation, - ValueInputWithDictEditor -} +import pl.touk.nussknacker.engine.api.component.{ComponentAdditionalConfig, DesignerWideComponentId, ParameterAdditionalUIConfig} +import pl.touk.nussknacker.engine.api.parameter.{ParameterName, ParameterValueCompileTimeValidation, ValueInputWithDictEditor} import pl.touk.nussknacker.engine.api.process.ComponentUseCase import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} -import pl.touk.nussknacker.engine.api.{DisplayJsonWithEncoder, FragmentSpecificData, MetaData} +import pl.touk.nussknacker.engine.api.{DisplayJsonWithEncoder, FragmentSpecificData, MetaData, StreamMetaData} import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.FlatNode import pl.touk.nussknacker.engine.compile.FragmentResolver -import pl.touk.nussknacker.engine.flink.test.{ - FlinkTestConfiguration, - RecordingExceptionConsumer, - RecordingExceptionConsumerProvider -} +import pl.touk.nussknacker.engine.deployment.AdditionalModelConfigs +import pl.touk.nussknacker.engine.flink.test.{FlinkTestConfiguration, RecordingExceptionConsumer, RecordingExceptionConsumerProvider} import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.node.FragmentInputDefinition.{FragmentClazzRef, FragmentParameter} import pl.touk.nussknacker.engine.graph.node.{Case, FragmentInputDefinition, FragmentOutputDefinition} +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunnerSpec.{fragmentWithValidationName, processWithFragmentParameterValidation} import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ -import pl.touk.nussknacker.engine.process.runner.FlinkTestMainSpec.{ - fragmentWithValidationName, - processWithFragmentParameterValidation -} import pl.touk.nussknacker.engine.testmode.TestProcess._ -import pl.touk.nussknacker.engine.util.ThreadUtils -import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} -import pl.touk.nussknacker.engine.deployment.AdditionalModelConfigs -import pl.touk.nussknacker.engine.testing.LocalModelData +import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ThreadUtils} import pl.touk.nussknacker.engine.util.loader.ModelClassLoader +import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} import java.util.{Date, UUID} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.concurrent.{Await, Future} -import scala.jdk.CollectionConverters._ -class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with BeforeAndAfterEach with OptionValues { +class FlinkProcessTestRunnerSpec + extends AnyWordSpec + with Matchers + with Inside + with BeforeAndAfterEach + with OptionValues { import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.util.Implicits.RichScalaMap @@ -244,7 +232,6 @@ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with Befor val nodeResults = results.nodeResults nodeResults(sourceNodeId) should have length 5 - } "detect errors" in { @@ -776,7 +763,13 @@ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with Befor resolveConfigs = false ) ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - FlinkTestMain.run(modelData, process, scenarioTestData, FlinkTestConfiguration.setupMemory(new Configuration)) + // TODO: reuse this instance between all test cases + val parallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + new FlinkProcessTestRunner(modelData, parallelism, FlinkTestConfiguration.setupMemory(new Configuration)) + .runTests(process, scenarioTestData) } } @@ -812,7 +805,7 @@ class FlinkTestMainSpec extends AnyWordSpec with Matchers with Inside with Befor } -object FlinkTestMainSpec { +object FlinkProcessTestRunnerSpec { private val fragmentWithValidationName = "fragmentWithValidation" private val processWithFragmentParameterValidation: CanonicalProcess = { diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala new file mode 100644 index 00000000000..4e142dcdf25 --- /dev/null +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala @@ -0,0 +1,59 @@ +package pl.touk.nussknacker.engine.management.testsmechanism + +import pl.touk.nussknacker.engine.api.process._ +import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, Service} +import pl.touk.nussknacker.engine.management.testsmechanism.SimpleProcessConfigCreator.{ + sinkForIntsResultsHolder, + valueMonitorResultsHolder +} +import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ +import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder + +import java.net.ConnectException + +class SimpleProcessConfigCreator extends EmptyProcessConfigCreator { + + override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = + Map( + "logService" -> WithCategories(LogService, "c1"), + "throwingService" -> WithCategories(new ThrowingService(new RuntimeException("Thrown as expected")), "c1"), + "throwingTransientService" -> WithCategories(new ThrowingService(new ConnectException()), "c1"), + "returningDependentTypeService" -> WithCategories(ReturningDependentTypeService, "c1"), + "collectingEager" -> WithCategories(CollectingEagerService, "c1"), + "returningComponentUseCaseService" -> WithCategories(ReturningComponentUseCaseService, "c1") + ) + + override def sinkFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SinkFactory]] = Map( + "monitor" -> WithCategories(SinkFactory.noParam(MonitorEmptySink), "c2"), + "valueMonitor" -> WithCategories(SinkForAny(valueMonitorResultsHolder), "c2"), + "sinkForInts" -> WithCategories.anyCategory(SinkForInts(sinkForIntsResultsHolder)) + ) + + override def customStreamTransformers( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[CustomStreamTransformer]] = Map( + "stateCustom" -> WithCategories.anyCategory(StateCustomNode), + "transformWithTime" -> WithCategories.anyCategory(TransformerWithTime), + "joinBranchExpression" -> WithCategories.anyCategory(CustomJoinUsingBranchExpressions), + "transformerAddingComponentUseCase" -> WithCategories.anyCategory(TransformerAddingComponentUseCase) + ) + + override def sourceFactories( + modelDependencies: ProcessObjectDependencies + ): Map[String, WithCategories[SourceFactory]] = Map( + "input" -> WithCategories(simpleRecordSource(Nil), "cat2"), + "jsonInput" -> WithCategories(jsonSource, "cat2"), + "typedJsonInput" -> WithCategories(TypedJsonSource, "cat2"), + "genericSourceWithCustomVariables" -> WithCategories.anyCategory(GenericSourceWithCustomVariables) + ) + +} + +object SimpleProcessConfigCreator extends Serializable { + + val valueMonitorResultsHolder = new TestResultsHolder[AnyRef] + val sinkForIntsResultsHolder = new TestResultsHolder[java.lang.Integer] + +} diff --git a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala similarity index 92% rename from engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala rename to engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala index 19ffbbfacc7..1eeed0a4a3c 100644 --- a/engine/flink/kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/kafka/source/flink/TestFromFileSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.engine.kafka.source.flink +package pl.touk.nussknacker.testsmechanism.kafka import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigValueFactory.fromAnyRef @@ -18,8 +18,9 @@ import pl.touk.nussknacker.engine.flink.test.FlinkTestConfiguration import pl.touk.nussknacker.engine.flink.util.sink.SingleValueSinkFactory.SingleValueParamName import pl.touk.nussknacker.engine.kafka.KafkaFactory.TopicParamName import pl.touk.nussknacker.engine.kafka.source.InputMeta +import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator.ResultsHolders -import pl.touk.nussknacker.engine.process.runner.FlinkTestMain +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults @@ -28,7 +29,6 @@ import pl.touk.nussknacker.engine.util.json.ToJsonEncoder import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} -import java.net.URL import java.util.Collections class TestFromFileSpec @@ -133,12 +133,9 @@ class TestFromFileSpec private def run(process: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - FlinkTestMain.run( - modelData, - process, - scenarioTestData, - FlinkTestConfiguration.setupMemory(new Configuration), - ) + // TODO: reuse this instance between all test cases + new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + .runTests(process, scenarioTestData) } } diff --git a/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala similarity index 91% rename from engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala rename to engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala index 0e9f4f160cb..be6af1b5bb5 100644 --- a/engine/flink/schemed-kafka-components-utils/src/test/scala/pl/touk/nussknacker/engine/schemedkafka/source/flink/TestWithTestDataSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.engine.schemedkafka.source.flink +package pl.touk.nussknacker.testsmechanism.schemedkafka import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigValueFactory.fromAnyRef @@ -8,20 +8,21 @@ import io.circe.Json._ import org.apache.avro.Schema import org.apache.flink.configuration.Configuration import org.apache.kafka.common.record.TimestampType -import org.scalatest.{LoneElement, OptionValues} import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers +import org.scalatest.{LoneElement, OptionValues} import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} import pl.touk.nussknacker.engine.build.ScenarioBuilder import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.flink.test.FlinkTestConfiguration import pl.touk.nussknacker.engine.flink.util.sink.SingleValueSinkFactory.SingleValueParamName +import pl.touk.nussknacker.engine.flink.util.test.FlinkTestScenarioRunner import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.kafka.UnspecializedTopicName import pl.touk.nussknacker.engine.kafka.source.InputMeta +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder -import pl.touk.nussknacker.engine.process.runner.FlinkTestMain import pl.touk.nussknacker.engine.schemedkafka.KafkaAvroIntegrationMockSchemaRegistry.schemaRegistryMockClient import pl.touk.nussknacker.engine.schemedkafka.KafkaAvroTestProcessConfigCreator import pl.touk.nussknacker.engine.schemedkafka.KafkaUniversalComponentTransformer.{ @@ -32,7 +33,6 @@ import pl.touk.nussknacker.engine.schemedkafka.schema.{Address, Company} import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.confluent.ConfluentUtils import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.universal.MockSchemaRegistryClientFactory import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.{SchemaRegistryClientFactory, SchemaVersionOption} -import pl.touk.nussknacker.engine.schemedkafka.source.flink.TestWithTestDataSpec.sinkForInputMetaResultsHolder import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.testmode.TestProcess._ @@ -40,8 +40,8 @@ import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.util.json.ToJsonEncoder import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} +import pl.touk.nussknacker.testsmechanism.schemedkafka.TestWithTestDataSpec.sinkForInputMetaResultsHolder -import java.net.URL import java.util.Collections class TestWithTestDataSpec @@ -202,17 +202,15 @@ class TestWithTestDataSpec private def run(process: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - FlinkTestMain.run( - LocalModelData( - config, - List.empty, - configCreator = creator, - modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) - ), - process, - scenarioTestData, - FlinkTestConfiguration.setupMemory(new Configuration), + val modelData = LocalModelData( + config, + List.empty, + configCreator = creator, + modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) ) + // TODO: reuse this instance between all test cases + new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + .runTests(process, scenarioTestData) } } From e5d1e002f58e31fd73a96286dacb5a31b310ffb6 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 20 Jan 2025 15:33:53 +0100 Subject: [PATCH 13/48] BatchDataGenerationSpec: added timestamp suffix for easier tests retry --- .../touk/nussknacker/BatchDataGenerationSpec.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/e2e-tests/src/test/scala/pl/touk/nussknacker/BatchDataGenerationSpec.scala b/e2e-tests/src/test/scala/pl/touk/nussknacker/BatchDataGenerationSpec.scala index 912431416a1..3b38799e3f4 100644 --- a/e2e-tests/src/test/scala/pl/touk/nussknacker/BatchDataGenerationSpec.scala +++ b/e2e-tests/src/test/scala/pl/touk/nussknacker/BatchDataGenerationSpec.scala @@ -27,8 +27,9 @@ class BatchDataGenerationSpec private val designerServiceUrl = "http://localhost:8080" - private val liveDataGenScenarioName = "SumTransactions-LiveData" - private val randomDataGenScenarioName = "SumTransactions-RandomData" + private val timeBasedRandomSuffix = System.currentTimeMillis() + private val liveDataGenScenarioName = s"SumTransactions-LiveData-$timeBasedRandomSuffix" + private val randomDataGenScenarioName = s"SumTransactions-RandomData-$timeBasedRandomSuffix" override def beforeAll(): Unit = { createEmptyBatchScenario(liveDataGenScenarioName, "Default") @@ -106,7 +107,7 @@ class BatchDataGenerationSpec | "nodeResults": { | "sourceId": [ | { - | "id": "SumTransactions-LiveData-sourceId-0-0", + | "id": "$liveDataGenScenarioName-sourceId-0-0", | "variables": { | "input": { | "pretty": { @@ -122,7 +123,7 @@ class BatchDataGenerationSpec | ], | "end": [ | { - | "id": "SumTransactions-LiveData-sourceId-0-0", + | "id": "$liveDataGenScenarioName-sourceId-0-0", | "variables": { | "input": { | "pretty": { @@ -183,7 +184,7 @@ class BatchDataGenerationSpec | "nodeResults": { | "sourceId": [ | { - | "id": "SumTransactions-LiveData-sourceId-0-0", + | "id": "$liveDataGenScenarioName-sourceId-0-0", | "variables": { | "input": { | "pretty": { @@ -199,7 +200,7 @@ class BatchDataGenerationSpec | ], | "end": [ | { - | "id": "SumTransactions-LiveData-sourceId-0-0", + | "id": "$liveDataGenScenarioName-sourceId-0-0", | "variables": { | "input": { | "pretty": { From 22f432fad60500550d2fd5a880de12ea13e0ef44 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 20 Jan 2025 15:49:06 +0100 Subject: [PATCH 14/48] FlinkDeploymentManager: close test runner + test runner reused in some tests --- .../management/FlinkDeploymentManager.scala | 5 +++ .../engine/management/FlinkRestManager.scala | 4 -- .../FlinkProcessTestRunner.scala | 8 +++- .../FlinkProcessTestRunnerSpec.scala | 39 ++++++++++------ .../kafka/TestFromFileSpec.scala | 32 +++++++------ .../schemedkafka/TestWithTestDataSpec.scala | 45 ++++++++++--------- 6 files changed, 77 insertions(+), 56 deletions(-) diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala index 9b995c45791..4f16942d8ea 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala @@ -265,6 +265,11 @@ abstract class FlinkDeploymentManager( override def processStateDefinitionManager: ProcessStateDefinitionManager = FlinkProcessStateDefinitionManager + override def close(): Unit = { + logger.info("Closing Flink Deployment Manager") + testRunner.close() + } + } object FlinkDeploymentManager { diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala index 3fef55f145e..3a436aed5ae 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala @@ -297,10 +297,6 @@ class FlinkRestManager( Future.successful(()) } - override def close(): Unit = { - logger.info("Closing Flink REST manager") - } - } object FlinkRestManager { diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala index 1bc413a1909..84cf2585bdf 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala @@ -10,7 +10,8 @@ import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker import scala.concurrent.{ExecutionContext, Future} -class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecutionConfig: Configuration) { +class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecutionConfig: Configuration) + extends AutoCloseable { private val streamExecutionEnvironment = TestsMechanismStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment(parallelism, streamExecutionConfig) @@ -42,4 +43,9 @@ class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecu scenarioTestData ) + def close(): Unit = { + miniCluster.close() + streamExecutionEnvironment.close() + } + } diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala index 5e53e6e7223..7ed17267c4c 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala @@ -7,8 +7,16 @@ import org.apache.flink.runtime.client.JobExecutionException import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec import org.scalatest.{BeforeAndAfterEach, Inside, OptionValues} -import pl.touk.nussknacker.engine.api.component.{ComponentAdditionalConfig, DesignerWideComponentId, ParameterAdditionalUIConfig} -import pl.touk.nussknacker.engine.api.parameter.{ParameterName, ParameterValueCompileTimeValidation, ValueInputWithDictEditor} +import pl.touk.nussknacker.engine.api.component.{ + ComponentAdditionalConfig, + DesignerWideComponentId, + ParameterAdditionalUIConfig +} +import pl.touk.nussknacker.engine.api.parameter.{ + ParameterName, + ParameterValueCompileTimeValidation, + ValueInputWithDictEditor +} import pl.touk.nussknacker.engine.api.process.ComponentUseCase import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} import pl.touk.nussknacker.engine.api.{DisplayJsonWithEncoder, FragmentSpecificData, MetaData, StreamMetaData} @@ -17,11 +25,18 @@ import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.FlatNode import pl.touk.nussknacker.engine.compile.FragmentResolver import pl.touk.nussknacker.engine.deployment.AdditionalModelConfigs -import pl.touk.nussknacker.engine.flink.test.{FlinkTestConfiguration, RecordingExceptionConsumer, RecordingExceptionConsumerProvider} +import pl.touk.nussknacker.engine.flink.test.{ + FlinkTestConfiguration, + RecordingExceptionConsumer, + RecordingExceptionConsumerProvider +} import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.node.FragmentInputDefinition.{FragmentClazzRef, FragmentParameter} import pl.touk.nussknacker.engine.graph.node.{Case, FragmentInputDefinition, FragmentOutputDefinition} -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunnerSpec.{fragmentWithValidationName, processWithFragmentParameterValidation} +import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunnerSpec.{ + fragmentWithValidationName, + processWithFragmentParameterValidation +} import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ import pl.touk.nussknacker.engine.testmode.TestProcess._ import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ThreadUtils} @@ -762,15 +777,13 @@ class FlinkProcessTestRunnerSpec ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround), resolveConfigs = false ) - ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - // TODO: reuse this instance between all test cases - val parallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - new FlinkProcessTestRunner(modelData, parallelism, FlinkTestConfiguration.setupMemory(new Configuration)) - .runTests(process, scenarioTestData) - } + // TODO: reuse this instance between all test cases + val parallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + new FlinkProcessTestRunner(modelData, parallelism, FlinkTestConfiguration.setupMemory(new Configuration)) + .runTests(process, scenarioTestData) } private def nodeResult(count: Int, vars: (String, Any)*): ResultContext[_] = diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala index 1eeed0a4a3c..49db5f3e0dc 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala @@ -7,13 +7,12 @@ import io.circe.Json import io.circe.Json.{Null, fromString, obj} import org.apache.flink.configuration.Configuration import org.apache.kafka.common.record.TimestampType -import org.scalatest.OptionValues import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers +import org.scalatest.{BeforeAndAfterAll, OptionValues} import pl.touk.nussknacker.engine.ModelData import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} import pl.touk.nussknacker.engine.build.ScenarioBuilder -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.flink.test.FlinkTestConfiguration import pl.touk.nussknacker.engine.flink.util.sink.SingleValueSinkFactory.SingleValueParamName import pl.touk.nussknacker.engine.kafka.KafkaFactory.TopicParamName @@ -23,8 +22,6 @@ import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessCo import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.testing.LocalModelData -import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.util.json.ToJsonEncoder import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} @@ -36,9 +33,10 @@ class TestFromFileSpec with Matchers with LazyLogging with EitherValuesDetailedMessage - with OptionValues { + with OptionValues + with BeforeAndAfterAll { - private lazy val config = ConfigFactory + private val config = ConfigFactory .empty() .withValue(KafkaConfigProperties.bootstrapServersProperty(), fromAnyRef("kafka_should_not_be_used:9092")) .withValue( @@ -47,7 +45,7 @@ class TestFromFileSpec ) .withValue("kafka.topicsExistenceValidationConfig.enabled", fromAnyRef(false)) - protected lazy val modelData: ModelData = + private val modelData: ModelData = LocalModelData( inputConfig = config, components = List.empty, @@ -55,6 +53,14 @@ class TestFromFileSpec modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) ) + private val testRunner = + new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + + override protected def afterAll(): Unit = { + super.afterAll() + testRunner.close() + } + test("Should pass correct timestamp from test data") { val topic = "simple" val expectedTimestamp = System.currentTimeMillis() @@ -97,7 +103,7 @@ class TestFromFileSpec _.add("value", obj("id" -> fromString("fooId"), "field" -> fromString("fooField"))) ) - val results = run(process, ScenarioTestData(ScenarioTestJsonRecord("start", consumerRecord) :: Nil)) + val results = testRunner.runTests(process, ScenarioTestData(ScenarioTestJsonRecord("start", consumerRecord) :: Nil)) val testResultVars = results.nodeResults("end").head.variables testResultVars("extractedTimestamp").hcursor.downField("pretty").as[Long].rightValue shouldBe expectedTimestamp @@ -126,19 +132,11 @@ class TestFromFileSpec .add("value", obj("id" -> fromString("1234"), "field" -> fromString("abcd"))) ) - val results = run(process, ScenarioTestData(ScenarioTestJsonRecord("start", consumerRecord) :: Nil)) + val results = testRunner.runTests(process, ScenarioTestData(ScenarioTestJsonRecord("start", consumerRecord) :: Nil)) results.nodeResults shouldBe Symbol("nonEmpty") } - private def run(process: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { - ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - // TODO: reuse this instance between all test cases - new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) - .runTests(process, scenarioTestData) - } - } - } object TestFromFileSpec extends Serializable { diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala index be6af1b5bb5..318fdc43fe9 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala @@ -10,7 +10,7 @@ import org.apache.flink.configuration.Configuration import org.apache.kafka.common.record.TimestampType import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers -import org.scalatest.{LoneElement, OptionValues} +import org.scalatest.{BeforeAndAfterAll, LoneElement, OptionValues} import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} import pl.touk.nussknacker.engine.build.ScenarioBuilder @@ -50,15 +50,16 @@ class TestWithTestDataSpec with LazyLogging with EitherValuesDetailedMessage with OptionValues - with LoneElement { + with LoneElement + with BeforeAndAfterAll { - private lazy val creator: KafkaAvroTestProcessConfigCreator = + private val creator: KafkaAvroTestProcessConfigCreator = new KafkaAvroTestProcessConfigCreator(sinkForInputMetaResultsHolder) { override protected def schemaRegistryClientFactory: SchemaRegistryClientFactory = MockSchemaRegistryClientFactory.confluentBased(schemaRegistryMockClient) } - private lazy val config = ConfigFactory + private val config = ConfigFactory .empty() .withValue(KafkaConfigProperties.bootstrapServersProperty(), fromAnyRef("kafka_should_not_be_used:9092")) .withValue( @@ -68,6 +69,22 @@ class TestWithTestDataSpec .withValue("kafka.topicsExistenceValidationConfig.enabled", fromAnyRef(false)) .withValue("kafka.avroKryoGenericRecordSchemaIdSerialization", fromAnyRef(false)) + private val modelData = + LocalModelData( + config, + List.empty, + configCreator = creator, + modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) + ) + + private val testRunner = + new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + + override protected def afterAll(): Unit = { + super.afterAll() + testRunner.close() + } + test("Should pass correct timestamp from test data") { val topic = UnspecializedTopicName("address") val expectedTimestamp = System.currentTimeMillis() @@ -114,7 +131,7 @@ class TestWithTestDataSpec val testRecordJson = obj("keySchemaId" -> Null, "valueSchemaId" -> fromInt(id), "consumerRecord" -> consumerRecord) val scenarioTestData = ScenarioTestData(ScenarioTestJsonRecord("start", testRecordJson) :: Nil) - val results = run(process, scenarioTestData) + val results = testRunner.runTests(process, scenarioTestData) val testResultVars = results.nodeResults("end").head.variables testResultVars("extractedTimestamp").hcursor.downField("pretty").as[Long].rightValue shouldBe expectedTimestamp @@ -146,7 +163,7 @@ class TestWithTestDataSpec ) val scenarioTestData = ScenarioTestData("start", parameterExpressions) - val results = run(process, scenarioTestData) + val results = testRunner.runTests(process, scenarioTestData) results .invocationResults("end") .head @@ -168,7 +185,7 @@ class TestWithTestDataSpec ParameterName("in") -> Expression.spel("'some-text-id'") ) val scenarioTestData = ScenarioTestData("fragment1", parameterExpressions) - val results = run(fragment, scenarioTestData) + val results = testRunner.runTests(fragment, scenarioTestData) results.nodeResults("fragment1").loneElement shouldBe ResultContext( "fragment1-fragment1-0-0", @@ -200,20 +217,6 @@ class TestWithTestDataSpec schemaRegistryMockClient.register(subject, parsedSchema) } - private def run(process: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { - ThreadUtils.withThisAsContextClassLoader(getClass.getClassLoader) { - val modelData = LocalModelData( - config, - List.empty, - configCreator = creator, - modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) - ) - // TODO: reuse this instance between all test cases - new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) - .runTests(process, scenarioTestData) - } - } - } object TestWithTestDataSpec { From 68eb84cf0105ec4fb624387a1ca4340d8508c3e0 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 20 Jan 2025 16:07:01 +0100 Subject: [PATCH 15/48] Closing FlinkDeploymentManager in more tests --- .../api/description/TestingApiEndpoints.scala | 19 ++++--- .../periodic/flink/FlinkJarManager.scala | 2 +- ...amingDeploymentManagerProviderHelper.scala | 13 ++--- .../FlinkStreamingProcessTestRunnerSpec.scala | 50 +++++++++++-------- .../streaming/StreamingDockerTest.scala | 1 + 5 files changed, 43 insertions(+), 42 deletions(-) diff --git a/designer/server/src/main/scala/pl/touk/nussknacker/ui/api/description/TestingApiEndpoints.scala b/designer/server/src/main/scala/pl/touk/nussknacker/ui/api/description/TestingApiEndpoints.scala index b04ef1f1712..51536d7edc8 100644 --- a/designer/server/src/main/scala/pl/touk/nussknacker/ui/api/description/TestingApiEndpoints.scala +++ b/designer/server/src/main/scala/pl/touk/nussknacker/ui/api/description/TestingApiEndpoints.scala @@ -1,34 +1,33 @@ package pl.touk.nussknacker.ui.api.description +import io.circe.Encoder import pl.touk.nussknacker.engine.api.StreamMetaData import pl.touk.nussknacker.engine.api.definition.Parameter import pl.touk.nussknacker.engine.api.graph.{ProcessProperties, ScenarioGraph} import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.process.ProcessName -import pl.touk.nussknacker.engine.api.typed.typing.Typed +import pl.touk.nussknacker.engine.api.typed.typing._ import pl.touk.nussknacker.engine.definition.test.TestingCapabilities import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.restmodel.BaseEndpointDefinitions import pl.touk.nussknacker.restmodel.BaseEndpointDefinitions.SecuredEndpoint -import pl.touk.nussknacker.restmodel.definition.{UIParameter, UISourceParameters} +import pl.touk.nussknacker.restmodel.definition.UISourceParameters import pl.touk.nussknacker.restmodel.validation.ValidationResults.{NodeValidationError, NodeValidationErrorType} -import pl.touk.nussknacker.ui.api.TapirCodecs.ScenarioNameCodec._ import pl.touk.nussknacker.security.AuthCredentials import pl.touk.nussknacker.ui.api.TapirCodecs.ScenarioGraphCodec._ +import pl.touk.nussknacker.ui.api.TapirCodecs.ScenarioNameCodec._ import pl.touk.nussknacker.ui.api.TapirCodecs.ScenarioTestingCodecs._ -import pl.touk.nussknacker.ui.definition.DefinitionsService -import sttp.model.StatusCode.Ok -import sttp.tapir.EndpointIO.Example -import sttp.tapir._ -import sttp.tapir.json.circe.jsonBody -import io.circe.Encoder -import pl.touk.nussknacker.engine.api.typed.typing._ import pl.touk.nussknacker.ui.api.TestingApiHttpService.Examples.{ malformedTypingResultExample, noScenarioExample, testDataGenerationErrorExample } import pl.touk.nussknacker.ui.api.TestingApiHttpService.TestingError +import pl.touk.nussknacker.ui.definition.DefinitionsService +import sttp.model.StatusCode.Ok +import sttp.tapir.EndpointIO.Example +import sttp.tapir._ +import sttp.tapir.json.circe.jsonBody class TestingApiEndpoints(auth: EndpointInput[AuthCredentials]) extends BaseEndpointDefinitions { import NodesApiEndpoints.Dtos._ diff --git a/engine/flink/management/periodic/src/main/scala/pl/touk/nussknacker/engine/management/periodic/flink/FlinkJarManager.scala b/engine/flink/management/periodic/src/main/scala/pl/touk/nussknacker/engine/management/periodic/flink/FlinkJarManager.scala index 47b33e7bc93..1b398dea598 100644 --- a/engine/flink/management/periodic/src/main/scala/pl/touk/nussknacker/engine/management/periodic/flink/FlinkJarManager.scala +++ b/engine/flink/management/periodic/src/main/scala/pl/touk/nussknacker/engine/management/periodic/flink/FlinkJarManager.scala @@ -2,7 +2,6 @@ package pl.touk.nussknacker.engine.management.periodic.flink import com.typesafe.scalalogging.LazyLogging import org.apache.flink.api.common.JobID -import pl.touk.nussknacker.engine.{BaseModelData, newdeployment} import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{DeploymentData, ExternalDeploymentId} @@ -16,6 +15,7 @@ import pl.touk.nussknacker.engine.management.{ FlinkStreamingRestManager } import pl.touk.nussknacker.engine.modelconfig.InputConfigDuringExecution +import pl.touk.nussknacker.engine.{BaseModelData, newdeployment} import sttp.client3.SttpBackend import java.nio.file.{Files, Path, Paths} diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerProviderHelper.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerProviderHelper.scala index 931b35d5794..81578b4fdd4 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerProviderHelper.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerProviderHelper.scala @@ -1,25 +1,18 @@ package pl.touk.nussknacker.engine.management.streaming +import _root_.sttp.client3.asynchttpclient.future.AsyncHttpClientFutureBackend import akka.actor.ActorSystem import org.asynchttpclient.DefaultAsyncHttpClientConfig +import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api.component.DesignerWideComponentId import pl.touk.nussknacker.engine.api.deployment.{ DeploymentManager, NoOpScenarioActivityManager, ProcessingTypeActionServiceStub, - ProcessingTypeDeployedScenariosProviderStub, - ScenarioActivityManager + ProcessingTypeDeployedScenariosProviderStub } import pl.touk.nussknacker.engine.definition.component.Components.ComponentDefinitionExtractionMode import pl.touk.nussknacker.engine.management.FlinkStreamingDeploymentManagerProvider -import pl.touk.nussknacker.engine.{ - ConfigWithUnresolvedVersion, - DeploymentManagerDependencies, - ModelData, - ModelDependencies, - ProcessingTypeConfig -} -import sttp.client3.asynchttpclient.future.AsyncHttpClientFutureBackend object FlinkStreamingDeploymentManagerProviderHelper { diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingProcessTestRunnerSpec.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingProcessTestRunnerSpec.scala index 560206a719f..adf823079f6 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingProcessTestRunnerSpec.scala @@ -17,6 +17,7 @@ import pl.touk.nussknacker.test.{KafkaConfigProperties, VeryPatientScalaFutures, import java.util.UUID import scala.concurrent.Await import scala.jdk.CollectionConverters._ +import scala.util.Using class FlinkStreamingProcessTestRunnerSpec extends AnyFlatSpec @@ -45,20 +46,26 @@ class FlinkStreamingProcessTestRunnerSpec ) it should "run scenario in test mode" in { - val deploymentManager = + Using.resource( FlinkStreamingDeploymentManagerProviderHelper.createDeploymentManager(ConfigWithUnresolvedVersion(config)) - - val processName = ProcessName(UUID.randomUUID().toString) - val processVersion = ProcessVersion.empty.copy(processName = processName) - - val process = SampleProcess.prepareProcess(processName) - - whenReady(deploymentManager.processCommand(DMTestScenarioCommand(processVersion, process, scenarioTestData))) { r => - r.nodeResults shouldBe Map( - "startProcess" -> List(ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere")))), - "nightFilter" -> List(ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere")))), - "endSend" -> List(ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere")))) - ) + ) { deploymentManager => + val processName = ProcessName(UUID.randomUUID().toString) + val processVersion = ProcessVersion.empty.copy(processName = processName) + + val process = SampleProcess.prepareProcess(processName) + + whenReady(deploymentManager.processCommand(DMTestScenarioCommand(processVersion, process, scenarioTestData))) { + r => + r.nodeResults shouldBe Map( + "startProcess" -> List( + ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere"))) + ), + "nightFilter" -> List( + ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere"))) + ), + "endSend" -> List(ResultContext(s"$processName-startProcess-0-0", Map("input" -> variable("terefere")))) + ) + } } } @@ -71,16 +78,17 @@ class FlinkStreamingProcessTestRunnerSpec .source("startProcess", "kafka-transaction") .emptySink("endSend", "sendSmsNotExist") - val deploymentManager = + Using.resource( FlinkStreamingDeploymentManagerProviderHelper.createDeploymentManager(ConfigWithUnresolvedVersion(config)) - - val caught = intercept[IllegalArgumentException] { - Await.result( - deploymentManager.processCommand(DMTestScenarioCommand(processVersion, process, scenarioTestData)), - patienceConfig.timeout - ) + ) { deploymentManager => + val caught = intercept[IllegalArgumentException] { + Await.result( + deploymentManager.processCommand(DMTestScenarioCommand(processVersion, process, scenarioTestData)), + patienceConfig.timeout + ) + } + caught.getMessage shouldBe "Compilation errors: MissingSinkFactory(sendSmsNotExist,endSend)" } - caught.getMessage shouldBe "Compilation errors: MissingSinkFactory(sendSmsNotExist,endSend)" } private def variable(value: String): Json = diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StreamingDockerTest.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StreamingDockerTest.scala index 176a27d84da..7418573086f 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StreamingDockerTest.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StreamingDockerTest.scala @@ -30,6 +30,7 @@ trait StreamingDockerTest extends DockerTest with BeforeAndAfterAll with Matcher override def afterAll(): Unit = { kafkaClient.shutdown() logger.info("Kafka client closed") + deploymentManager.close() super.afterAll() } From b14fbd2d8f04d8bbf3c00a074d1258474b60c5ce Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 20 Jan 2025 16:32:26 +0100 Subject: [PATCH 16/48] FlinkProcessTestRunnerSpec: reusing test runner --- .../FlinkProcessTestRunner.scala | 23 +++++- .../FlinkProcessTestRunnerSpec.scala | 79 +++++++++++-------- 2 files changed, 66 insertions(+), 36 deletions(-) diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala index 84cf2585bdf..f620a53f220 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala @@ -3,10 +3,11 @@ package pl.touk.nussknacker.engine.management.testsmechanism import io.circe.Json import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.StreamMetaData import pl.touk.nussknacker.engine.api.test.ScenarioTestData import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker +import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ReflectiveMethodInvoker} import scala.concurrent.{ExecutionContext, Future} @@ -39,10 +40,28 @@ class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecu miniCluster, streamExecutionEnvironment, modelData, - canonicalProcess, + rewriteParallelismIfHigherThanMaxParallelism(canonicalProcess), scenarioTestData ) + private def rewriteParallelismIfHigherThanMaxParallelism(canonicalProcess: CanonicalProcess): CanonicalProcess = { + val scenarioParallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](canonicalProcess.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + if (scenarioParallelism > parallelism) { + canonicalProcess.copy(metaData = + canonicalProcess.metaData.copy(additionalFields = + canonicalProcess.metaData.additionalFields.copy(properties = + canonicalProcess.metaData.additionalFields.properties + (StreamMetaData.parallelismName -> parallelism.toString) + ) + ) + ) + } else { + canonicalProcess + } + } + def close(): Unit = { miniCluster.close() streamExecutionEnvironment.close() diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala index 7ed17267c4c..181f226cb5a 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala @@ -47,6 +47,7 @@ import java.util.{Date, UUID} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.concurrent.{Await, Future} +import scala.util.Using class FlinkProcessTestRunnerSpec extends AnyWordSpec @@ -78,6 +79,7 @@ class FlinkProcessTestRunnerSpec } private def runTests(useIOMonadInInterpreter: Boolean): Unit = { + val testRunner = prepareTestRunner(useIOMonadInInterpreter) "be able to return test results" in { val process = ScenarioBuilder @@ -92,15 +94,14 @@ class FlinkProcessTestRunnerSpec val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData( List( ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")), ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) ) - ), - useIOMonadInInterpreter + ) ) val nodeResults = results.nodeResults @@ -150,10 +151,9 @@ class FlinkProcessTestRunnerSpec .source(sourceNodeId, "input") .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), - useIOMonadInInterpreter ) results.nodeResults("splitId1") shouldBe List( @@ -184,10 +184,9 @@ class FlinkProcessTestRunnerSpec val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), - useIOMonadInInterpreter ) val nodeResults = results.nodeResults @@ -238,10 +237,9 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "monitor") val results = - runFlinkTest( + testRunner.runTests( process, ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), - useIOMonadInInterpreter ) val nodeResults = results.nodeResults @@ -258,7 +256,7 @@ class FlinkProcessTestRunnerSpec .filter("filter", "1 / #input.value1 >= 0".spel) .emptySink("out", "monitor") - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData( List( @@ -268,7 +266,6 @@ class FlinkProcessTestRunnerSpec createTestRecord(id = "3", value1 = 4) ) ), - useIOMonadInInterpreter ) val nodeResults = results.nodeResults @@ -315,7 +312,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "monitor") val exceptionConsumerId = UUID.randomUUID().toString - val results = runFlinkTest( + val results = runTestsWithCustomModel( process = process, scenarioTestData = ScenarioTestData( List( @@ -326,7 +323,7 @@ class FlinkProcessTestRunnerSpec ) ), useIOMonadInInterpreter, - enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) + enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId), ) val nodeResults = results.nodeResults @@ -347,7 +344,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "monitor") val run = Future { - runFlinkTest(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))), useIOMonadInInterpreter) + testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) } intercept[JobExecutionException](Await.result(run, 10 seconds)) @@ -376,7 +373,7 @@ class FlinkProcessTestRunnerSpec ) ) - val results = runFlinkTest(process, testData, useIOMonadInInterpreter) + val results = testRunner.runTests(process, testData) results.nodeResults(sourceNodeId) should have size 3 results.externalInvocationResults("out") shouldBe @@ -406,7 +403,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) - val results = runFlinkTest(process, testData, useIOMonadInInterpreter) + val results = testRunner.runTests(process, testData) results.nodeResults(sourceNodeId) should have size 1 results.externalInvocationResults("out") shouldBe @@ -427,7 +424,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) val results = - runFlinkTest(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))), useIOMonadInInterpreter) + testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) results.exceptions should have length 1 results.exceptions.head.nodeId shouldBe Some("out") @@ -447,7 +444,7 @@ class FlinkProcessTestRunnerSpec def recordWithSeconds(duration: FiniteDuration) = ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData( List( @@ -457,8 +454,7 @@ class FlinkProcessTestRunnerSpec recordWithSeconds(9 second), recordWithSeconds(20 second) ) - ), - useIOMonadInInterpreter + ) ) val nodeResults = results.nodeResults @@ -478,15 +474,14 @@ class FlinkProcessTestRunnerSpec ) .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) - val results = runFlinkTest( + val results = testRunner.runTests( process, ScenarioTestData( ScenarioTestJsonRecord( sourceNodeId, Json.obj("field1" -> Json.fromString("abc"), "field2" -> Json.fromString("def")) ) :: Nil - ), - useIOMonadInInterpreter + ) ) results.invocationResults("out").map(_.value) shouldBe List(variable("abcdef")) @@ -510,7 +505,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) val results = - runFlinkTest(process, ScenarioTestData(List(createTestRecord(value1 = valueToReturn))), useIOMonadInInterpreter) + testRunner.runTests(process, ScenarioTestData(List(createTestRecord(value1 = valueToReturn)))) results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) } @@ -533,7 +528,7 @@ class FlinkProcessTestRunnerSpec val recordTrue = createTestRecord(id = "ala") val recordFalse = createTestRecord(id = "bela") - val results = runFlinkTest(process, ScenarioTestData(List(recordTrue, recordFalse)), useIOMonadInInterpreter) + val results = testRunner.runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) val invocationResults = results.invocationResults @@ -571,7 +566,7 @@ class FlinkProcessTestRunnerSpec val recB = createTestRecord(id = "b") val recC = createTestRecord(id = "c") - val results = runFlinkTest(process, ScenarioTestData(List(recA, recB, recC)), useIOMonadInInterpreter) + val results = testRunner.runTests(process, ScenarioTestData(List(recA, recB, recC))) results.invocationResults("proc2").map(_.contextId) should contain only ( s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", @@ -580,7 +575,9 @@ class FlinkProcessTestRunnerSpec s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end2" ) - results.externalInvocationResults("proc2").map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( + results + .externalInvocationResults("proc2") + .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( "b", "a", "c", @@ -626,7 +623,7 @@ class FlinkProcessTestRunnerSpec val recordC = recordA.copy(id = "c") val recordD = recordA.copy(id = "d") - val results = runFlinkTest(process, scenarioTestData, useIOMonadInInterpreter) + val results = testRunner.runTests(process, scenarioTestData) val nodeResults = results.nodeResults nodeResults("source1") shouldBe List( @@ -673,7 +670,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) val results = - runFlinkTest(process, ScenarioTestData(List(createTestRecord(sourceId = "start"))), useIOMonadInInterpreter) + testRunner.runTests(process, ScenarioTestData(List(createTestRecord(sourceId = "start")))) results.invocationResults("out").map(_.value) shouldBe List( variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) @@ -694,7 +691,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) val run = Future { - runFlinkTest(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))), useIOMonadInInterpreter) + testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) } val dictEditorException = intercept[IllegalStateException](Await.result(run, 10 seconds)) dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" @@ -715,7 +712,7 @@ class FlinkProcessTestRunnerSpec ) .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - val results = runFlinkTest( + val results = runTestsWithCustomModel( process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))), useIOMonadInInterpreter, @@ -745,10 +742,9 @@ class FlinkProcessTestRunnerSpec val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) - val results = runFlinkTest( + val results = testRunner.runTests( resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), - useIOMonadInInterpreter ) results.exceptions.length shouldBe 0 } @@ -761,7 +757,7 @@ class FlinkProcessTestRunnerSpec ): ScenarioTestJsonRecord = ScenarioTestJsonRecord(sourceId, Json.fromString(s"$id|$value1|2|3|4|5|6")) - private def runFlinkTest( + private def runTestsWithCustomModel( process: CanonicalProcess, scenarioTestData: ScenarioTestData, useIOMonadInInterpreter: Boolean, @@ -786,6 +782,21 @@ class FlinkProcessTestRunnerSpec .runTests(process, scenarioTestData) } + private def prepareTestRunner(useIOMonadInInterpreter: Boolean) = { + val config = ConfigFactory + .load("application.conf") + .withValue("globalParameters.useIOMonadInInterpreter", ConfigValueFactory.fromAnyRef(useIOMonadInInterpreter)) + + // We need to set context loader to avoid forking in sbt + val modelData = ModelData.duringExecution( + ModelConfigs(config, AdditionalModelConfigs(Map.empty)), + ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround), + resolveConfigs = false + ) + + new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + } + private def nodeResult(count: Int, vars: (String, Any)*): ResultContext[_] = nodeResult(count, sourceNodeId, vars: _*) From 09a5efb2b7dfbfe8a9e57ef670f9d92172a510d3 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 10:48:17 +0100 Subject: [PATCH 17/48] Changed docs next to method invoker --- .../management/testsmechanism/FlinkProcessTestRunner.scala | 5 +++-- .../management/testsmechanism/FlinkProcessVerifier.scala | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala index f620a53f220..40943fced23 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala @@ -19,8 +19,9 @@ class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecu private val miniCluster = TestsMechanismMiniClusterFactory.createConfiguredMiniCluster(parallelism) - // We use reflection to avoid bundling of flinkExecutor.jar inside flinkDeploymentManager assembly jar - // TODO: use provided dependency instead + // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar + // because it is already in separate assembly for purpose of sending it to Flink during deployment. + // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded private val methodInvoker = new ReflectiveMethodInvoker[TestResults[Json]]( modelData.modelClassLoader.classLoader, "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala index 2df63253ac1..9e432e8b3df 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala @@ -14,8 +14,9 @@ import scala.util.control.NonFatal class FlinkProcessVerifier(modelData: ModelData) extends LazyLogging { - // We use reflection to avoid bundling of flinkExecutor.jar inside flinkDeploymentManager assembly jar - // TODO: use provided dependency instead + // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar + // because it is already in separate assembly for purpose of sending it to Flink during deployment. + // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded private val methodInvoker = new ReflectiveMethodInvoker[Unit]( modelData.modelClassLoader.classLoader, "pl.touk.nussknacker.engine.process.runner.FlinkVerificationMain", From 6325dc49331039c802ff91c1c84e92551f6d03cd Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 11:04:54 +0100 Subject: [PATCH 18/48] removed SimpleProcessConfigCreator to allow run tests from IDE --- build.sbt | 5 +- ...er.engine.api.process.ProcessConfigCreator | 1 - .../FlinkProcessTestRunnerSpec.scala | 2 - .../SimpleProcessConfigCreator.scala | 59 ------------------- 4 files changed, 3 insertions(+), 64 deletions(-) delete mode 100644 engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator delete mode 100644 engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala diff --git a/build.sbt b/build.sbt index e43de9f0c93..b19adb77a37 100644 --- a/build.sbt +++ b/build.sbt @@ -631,7 +631,8 @@ lazy val flinkDeploymentManager = (project in flink("management")) componentsApi % Provided, httpUtils % Provided, flinkScalaUtils % Provided, - flinkExecutor % Test, + // test->test dependency is needed to load SimpleProcessConfigCreator + flinkExecutor % "test,test->test", flinkTestUtils % "it,test", kafkaTestUtils % "it,test" ) @@ -721,7 +722,7 @@ lazy val flinkTests = (project in flink("tests")) kafkaTestUtils % Test, flinkComponentsTestkit % Test, flinkDeploymentManager % Test, - // TODO: cleanup kafka testsmechanism tests in order to remove test->test dependency + // test->test dependencies are needed to load components from these modules flinkKafkaComponentsUtils % "test->test", flinkSchemedKafkaComponentsUtils % "test->test", // for local development diff --git a/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator b/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator deleted file mode 100644 index dc12680791f..00000000000 --- a/engine/flink/management/src/test/resources/META-INF/services/pl.touk.nussknacker.engine.api.process.ProcessConfigCreator +++ /dev/null @@ -1 +0,0 @@ -pl.touk.nussknacker.engine.management.testsmechanism.SimpleProcessConfigCreator diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala index 181f226cb5a..9e64878a596 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala @@ -429,8 +429,6 @@ class FlinkProcessTestRunnerSpec results.exceptions should have length 1 results.exceptions.head.nodeId shouldBe Some("out") results.exceptions.head.throwable.getMessage should include("message: / by zero") - - SimpleProcessConfigCreator.sinkForIntsResultsHolder.results should have length 0 } "be able to test process with time windows" in { diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala deleted file mode 100644 index 4e142dcdf25..00000000000 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/SimpleProcessConfigCreator.scala +++ /dev/null @@ -1,59 +0,0 @@ -package pl.touk.nussknacker.engine.management.testsmechanism - -import pl.touk.nussknacker.engine.api.process._ -import pl.touk.nussknacker.engine.api.{CustomStreamTransformer, Service} -import pl.touk.nussknacker.engine.management.testsmechanism.SimpleProcessConfigCreator.{ - sinkForIntsResultsHolder, - valueMonitorResultsHolder -} -import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ -import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder - -import java.net.ConnectException - -class SimpleProcessConfigCreator extends EmptyProcessConfigCreator { - - override def services(modelDependencies: ProcessObjectDependencies): Map[String, WithCategories[Service]] = - Map( - "logService" -> WithCategories(LogService, "c1"), - "throwingService" -> WithCategories(new ThrowingService(new RuntimeException("Thrown as expected")), "c1"), - "throwingTransientService" -> WithCategories(new ThrowingService(new ConnectException()), "c1"), - "returningDependentTypeService" -> WithCategories(ReturningDependentTypeService, "c1"), - "collectingEager" -> WithCategories(CollectingEagerService, "c1"), - "returningComponentUseCaseService" -> WithCategories(ReturningComponentUseCaseService, "c1") - ) - - override def sinkFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SinkFactory]] = Map( - "monitor" -> WithCategories(SinkFactory.noParam(MonitorEmptySink), "c2"), - "valueMonitor" -> WithCategories(SinkForAny(valueMonitorResultsHolder), "c2"), - "sinkForInts" -> WithCategories.anyCategory(SinkForInts(sinkForIntsResultsHolder)) - ) - - override def customStreamTransformers( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[CustomStreamTransformer]] = Map( - "stateCustom" -> WithCategories.anyCategory(StateCustomNode), - "transformWithTime" -> WithCategories.anyCategory(TransformerWithTime), - "joinBranchExpression" -> WithCategories.anyCategory(CustomJoinUsingBranchExpressions), - "transformerAddingComponentUseCase" -> WithCategories.anyCategory(TransformerAddingComponentUseCase) - ) - - override def sourceFactories( - modelDependencies: ProcessObjectDependencies - ): Map[String, WithCategories[SourceFactory]] = Map( - "input" -> WithCategories(simpleRecordSource(Nil), "cat2"), - "jsonInput" -> WithCategories(jsonSource, "cat2"), - "typedJsonInput" -> WithCategories(TypedJsonSource, "cat2"), - "genericSourceWithCustomVariables" -> WithCategories.anyCategory(GenericSourceWithCustomVariables) - ) - -} - -object SimpleProcessConfigCreator extends Serializable { - - val valueMonitorResultsHolder = new TestResultsHolder[AnyRef] - val sinkForIntsResultsHolder = new TestResultsHolder[java.lang.Integer] - -} From cd419c663e2652dca0eb8f70a5b4f201e1c97b9c Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 12:11:22 +0100 Subject: [PATCH 19/48] Test for multiple tests run + refactor --- .../process/runner/FlinkProcessMain.scala | 18 ++++++-- .../engine/process/runner/FlinkRunner.scala | 20 --------- .../engine/process/runner/FlinkTestMain.scala | 8 ++-- .../runner/FlinkVerificationMain.scala | 8 ++-- ...ala => FlinkMiniClusterJobSubmitter.scala} | 8 ++-- .../TestsMechanismMiniClusterFactory.scala | 6 +-- .../FlinkProcessTestRunnerSpec.scala | 41 ++++++++++++++++++- 7 files changed, 69 insertions(+), 40 deletions(-) delete mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkRunner.scala rename engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/{FlinkStubbedRunner.scala => FlinkMiniClusterJobSubmitter.scala} (78%) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkProcessMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkProcessMain.scala index da869ea2d97..66889fe2521 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkProcessMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkProcessMain.scala @@ -1,18 +1,21 @@ package pl.touk.nussknacker.engine.process.runner -import java.io.File import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.scalalogging.LazyLogging import org.apache.flink.api.common.ExecutionConfig -import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} import pl.touk.nussknacker.engine.api.{CirceUtil, ProcessVersion} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData +import pl.touk.nussknacker.engine.marshall.ScenarioParser import pl.touk.nussknacker.engine.process.ExecutionConfigPreparer +import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} +import java.io.File +import java.nio.charset.StandardCharsets +import scala.util.Using import scala.util.control.NonFatal -trait FlinkProcessMain[Env] extends FlinkRunner with LazyLogging { +trait FlinkProcessMain[Env] extends LazyLogging { def main(argsWithHack: Array[String]): Unit = { try { @@ -61,6 +64,15 @@ trait FlinkProcessMain[Env] extends FlinkRunner with LazyLogging { prepareExecutionConfig: ExecutionConfigPreparer ): Unit + protected def readProcessFromArg(arg: String): CanonicalProcess = { + val canonicalJson = if (arg.startsWith("@")) { + Using.resource(scala.io.Source.fromFile(arg.substring(1), StandardCharsets.UTF_8.name()))(_.mkString) + } else { + arg + } + ScenarioParser.parseUnsafe(canonicalJson) + } + private def parseProcessVersion(json: String): ProcessVersion = CirceUtil.decodeJsonUnsafe[ProcessVersion](json, "invalid scenario version") diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkRunner.scala deleted file mode 100644 index 9089c1d0807..00000000000 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkRunner.scala +++ /dev/null @@ -1,20 +0,0 @@ -package pl.touk.nussknacker.engine.process.runner - -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.marshall.ScenarioParser - -import java.nio.charset.StandardCharsets -import scala.util.Using - -trait FlinkRunner { - - protected def readProcessFromArg(arg: String): CanonicalProcess = { - val canonicalJson = if (arg.startsWith("@")) { - Using.resource(scala.io.Source.fromFile(arg.substring(1), StandardCharsets.UTF_8.name()))(_.mkString) - } else { - arg - } - ScenarioParser.parseUnsafe(canonicalJson) - } - -} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala index eee9b7afb2e..b6b5dd785a5 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala @@ -12,7 +12,7 @@ import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{AdditionalModelConfigs, DeploymentData} import pl.touk.nussknacker.engine.process.compiler.TestFlinkProcessCompilerDataFactory import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar -import pl.touk.nussknacker.engine.process.testmechanism.FlinkStubbedRunner +import pl.touk.nussknacker.engine.process.testmechanism.FlinkMiniClusterJobSubmitter import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults import pl.touk.nussknacker.engine.testmode.{ @@ -21,7 +21,7 @@ import pl.touk.nussknacker.engine.testmode.{ TestServiceInvocationCollector } -object FlinkTestMain extends FlinkRunner { +object FlinkTestMain { def run( miniCluster: MiniCluster, @@ -58,7 +58,7 @@ class FlinkTestMain( deploymentData: DeploymentData ) { - private val stubbedRunner = new FlinkStubbedRunner(miniCluster, env) + private val stubbedRunner = new FlinkMiniClusterJobSubmitter(miniCluster, env) def runTest: TestResults[Json] = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener @@ -67,7 +67,7 @@ class FlinkTestMain( val registrar = prepareRegistrar(collectingListener, scenarioTestData) registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute(process.name, SavepointRestoreSettings.none(), modelData.modelClassLoader) + stubbedRunner.submitJobAndCleanEnv(process.name, SavepointRestoreSettings.none(), modelData.modelClassLoader) collectingListener.results } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala index 86147380cda..fb38f1576ce 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala @@ -9,11 +9,11 @@ import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.process.compiler.VerificationFlinkProcessCompilerDataFactory import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar -import pl.touk.nussknacker.engine.process.testmechanism.FlinkStubbedRunner +import pl.touk.nussknacker.engine.process.testmechanism.FlinkMiniClusterJobSubmitter import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestServiceInvocationCollector} -object FlinkVerificationMain extends FlinkRunner { +object FlinkVerificationMain { def run( miniCluster: MiniCluster, @@ -46,7 +46,7 @@ class FlinkVerificationMain( savepointPath: String ) { - private val stubbedRunner = new FlinkStubbedRunner(miniCluster, env) + private val stubbedRunner = new FlinkMiniClusterJobSubmitter(miniCluster, env) def runTest(): Unit = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener @@ -54,7 +54,7 @@ class FlinkVerificationMain( val registrar = prepareRegistrar() registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.execute( + stubbedRunner.submitJobAndCleanEnv( process.name, SavepointRestoreSettings.forPath(savepointPath, true), modelData.modelClassLoader diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala similarity index 78% rename from engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala rename to engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala index 446a182879d..2357418a1e0 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkStubbedRunner.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala @@ -8,15 +8,15 @@ import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import scala.jdk.CollectionConverters._ -// FIXME abr: rename -// we use own LocalFlinkMiniCluster, instead of LocalExecutionEnvironment, to be able to pass own classpath... -final class FlinkStubbedRunner(miniCluster: MiniCluster, env: StreamExecutionEnvironment) { +// We use MiniCluster directly, instead of LocalExecutionEnvironment, to be able to pass own classpath... +final class FlinkMiniClusterJobSubmitter(miniCluster: MiniCluster, env: StreamExecutionEnvironment) { - def execute( + def submitJobAndCleanEnv( scenarioName: ProcessName, savepointRestoreSettings: SavepointRestoreSettings, modelClassLoader: ModelClassLoader ): Unit = { + // This step clean env transformations. It allows to reuse the same StreamExecutionEnvironment many times val streamGraph = env.getStreamGraph streamGraph.setJobName(scenarioName.value) val jobGraph = streamGraph.getJobGraph diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala index f0a4aed9437..0fa6174d1a7 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala @@ -6,13 +6,13 @@ import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfigurati object TestsMechanismMiniClusterFactory { - def createConfiguredMiniCluster(parallelism: Int): MiniCluster = { - val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = parallelism) + def createConfiguredMiniCluster(nomTaskSlots: Int): MiniCluster = { + val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = nomTaskSlots) // it is required for proper working of HadoopFileSystem FileSystem.initialize(miniClusterConfiguration, null) - createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = parallelism) + createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = nomTaskSlots) } private def prepareMiniClusterConfiguration(numTaskSlots: Int) = { diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala index 9e64878a596..413e17fd436 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala @@ -39,7 +39,7 @@ import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunn } import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ import pl.touk.nussknacker.engine.testmode.TestProcess._ -import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ThreadUtils} +import pl.touk.nussknacker.engine.util.MetaDataExtractor import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} @@ -47,7 +47,6 @@ import java.util.{Date, UUID} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.concurrent.{Await, Future} -import scala.util.Using class FlinkProcessTestRunnerSpec extends AnyWordSpec @@ -144,6 +143,44 @@ class FlinkProcessTestRunnerSpec LogService.invocationsCount.get() shouldBe 0 } + "be able to run tests multiple time on the same test runner" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) + + val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") + + def runTestAndVerify() = { + val results = testRunner.runTests( + process, + ScenarioTestData( + List( + ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) + ) + ) + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input)) + nodeResults("out") shouldBe List(nodeResult(0, "input" -> input)) + + val invocationResults = results.invocationResults + + invocationResults("out") shouldBe + List(ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable(11))) + + results.externalInvocationResults("out") shouldBe List( + ExternalInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "valueMonitor", variable(11)) + ) + } + + runTestAndVerify() + runTestAndVerify() + } + "collect results for split" in { val process = ScenarioBuilder From a9eca37b1059cfb389ea89818d3f507854be93cf Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 15:08:16 +0100 Subject: [PATCH 20/48] State verification mechanism reusing mini cluster + configuration + refactors --- .../test/mock/MockDeploymentManager.scala | 9 +- ...DevelopmentDeploymentManagerProvider.scala | 19 ++- .../MockableDeploymentManagerProvider.scala | 26 +++- .../engine/process/runner/FlinkTestMain.scala | 94 ------------ .../runner/FlinkVerificationMain.scala | 72 --------- .../AdHocMiniClusterFallbackHandler.scala | 36 +++++ .../scenariotesting/FlinkTestMain.scala | 80 ++++++++++ .../FlinkVerificationMain.scala | 60 ++++++++ .../ScenarioTestingMiniClusterFactory.scala} | 4 +- .../ScenarioTestingMiniClusterWrapper.scala | 81 ++++++++++ ...ngStreamExecutionEnvironmentFactory.scala} | 4 +- .../FlinkMiniClusterJobSubmitter.scala | 38 ----- .../engine/management/FlinkConfig.scala | 23 ++- .../management/FlinkDeploymentManager.scala | 29 +++- .../engine/management/FlinkRestManager.scala | 19 ++- .../FlinkScheduledExecutionPerformer.scala | 1 + ...nkStreamingDeploymentManagerProvider.scala | 1 + .../FlinkProcessTestRunner.scala | 39 +++++ .../FlinkProcessVerifier.scala | 52 +++++++ ...arioTestingMiniClusterWrapperFactory.scala | 34 +++++ .../FlinkProcessTestRunner.scala | 71 --------- .../testsmechanism/FlinkProcessVerifier.scala | 69 --------- .../FlinkProcessTestRunnerSpec.scala | 138 +++++++++--------- .../kafka/KafkaScenarioTestingSpec.scala} | 23 ++- .../SchemedKafkaScenarioTestingSpec.scala} | 28 ++-- 25 files changed, 582 insertions(+), 468 deletions(-) delete mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala delete mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala create mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala create mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala create mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala rename engine/flink/{management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala => executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala} (92%) create mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala rename engine/flink/{management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala => executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingStreamExecutionEnvironmentFactory.scala} (82%) delete mode 100644 engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunner.scala create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessVerifier.scala create mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala delete mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala delete mode 100644 engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala rename engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/{testsmechanism => scenariotesting}/FlinkProcessTestRunnerSpec.scala (89%) rename engine/flink/tests/src/test/scala/pl/touk/nussknacker/{testsmechanism/kafka/TestFromFileSpec.scala => scenariotesting/kafka/KafkaScenarioTestingSpec.scala} (89%) rename engine/flink/tests/src/test/scala/pl/touk/nussknacker/{testsmechanism/schemedkafka/TestWithTestDataSpec.scala => scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala} (91%) diff --git a/designer/server/src/test/scala/pl/touk/nussknacker/test/mock/MockDeploymentManager.scala b/designer/server/src/test/scala/pl/touk/nussknacker/test/mock/MockDeploymentManager.scala index 7f2a7b58edf..61fe7fa79c9 100644 --- a/designer/server/src/test/scala/pl/touk/nussknacker/test/mock/MockDeploymentManager.scala +++ b/designer/server/src/test/scala/pl/touk/nussknacker/test/mock/MockDeploymentManager.scala @@ -13,7 +13,11 @@ import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.api.{ProcessVersion, StreamMetaData} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment._ -import pl.touk.nussknacker.engine.management.{FlinkDeploymentManager, FlinkStreamingDeploymentManagerProvider} +import pl.touk.nussknacker.engine.management.{ + FlinkDeploymentManager, + FlinkStreamingDeploymentManagerProvider, + ScenarioTestingConfig +} import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.test.config.ConfigWithScalaVersion import pl.touk.nussknacker.test.utils.domain.TestFactory @@ -55,7 +59,8 @@ class MockDeploymentManager( SttpBackendStub.asynchronousFuture ), shouldVerifyBeforeDeploy = false, - mainClassName = "UNUSED" + mainClassName = "UNUSED", + scenarioTestingConfig = ScenarioTestingConfig() ) { import MockDeploymentManager._ diff --git a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala index bcfd4f47e8c..ac89554a763 100644 --- a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala +++ b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/DevelopmentDeploymentManagerProvider.scala @@ -20,8 +20,11 @@ import pl.touk.nussknacker.engine.api.deployment.simple.{SimpleProcessStateDefin import pl.touk.nussknacker.engine.api.process.ProcessName import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment._ -import pl.touk.nussknacker.engine.management.FlinkStreamingPropertiesConfig -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner +import pl.touk.nussknacker.engine.management.{FlinkStreamingPropertiesConfig, ScenarioTestingConfig} +import pl.touk.nussknacker.engine.management.scenariotesting.{ + FlinkProcessTestRunner, + ScenarioTestingMiniClusterWrapperFactory +} import java.net.URI import java.util.UUID @@ -50,8 +53,14 @@ class DevelopmentDeploymentManager(actorSystem: ActorSystem, modelData: BaseMode private val memory: TrieMap[ProcessName, StatusDetails] = TrieMap[ProcessName, StatusDetails]() private val random = new scala.util.Random() + private lazy val scenarioTestingMiniClusterWrapperOpt = + ScenarioTestingMiniClusterWrapperFactory.createIfConfigured( + modelData.asInvokableModelData.modelClassLoader, + ScenarioTestingConfig() + ) + private lazy val flinkTestRunner = - new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration()) + new FlinkProcessTestRunner(modelData.asInvokableModelData, scenarioTestingMiniClusterWrapperOpt) implicit private class ProcessStateExpandable(processState: StatusDetails) { @@ -154,7 +163,9 @@ class DevelopmentDeploymentManager(actorSystem: ActorSystem, modelData: BaseMode notImplemented } - override def close(): Unit = {} + override def close(): Unit = { + scenarioTestingMiniClusterWrapperOpt.foreach(_.close()) + } private def changeState(name: ProcessName, stateStatus: StateStatus): Unit = memory.get(name).foreach { processState => diff --git a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala index 2ac56d59912..410323ba0be 100644 --- a/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala +++ b/engine/development/deploymentManager/src/main/scala/pl/touk/nussknacker/development/manager/MockableDeploymentManagerProvider.scala @@ -4,7 +4,6 @@ import cats.data.Validated.valid import cats.data.ValidatedNel import com.typesafe.config.Config import io.circe.Json -import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.development.manager.MockableDeploymentManagerProvider.MockableDeploymentManager import pl.touk.nussknacker.engine.ModelData.BaseModelDataExt import pl.touk.nussknacker.engine._ @@ -13,8 +12,11 @@ import pl.touk.nussknacker.engine.api.deployment._ import pl.touk.nussknacker.engine.api.deployment.simple.{SimpleProcessStateDefinitionManager, SimpleStateStatus} import pl.touk.nussknacker.engine.api.process.{ProcessIdWithName, ProcessName, VersionId} import pl.touk.nussknacker.engine.deployment.ExternalDeploymentId -import pl.touk.nussknacker.engine.management.FlinkStreamingPropertiesConfig -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner +import pl.touk.nussknacker.engine.management.scenariotesting.{ + FlinkProcessTestRunner, + ScenarioTestingMiniClusterWrapperFactory +} +import pl.touk.nussknacker.engine.management.{FlinkStreamingPropertiesConfig, ScenarioTestingConfig} import pl.touk.nussknacker.engine.newdeployment.DeploymentId import pl.touk.nussknacker.engine.testing.StubbingCommands import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults @@ -59,10 +61,17 @@ object MockableDeploymentManagerProvider { with ManagerSpecificScenarioActivitiesStoredByManager with StubbingCommands { - private lazy val testRunnerOpt = - modelDataOpt.map(modelData => - new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration()) + private lazy val scenarioTestingMiniClusterWrapperOpt = modelDataOpt.flatMap { modelData => + ScenarioTestingMiniClusterWrapperFactory.createIfConfigured( + modelData.asInvokableModelData.modelClassLoader, + ScenarioTestingConfig() ) + } + + private lazy val testRunnerOpt = + modelDataOpt.map { modelData => + new FlinkProcessTestRunner(modelData.asInvokableModelData, scenarioTestingMiniClusterWrapperOpt) + } override def resolve( idWithName: ProcessIdWithName, @@ -128,7 +137,10 @@ object MockableDeploymentManagerProvider { ): Future[List[ScenarioActivity]] = Future.successful(MockableDeploymentManager.managerSpecificScenarioActivities.get()) - override def close(): Unit = {} + override def close(): Unit = { + scenarioTestingMiniClusterWrapperOpt.foreach(_.close()) + } + } // note: At the moment this manager cannot be used in tests which are executed in parallel. It can be obviously diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala deleted file mode 100644 index b6b5dd785a5..00000000000 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkTestMain.scala +++ /dev/null @@ -1,94 +0,0 @@ -package pl.touk.nussknacker.engine.process.runner - -import io.circe.Json -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings -import org.apache.flink.runtime.minicluster.MiniCluster -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.process.ProcessName -import pl.touk.nussknacker.engine.api.test.ScenarioTestData -import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion} -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.deployment.{AdditionalModelConfigs, DeploymentData} -import pl.touk.nussknacker.engine.process.compiler.TestFlinkProcessCompilerDataFactory -import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar -import pl.touk.nussknacker.engine.process.testmechanism.FlinkMiniClusterJobSubmitter -import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} -import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.testmode.{ - ResultsCollectingListener, - ResultsCollectingListenerHolder, - TestServiceInvocationCollector -} - -object FlinkTestMain { - - def run( - miniCluster: MiniCluster, - env: StreamExecutionEnvironment, - modelData: ModelData, - process: CanonicalProcess, - scenarioTestData: ScenarioTestData - ): TestResults[Json] = { - val processVersion = ProcessVersion.empty.copy(processName = - ProcessName("snapshot version") - ) // testing process may be unreleased, so it has no version - new FlinkTestMain( - miniCluster, - env, - modelData, - process, - scenarioTestData, - processVersion, - DeploymentData.empty.copy(additionalModelConfigs = - AdditionalModelConfigs(modelData.additionalConfigsFromProvider) - ) - ).runTest - } - -} - -class FlinkTestMain( - miniCluster: MiniCluster, - env: StreamExecutionEnvironment, - modelData: ModelData, - process: CanonicalProcess, - scenarioTestData: ScenarioTestData, - processVersion: ProcessVersion, - deploymentData: DeploymentData -) { - - private val stubbedRunner = new FlinkMiniClusterJobSubmitter(miniCluster, env) - - def runTest: TestResults[Json] = { - val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener - try { - val resultCollector = new TestServiceInvocationCollector(collectingListener) - val registrar = prepareRegistrar(collectingListener, scenarioTestData) - - registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.submitJobAndCleanEnv(process.name, SavepointRestoreSettings.none(), modelData.modelClassLoader) - collectingListener.results - } finally { - collectingListener.clean() - } - } - - protected def prepareRegistrar( - collectingListener: ResultsCollectingListener[Json], - scenarioTestData: ScenarioTestData - ): FlinkProcessRegistrar = { - FlinkProcessRegistrar( - TestFlinkProcessCompilerDataFactory( - process, - scenarioTestData, - modelData, - JobData(process.metaData, processVersion), - collectingListener - ), - FlinkJobConfig.parse(modelData.modelConfig).copy(rocksDB = None), - ExecutionConfigPreparer.defaultChain(modelData) - ) - } - -} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala deleted file mode 100644 index fb38f1576ce..00000000000 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/runner/FlinkVerificationMain.scala +++ /dev/null @@ -1,72 +0,0 @@ -package pl.touk.nussknacker.engine.process.runner - -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings -import org.apache.flink.runtime.minicluster.MiniCluster -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.ProcessVersion -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.deployment.DeploymentData -import pl.touk.nussknacker.engine.process.compiler.VerificationFlinkProcessCompilerDataFactory -import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar -import pl.touk.nussknacker.engine.process.testmechanism.FlinkMiniClusterJobSubmitter -import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} -import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestServiceInvocationCollector} - -object FlinkVerificationMain { - - def run( - miniCluster: MiniCluster, - env: StreamExecutionEnvironment, - modelData: ModelData, - process: CanonicalProcess, - processVersion: ProcessVersion, - deploymentData: DeploymentData, - savepointPath: String - ): Unit = - new FlinkVerificationMain( - miniCluster, - env, - modelData, - process, - processVersion, - deploymentData, - savepointPath - ).runTest() - -} - -class FlinkVerificationMain( - miniCluster: MiniCluster, - env: StreamExecutionEnvironment, - modelData: ModelData, - process: CanonicalProcess, - processVersion: ProcessVersion, - deploymentData: DeploymentData, - savepointPath: String -) { - - private val stubbedRunner = new FlinkMiniClusterJobSubmitter(miniCluster, env) - - def runTest(): Unit = { - val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener - val resultCollector = new TestServiceInvocationCollector(collectingListener) - val registrar = prepareRegistrar() - - registrar.register(env, process, processVersion, deploymentData, resultCollector) - stubbedRunner.submitJobAndCleanEnv( - process.name, - SavepointRestoreSettings.forPath(savepointPath, true), - modelData.modelClassLoader - ) - } - - protected def prepareRegistrar(): FlinkProcessRegistrar = { - FlinkProcessRegistrar( - VerificationFlinkProcessCompilerDataFactory(process, modelData), - FlinkJobConfig.parse(modelData.modelConfig), - ExecutionConfigPreparer.defaultChain(modelData) - ) - } - -} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala new file mode 100644 index 00000000000..92126f5e063 --- /dev/null +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala @@ -0,0 +1,36 @@ +package pl.touk.nussknacker.engine.process.scenariotesting + +import org.apache.flink.configuration.Configuration +import pl.touk.nussknacker.engine.api.StreamMetaData +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.util.MetaDataExtractor + +// This class handles a legacy ad-hoc way to create minicluster. +// After we fully switch to single mini cluster approach, it should be removed +object AdHocMiniClusterFallbackHandler { + + def handleAdHocMniClusterFallback[R]( + reusableMiniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], + scenario: CanonicalProcess + )(f: ScenarioTestingMiniClusterWrapper => R): R = { + val miniClusterWrapper = reusableMiniClusterWrapperOpt.getOrElse { + createAdHocMiniClusterWrapper(scenario) + } + try { + f(miniClusterWrapper) + } finally { + if (reusableMiniClusterWrapperOpt.isEmpty) { + miniClusterWrapper.close() + } + } + } + + private def createAdHocMiniClusterWrapper(process: CanonicalProcess) = { + val scenarioParallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) + .parallelism + .getOrElse(1) + ScenarioTestingMiniClusterWrapper.create(scenarioParallelism, new Configuration()) + } + +} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala new file mode 100644 index 00000000000..803a442a376 --- /dev/null +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala @@ -0,0 +1,80 @@ +package pl.touk.nussknacker.engine.process.scenariotesting + +import io.circe.Json +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.test.ScenarioTestData +import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion} +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.deployment.{AdditionalModelConfigs, DeploymentData} +import pl.touk.nussknacker.engine.process.compiler.TestFlinkProcessCompilerDataFactory +import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar +import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} +import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults +import pl.touk.nussknacker.engine.testmode.{ + ResultsCollectingListener, + ResultsCollectingListenerHolder, + TestServiceInvocationCollector +} + +object FlinkTestMain { + + def run( + miniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], + modelData: ModelData, + scenario: CanonicalProcess, + scenarioTestData: ScenarioTestData + ): TestResults[Json] = { + new FlinkTestMain(miniClusterWrapperOpt, modelData).testScenario(scenario, scenarioTestData) + } + +} + +class FlinkTestMain(miniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], modelData: ModelData) { + + def testScenario(scenario: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { + val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener + try { + AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback(miniClusterWrapperOpt, scenario) { + miniClusterWrapper => + val alignedScenario = miniClusterWrapper.alignParallelism(scenario) + val resultCollector = new TestServiceInvocationCollector(collectingListener) + // ProcessVersion can't be passed from DM because testing mechanism can be used with not saved scenario + val processVersion = ProcessVersion.empty.copy(processName = alignedScenario.name) + val deploymentData = DeploymentData.empty.copy(additionalModelConfigs = + AdditionalModelConfigs(modelData.additionalConfigsFromProvider) + ) + val registrar = prepareRegistrar(collectingListener, alignedScenario, scenarioTestData, processVersion) + registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) + miniClusterWrapper.submitJobAndCleanEnv( + alignedScenario.name, + SavepointRestoreSettings.none(), + modelData.modelClassLoader + ) + collectingListener.results + } + } finally { + collectingListener.clean() + } + } + + protected def prepareRegistrar( + collectingListener: ResultsCollectingListener[Json], + process: CanonicalProcess, + scenarioTestData: ScenarioTestData, + processVersion: ProcessVersion, + ): FlinkProcessRegistrar = { + FlinkProcessRegistrar( + TestFlinkProcessCompilerDataFactory( + process, + scenarioTestData, + modelData, + JobData(process.metaData, processVersion), + collectingListener + ), + FlinkJobConfig.parse(modelData.modelConfig).copy(rocksDB = None), + ExecutionConfigPreparer.defaultChain(modelData) + ) + } + +} diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala new file mode 100644 index 00000000000..097535f9a7f --- /dev/null +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala @@ -0,0 +1,60 @@ +package pl.touk.nussknacker.engine.process.scenariotesting + +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.ProcessVersion +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.deployment.DeploymentData +import pl.touk.nussknacker.engine.process.compiler.VerificationFlinkProcessCompilerDataFactory +import pl.touk.nussknacker.engine.process.registrar.FlinkProcessRegistrar +import pl.touk.nussknacker.engine.process.{ExecutionConfigPreparer, FlinkJobConfig} +import pl.touk.nussknacker.engine.testmode.{ResultsCollectingListenerHolder, TestServiceInvocationCollector} + +object FlinkVerificationMain { + + def run( + miniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], + modelData: ModelData, + scenario: CanonicalProcess, + processVersion: ProcessVersion, + savepointPath: String + ): Unit = + new FlinkVerificationMain(miniClusterWrapperOpt, modelData).runTest(scenario, processVersion, savepointPath) + +} + +class FlinkVerificationMain( + miniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], + modelData: ModelData, +) { + + def runTest(scenario: CanonicalProcess, processVersion: ProcessVersion, savepointPath: String): Unit = { + val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener + try { + AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback(miniClusterWrapperOpt, scenario) { + miniClusterWrapper => + val resultCollector = new TestServiceInvocationCollector(collectingListener) + val registrar = prepareRegistrar(scenario) + val deploymentData = DeploymentData.empty + + registrar.register(miniClusterWrapper.env, scenario, processVersion, deploymentData, resultCollector) + miniClusterWrapper.submitJobAndCleanEnv( + scenario.name, + SavepointRestoreSettings.forPath(savepointPath, true), + modelData.modelClassLoader + ) + } + } finally { + collectingListener.clean() + } + } + + protected def prepareRegistrar(scenario: CanonicalProcess): FlinkProcessRegistrar = { + FlinkProcessRegistrar( + VerificationFlinkProcessCompilerDataFactory(scenario, modelData), + FlinkJobConfig.parse(modelData.modelConfig), + ExecutionConfigPreparer.defaultChain(modelData) + ) + } + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala similarity index 92% rename from engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala rename to engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala index 0fa6174d1a7..eb978ff6bca 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismMiniClusterFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala @@ -1,10 +1,10 @@ -package pl.touk.nussknacker.engine.management.testsmechanism +package pl.touk.nussknacker.engine.process.scenariotesting import org.apache.flink.configuration.{Configuration, CoreOptions, RestOptions, TaskManagerOptions} import org.apache.flink.core.fs.FileSystem import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} -object TestsMechanismMiniClusterFactory { +object ScenarioTestingMiniClusterFactory { def createConfiguredMiniCluster(nomTaskSlots: Int): MiniCluster = { val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = nomTaskSlots) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala new file mode 100644 index 00000000000..caef70c568e --- /dev/null +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -0,0 +1,81 @@ +package pl.touk.nussknacker.engine.process.scenariotesting + +import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} +import org.apache.flink.runtime.minicluster.MiniCluster +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import pl.touk.nussknacker.engine.api.StreamMetaData +import pl.touk.nussknacker.engine.api.process.ProcessName +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.util.MetaDataExtractor +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader + +import scala.jdk.CollectionConverters._ + +// We use MiniCluster directly, instead of LocalExecutionEnvironment, to be able to pass own classpath... +final class ScenarioTestingMiniClusterWrapper( + miniCluster: MiniCluster, + val env: StreamExecutionEnvironment, + parallelism: Int +) extends AutoCloseable { + + def alignParallelism(canonicalProcess: CanonicalProcess): CanonicalProcess = { + val scenarioParallelism = MetaDataExtractor + .extractTypeSpecificDataOrDefault[StreamMetaData](canonicalProcess.metaData, StreamMetaData()) + .parallelism + if (scenarioParallelism.exists(_ > parallelism)) { + canonicalProcess.copy(metaData = + canonicalProcess.metaData.copy(additionalFields = + canonicalProcess.metaData.additionalFields.copy(properties = + canonicalProcess.metaData.additionalFields.properties + (StreamMetaData.parallelismName -> parallelism.toString) + ) + ) + ) + } else { + canonicalProcess + } + } + + def submitJobAndCleanEnv( + scenarioName: ProcessName, + savepointRestoreSettings: SavepointRestoreSettings, + modelClassLoader: ModelClassLoader + ): Unit = { + // This step clean env transformations. It allows to reuse the same StreamExecutionEnvironment many times + val streamGraph = env.getStreamGraph + streamGraph.setJobName(scenarioName.value) + val jobGraph = streamGraph.getJobGraph + setupJobGraph(jobGraph, savepointRestoreSettings, modelClassLoader) + + val id = miniCluster.submitJob(jobGraph).get().getJobID + miniCluster.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) + } + + private def setupJobGraph( + jobGraph: JobGraph, + savepointRestoreSettings: SavepointRestoreSettings, + modelClassLoader: ModelClassLoader + ): Unit = { + jobGraph.setClasspaths(modelClassLoader.urls.asJava) + jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) + } + + def close(): Unit = { + env.close() + miniCluster.close() + } + +} + +object ScenarioTestingMiniClusterWrapper { + + def create(parallelism: Int, streamExecutionConfig: Configuration): ScenarioTestingMiniClusterWrapper = { + val miniCluster = ScenarioTestingMiniClusterFactory.createConfiguredMiniCluster(parallelism) + val env = ScenarioTestingStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment( + parallelism, + streamExecutionConfig + ) + new ScenarioTestingMiniClusterWrapper(miniCluster, env, parallelism) + } + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingStreamExecutionEnvironmentFactory.scala similarity index 82% rename from engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala rename to engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingStreamExecutionEnvironmentFactory.scala index 172ff53f26d..485e99e2080 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/TestsMechanismStreamExecutionEnvironmentFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingStreamExecutionEnvironmentFactory.scala @@ -1,9 +1,9 @@ -package pl.touk.nussknacker.engine.management.testsmechanism +package pl.touk.nussknacker.engine.process.scenariotesting import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -object TestsMechanismStreamExecutionEnvironmentFactory { +object ScenarioTestingStreamExecutionEnvironmentFactory { def createStreamExecutionEnvironment(parallelism: Int, configuration: Configuration): StreamExecutionEnvironment = { val env = StreamExecutionEnvironment.createLocalEnvironment( diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala deleted file mode 100644 index 2357418a1e0..00000000000 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/testmechanism/FlinkMiniClusterJobSubmitter.scala +++ /dev/null @@ -1,38 +0,0 @@ -package pl.touk.nussknacker.engine.process.testmechanism - -import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} -import org.apache.flink.runtime.minicluster.MiniCluster -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import pl.touk.nussknacker.engine.api.process.ProcessName -import pl.touk.nussknacker.engine.util.loader.ModelClassLoader - -import scala.jdk.CollectionConverters._ - -// We use MiniCluster directly, instead of LocalExecutionEnvironment, to be able to pass own classpath... -final class FlinkMiniClusterJobSubmitter(miniCluster: MiniCluster, env: StreamExecutionEnvironment) { - - def submitJobAndCleanEnv( - scenarioName: ProcessName, - savepointRestoreSettings: SavepointRestoreSettings, - modelClassLoader: ModelClassLoader - ): Unit = { - // This step clean env transformations. It allows to reuse the same StreamExecutionEnvironment many times - val streamGraph = env.getStreamGraph - streamGraph.setJobName(scenarioName.value) - val jobGraph = streamGraph.getJobGraph - setupJobGraph(jobGraph, savepointRestoreSettings, modelClassLoader) - - val id = miniCluster.submitJob(jobGraph).get().getJobID - miniCluster.requestJobResult(id).get().toJobExecutionResult(getClass.getClassLoader) - } - - private def setupJobGraph( - jobGraph: JobGraph, - savepointRestoreSettings: SavepointRestoreSettings, - modelClassLoader: ModelClassLoader - ): Unit = { - jobGraph.setClasspaths(modelClassLoader.urls.asJava) - jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) - } - -} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala index 0a5021c6bb5..b8ef97670d7 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala @@ -1,8 +1,11 @@ package pl.touk.nussknacker.engine.management -import pl.touk.nussknacker.engine.api.deployment.cache.ScenarioStateCachingConfig +import net.ceedubs.ficus.Ficus +import net.ceedubs.ficus.readers.ValueReader +import org.apache.flink.configuration.Configuration import scala.concurrent.duration.{DurationInt, FiniteDuration} +import scala.jdk.CollectionConverters._ /** * FlinkConfig deployment configuration. @@ -15,12 +18,14 @@ import scala.concurrent.duration.{DurationInt, FiniteDuration} final case class FlinkConfig( restUrl: Option[String], jobManagerTimeout: FiniteDuration = 1 minute, + // TODO: move to scenarioTesting shouldVerifyBeforeDeploy: Boolean = true, shouldCheckAvailableSlots: Boolean = true, waitForDuringDeployFinish: FlinkWaitForDuringDeployFinishedConfig = FlinkWaitForDuringDeployFinishedConfig(enabled = true, Some(180), Some(1 second)), scenarioStateRequestTimeout: FiniteDuration = 3 seconds, jobConfigsCacheSize: Int = 1000, + scenarioTesting: ScenarioTestingConfig = ScenarioTestingConfig() ) object FlinkConfig { @@ -51,3 +56,19 @@ final case class FlinkWaitForDuringDeployFinishedConfig( } final case class EnabledFlinkWaitForDuringDeployFinishedConfig(maxChecks: Int, delay: FiniteDuration) + +final case class ScenarioTestingConfig( + reuseMiniClusterForScenarioTesting: Boolean = true, + reuseMiniClusterForScenarioStateVerification: Boolean = true, + parallelism: Int = 1, + streamExecutionConfig: Configuration = new Configuration +) + +object ScenarioTestingConfig { + + import Ficus._ + + implicit val flinkConfigurationValueReader: ValueReader[Configuration] = + Ficus.mapValueReader[String].map(map => Configuration.fromMap(map.asJava)) + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala index 4f16942d8ea..180263c7b7c 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala @@ -3,7 +3,6 @@ package pl.touk.nussknacker.engine.management import cats.implicits._ import com.typesafe.scalalogging.LazyLogging import io.circe.syntax.EncoderOps -import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.engine.ModelData._ import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.api.deployment.DeploymentUpdateStrategy.StateRestoringStrategy @@ -14,7 +13,11 @@ import pl.touk.nussknacker.engine.api.process.{ProcessIdWithName, ProcessName, V import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.deployment.{DeploymentData, ExternalDeploymentId} import pl.touk.nussknacker.engine.management.FlinkDeploymentManager.prepareProgramArgs -import pl.touk.nussknacker.engine.management.testsmechanism.{FlinkProcessTestRunner, FlinkProcessVerifier} +import pl.touk.nussknacker.engine.management.scenariotesting.{ + FlinkProcessTestRunner, + FlinkProcessVerifier, + ScenarioTestingMiniClusterWrapperFactory +} import pl.touk.nussknacker.engine.{BaseModelData, DeploymentManagerDependencies, newdeployment} import scala.concurrent.Future @@ -23,16 +26,28 @@ abstract class FlinkDeploymentManager( modelData: BaseModelData, dependencies: DeploymentManagerDependencies, shouldVerifyBeforeDeploy: Boolean, - mainClassName: String + mainClassName: String, + scenarioTestingConfig: ScenarioTestingConfig ) extends DeploymentManager with LazyLogging { import dependencies._ - private lazy val testRunner = - new FlinkProcessTestRunner(modelData.asInvokableModelData, parallelism = 1, new Configuration) + private lazy val scenarioTestingMiniClusterWrapperOpt = + ScenarioTestingMiniClusterWrapperFactory.createIfConfigured( + modelData.asInvokableModelData.modelClassLoader, + scenarioTestingConfig + ) + + private lazy val testRunner = new FlinkProcessTestRunner( + modelData.asInvokableModelData, + scenarioTestingMiniClusterWrapperOpt.filter(_ => scenarioTestingConfig.reuseMiniClusterForScenarioTesting) + ) - private lazy val verification = new FlinkProcessVerifier(modelData.asInvokableModelData) + private lazy val verification = new FlinkProcessVerifier( + modelData.asInvokableModelData, + scenarioTestingMiniClusterWrapperOpt.filter(_ => scenarioTestingConfig.reuseMiniClusterForScenarioStateVerification) + ) /** * Gets status from engine, handles finished state, resolves possible inconsistency with lastAction and formats status using `ProcessStateDefinitionManager` @@ -267,7 +282,7 @@ abstract class FlinkDeploymentManager( override def close(): Unit = { logger.info("Closing Flink Deployment Manager") - testRunner.close() + scenarioTestingMiniClusterWrapperOpt.foreach(_.close()) } } diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala index 1a66968f1e1..efa2e32762b 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkRestManager.scala @@ -5,14 +5,7 @@ import com.typesafe.scalalogging.LazyLogging import org.apache.flink.api.common.{JobID, JobStatus} import pl.touk.nussknacker.engine.api.ProcessVersion import pl.touk.nussknacker.engine.api.deployment._ -import pl.touk.nussknacker.engine.api.deployment.scheduler._ -import pl.touk.nussknacker.engine.api.deployment.scheduler.services.{ - AdditionalDeploymentDataProvider, - ProcessConfigEnricherFactory, - SchedulePropertyExtractorFactory, - ScheduledExecutionPerformer, - ScheduledProcessListenerFactory -} +import pl.touk.nussknacker.engine.api.deployment.scheduler.services._ import pl.touk.nussknacker.engine.api.deployment.simple.SimpleStateStatus import pl.touk.nussknacker.engine.api.process.{ProcessId, ProcessName, VersionId} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess @@ -30,8 +23,14 @@ class FlinkRestManager( config: FlinkConfig, modelData: BaseModelData, dependencies: DeploymentManagerDependencies, - mainClassName: String -) extends FlinkDeploymentManager(modelData, dependencies, config.shouldVerifyBeforeDeploy, mainClassName) + mainClassName: String, +) extends FlinkDeploymentManager( + modelData, + dependencies, + config.shouldVerifyBeforeDeploy, + mainClassName, + config.scenarioTesting + ) with LazyLogging { import dependencies._ diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkScheduledExecutionPerformer.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkScheduledExecutionPerformer.scala index c312c7834db..480620ea3d2 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkScheduledExecutionPerformer.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkScheduledExecutionPerformer.scala @@ -29,6 +29,7 @@ object FlinkScheduledExecutionPerformer { import dependencies._ import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ArbitraryTypeReader._ + import ScenarioTestingConfig._ val flinkConfig = config.rootAs[FlinkConfig] new FlinkScheduledExecutionPerformer( flinkClient = HttpFlinkClient.createUnsafe(flinkConfig), diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala index 7081194a2cd..b6ae35d6be3 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala @@ -20,6 +20,7 @@ class FlinkStreamingDeploymentManagerProvider extends DeploymentManagerProvider import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ArbitraryTypeReader._ import pl.touk.nussknacker.engine.util.config.ConfigEnrichments._ + import ScenarioTestingConfig._ override def createDeploymentManager( modelData: BaseModelData, diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunner.scala new file mode 100644 index 00000000000..3e69d0ab93d --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunner.scala @@ -0,0 +1,39 @@ +package pl.touk.nussknacker.engine.management.scenariotesting + +import io.circe.Json +import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.test.ScenarioTestData +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker + +import scala.concurrent.{ExecutionContext, Future} + +class FlinkProcessTestRunner(modelData: ModelData, miniClusterWrapperOpt: Option[AutoCloseable]) { + + // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar + // because it is already in separate assembly for purpose of sending it to Flink during deployment. + // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded + private val mainRunner = new ReflectiveMethodInvoker[TestResults[Json]]( + modelData.modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.scenariotesting.FlinkTestMain", + "run" + ) + + def runTestsAsync(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData)( + implicit ec: ExecutionContext + ): Future[TestResults[Json]] = + Future { + runTests(canonicalProcess, scenarioTestData) + } + + // NU-1455: We encode variable on the engine, because of classLoader's problems + def runTests(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = + mainRunner.invokeStaticMethod( + miniClusterWrapperOpt, + modelData, + canonicalProcess, + scenarioTestData + ) + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessVerifier.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessVerifier.scala new file mode 100644 index 00000000000..8640660effe --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessVerifier.scala @@ -0,0 +1,52 @@ +package pl.touk.nussknacker.engine.management.scenariotesting + +import com.typesafe.scalalogging.LazyLogging +import pl.touk.nussknacker.engine.ModelData +import pl.touk.nussknacker.engine.api.ProcessVersion +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker + +import scala.concurrent.Future +import scala.util.control.NonFatal + +class FlinkProcessVerifier(modelData: ModelData, miniClusterWrapperOpt: Option[Any]) extends LazyLogging { + + // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar + // because it is already in separate assembly for purpose of sending it to Flink during deployment. + // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded + private val mainRunner = new ReflectiveMethodInvoker[Unit]( + modelData.modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.scenariotesting.FlinkVerificationMain", + "run" + ) + + def verify( + processVersion: ProcessVersion, + canonicalProcess: CanonicalProcess, + savepointPath: String + ): Future[Unit] = { + val processId = processVersion.processName + try { + logger.info(s"Starting to verify $processId") + mainRunner.invokeStaticMethod( + miniClusterWrapperOpt, + modelData, + canonicalProcess, + processVersion, + savepointPath + ) + logger.info(s"Verification of $processId successful") + Future.successful(()) + } catch { + case NonFatal(e) => + logger.info(s"Failed to verify $processId", e) + Future.failed( + new IllegalArgumentException( + "State is incompatible, please stop scenario and start again with clean state", + e + ) + ) + } + } + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala new file mode 100644 index 00000000000..9a86da446a6 --- /dev/null +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala @@ -0,0 +1,34 @@ +package pl.touk.nussknacker.engine.management.scenariotesting + +import org.apache.flink.configuration.Configuration +import pl.touk.nussknacker.engine.management.ScenarioTestingConfig +import pl.touk.nussknacker.engine.util.ReflectiveMethodInvoker +import pl.touk.nussknacker.engine.util.loader.ModelClassLoader + +object ScenarioTestingMiniClusterWrapperFactory { + + // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar + // because it is already in separate assembly for purpose of sending it to Flink during deployment. + // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded + def createIfConfigured(modelClassLoader: ModelClassLoader, config: ScenarioTestingConfig): Option[AutoCloseable] = { + if (config.reuseMiniClusterForScenarioTesting || config.reuseMiniClusterForScenarioStateVerification) { + Some(create(modelClassLoader, config.parallelism, config.streamExecutionConfig)) + } else { + None + } + } + + private[nussknacker] def create( + modelClassLoader: ModelClassLoader, + parallelism: Int, + streamExecutionConfig: Configuration + ): AutoCloseable = { + val methodInvoker = new ReflectiveMethodInvoker[AutoCloseable]( + modelClassLoader.classLoader, + "pl.touk.nussknacker.engine.process.scenariotesting.ScenarioTestingMiniClusterWrapper", + "create" + ) + methodInvoker.invokeStaticMethod(parallelism, streamExecutionConfig) + } + +} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala deleted file mode 100644 index 40943fced23..00000000000 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunner.scala +++ /dev/null @@ -1,71 +0,0 @@ -package pl.touk.nussknacker.engine.management.testsmechanism - -import io.circe.Json -import org.apache.flink.configuration.Configuration -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.StreamMetaData -import pl.touk.nussknacker.engine.api.test.ScenarioTestData -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.testmode.TestProcess.TestResults -import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ReflectiveMethodInvoker} - -import scala.concurrent.{ExecutionContext, Future} - -class FlinkProcessTestRunner(modelData: ModelData, parallelism: Int, streamExecutionConfig: Configuration) - extends AutoCloseable { - - private val streamExecutionEnvironment = - TestsMechanismStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment(parallelism, streamExecutionConfig) - - private val miniCluster = TestsMechanismMiniClusterFactory.createConfiguredMiniCluster(parallelism) - - // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar - // because it is already in separate assembly for purpose of sending it to Flink during deployment. - // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded - private val methodInvoker = new ReflectiveMethodInvoker[TestResults[Json]]( - modelData.modelClassLoader.classLoader, - "pl.touk.nussknacker.engine.process.runner.FlinkTestMain", - "run" - ) - - def runTestsAsync(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData)( - implicit ec: ExecutionContext - ): Future[TestResults[Json]] = - Future { - runTests(canonicalProcess, scenarioTestData) - } - - // NU-1455: We encode variable on the engine, because of classLoader's problems - def runTests(canonicalProcess: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = - methodInvoker.invokeStaticMethod( - miniCluster, - streamExecutionEnvironment, - modelData, - rewriteParallelismIfHigherThanMaxParallelism(canonicalProcess), - scenarioTestData - ) - - private def rewriteParallelismIfHigherThanMaxParallelism(canonicalProcess: CanonicalProcess): CanonicalProcess = { - val scenarioParallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](canonicalProcess.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - if (scenarioParallelism > parallelism) { - canonicalProcess.copy(metaData = - canonicalProcess.metaData.copy(additionalFields = - canonicalProcess.metaData.additionalFields.copy(properties = - canonicalProcess.metaData.additionalFields.properties + (StreamMetaData.parallelismName -> parallelism.toString) - ) - ) - ) - } else { - canonicalProcess - } - } - - def close(): Unit = { - miniCluster.close() - streamExecutionEnvironment.close() - } - -} diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala deleted file mode 100644 index 9e432e8b3df..00000000000 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessVerifier.scala +++ /dev/null @@ -1,69 +0,0 @@ -package pl.touk.nussknacker.engine.management.testsmechanism - -import com.typesafe.scalalogging.LazyLogging -import org.apache.flink.configuration.Configuration -import pl.touk.nussknacker.engine.ModelData -import pl.touk.nussknacker.engine.api.{ProcessVersion, StreamMetaData} -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess -import pl.touk.nussknacker.engine.deployment.DeploymentData -import pl.touk.nussknacker.engine.util.{MetaDataExtractor, ReflectiveMethodInvoker} - -import scala.concurrent.Future -import scala.util.Using -import scala.util.control.NonFatal - -class FlinkProcessVerifier(modelData: ModelData) extends LazyLogging { - - // We use reflection, because we don't want to bundle flinkExecutor.jar inside flinkDeploymentManager assembly jar - // because it is already in separate assembly for purpose of sending it to Flink during deployment. - // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded - private val methodInvoker = new ReflectiveMethodInvoker[Unit]( - modelData.modelClassLoader.classLoader, - "pl.touk.nussknacker.engine.process.runner.FlinkVerificationMain", - "run" - ) - - def verify( - processVersion: ProcessVersion, - canonicalProcess: CanonicalProcess, - savepointPath: String - ): Future[Unit] = { - val parallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](canonicalProcess.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - val processId = processVersion.processName - try { - logger.info(s"Starting to verify $processId") - // TODO: reuse a single mini cluster between each verifications - Using.resource(TestsMechanismMiniClusterFactory.createConfiguredMiniCluster(parallelism)) { miniCluster => - val env = TestsMechanismStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment( - parallelism, - new Configuration() - ) - - methodInvoker.invokeStaticMethod( - miniCluster, - env, - modelData, - canonicalProcess, - processVersion, - DeploymentData.empty, - savepointPath - ) - } - logger.info(s"Verification of $processId successful") - Future.successful(()) - } catch { - case NonFatal(e) => - logger.info(s"Failed to verify $processId", e) - Future.failed( - new IllegalArgumentException( - "State is incompatible, please stop scenario and start again with clean state", - e - ) - ) - } - } - -} diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala similarity index 89% rename from engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala rename to engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala index 413e17fd436..80200d0c17d 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/testsmechanism/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.engine.management.testsmechanism +package pl.touk.nussknacker.engine.management.scenariotesting import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} import io.circe.Json @@ -6,7 +6,7 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.client.JobExecutionException import org.scalatest.matchers.should.Matchers import org.scalatest.wordspec.AnyWordSpec -import org.scalatest.{BeforeAndAfterEach, Inside, OptionValues} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Inside, OptionValues} import pl.touk.nussknacker.engine.api.component.{ ComponentAdditionalConfig, DesignerWideComponentId, @@ -19,7 +19,7 @@ import pl.touk.nussknacker.engine.api.parameter.{ } import pl.touk.nussknacker.engine.api.process.ComponentUseCase import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} -import pl.touk.nussknacker.engine.api.{DisplayJsonWithEncoder, FragmentSpecificData, MetaData, StreamMetaData} +import pl.touk.nussknacker.engine.api.{DisplayJsonWithEncoder, FragmentSpecificData, MetaData} import pl.touk.nussknacker.engine.build.{GraphBuilder, ScenarioBuilder} import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.canonicalgraph.canonicalnode.FlatNode @@ -33,26 +33,24 @@ import pl.touk.nussknacker.engine.flink.test.{ import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.node.FragmentInputDefinition.{FragmentClazzRef, FragmentParameter} import pl.touk.nussknacker.engine.graph.node.{Case, FragmentInputDefinition, FragmentOutputDefinition} -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunnerSpec.{ +import pl.touk.nussknacker.engine.management.scenariotesting.FlinkProcessTestRunnerSpec.{ fragmentWithValidationName, processWithFragmentParameterValidation } import pl.touk.nussknacker.engine.process.helpers.SampleNodes._ import pl.touk.nussknacker.engine.testmode.TestProcess._ -import pl.touk.nussknacker.engine.util.MetaDataExtractor import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import pl.touk.nussknacker.engine.{ModelConfigs, ModelData} import java.util.{Date, UUID} -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ -import scala.concurrent.{Await, Future} class FlinkProcessTestRunnerSpec extends AnyWordSpec with Matchers with Inside with BeforeAndAfterEach + with BeforeAndAfterAll with OptionValues { import pl.touk.nussknacker.engine.spel.SpelExtension._ @@ -61,6 +59,13 @@ class FlinkProcessTestRunnerSpec private val scenarioName = "proc1" private val sourceNodeId = "id" private val firstSubtaskIndex = 0 + private val modelClassLoader = ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) + + private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( + modelClassLoader, + parallelism = 1, + streamExecutionConfig = new Configuration + ) override def beforeEach(): Unit = { super.beforeEach() @@ -68,6 +73,11 @@ class FlinkProcessTestRunnerSpec LogService.clear() } + override protected def afterAll(): Unit = { + super.afterAll() + scenarioTestingMiniClusterWrapper.close() + } + "A scenario run on Flink engine" when { "IO monad interpreter is used" should { runTests(useIOMonadInInterpreter = true) @@ -78,7 +88,6 @@ class FlinkProcessTestRunnerSpec } private def runTests(useIOMonadInInterpreter: Boolean): Unit = { - val testRunner = prepareTestRunner(useIOMonadInInterpreter) "be able to return test results" in { val process = ScenarioBuilder @@ -93,7 +102,7 @@ class FlinkProcessTestRunnerSpec val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData( List( @@ -143,7 +152,7 @@ class FlinkProcessTestRunnerSpec LogService.invocationsCount.get() shouldBe 0 } - "be able to run tests multiple time on the same test runner" in { + "be able to run tests multiple time on the same mini cluster" in { val process = ScenarioBuilder .streaming(scenarioName) @@ -152,6 +161,9 @@ class FlinkProcessTestRunnerSpec val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") + + val testRunner = prepareTestRunner(useIOMonadInInterpreter) + def runTestAndVerify() = { val results = testRunner.runTests( process, @@ -188,7 +200,7 @@ class FlinkProcessTestRunnerSpec .source(sourceNodeId, "input") .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), ) @@ -221,7 +233,7 @@ class FlinkProcessTestRunnerSpec val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), ) @@ -274,7 +286,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "monitor") val results = - testRunner.runTests( + prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), ) @@ -293,7 +305,7 @@ class FlinkProcessTestRunnerSpec .filter("filter", "1 / #input.value1 >= 0".spel) .emptySink("out", "monitor") - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData( List( @@ -349,8 +361,11 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "monitor") val exceptionConsumerId = UUID.randomUUID().toString - val results = runTestsWithCustomModel( - process = process, + val results = prepareTestRunner( + useIOMonadInInterpreter, + enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) + ).runTests( + process, scenarioTestData = ScenarioTestData( List( createTestRecord(id = "0", value1 = 1), @@ -358,9 +373,7 @@ class FlinkProcessTestRunnerSpec createTestRecord(id = "2", value1 = 2), createTestRecord(id = "3", value1 = 4) ) - ), - useIOMonadInInterpreter, - enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId), + ) ) val nodeResults = results.nodeResults @@ -380,11 +393,12 @@ class FlinkProcessTestRunnerSpec .processor("failing", "throwingTransientService", "throw" -> "#input.value1 == 2".spel) .emptySink("out", "monitor") - val run = Future { - testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) + intercept[JobExecutionException] { + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) } - - intercept[JobExecutionException](Await.result(run, 10 seconds)) } "handle json input" in { @@ -410,7 +424,7 @@ class FlinkProcessTestRunnerSpec ) ) - val results = testRunner.runTests(process, testData) + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) results.nodeResults(sourceNodeId) should have size 3 results.externalInvocationResults("out") shouldBe @@ -440,7 +454,7 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) - val results = testRunner.runTests(process, testData) + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) results.nodeResults(sourceNodeId) should have size 1 results.externalInvocationResults("out") shouldBe @@ -461,7 +475,10 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) val results = - testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) results.exceptions should have length 1 results.exceptions.head.nodeId shouldBe Some("out") @@ -479,7 +496,7 @@ class FlinkProcessTestRunnerSpec def recordWithSeconds(duration: FiniteDuration) = ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData( List( @@ -509,7 +526,7 @@ class FlinkProcessTestRunnerSpec ) .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( process, ScenarioTestData( ScenarioTestJsonRecord( @@ -540,7 +557,10 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) val results = - testRunner.runTests(process, ScenarioTestData(List(createTestRecord(value1 = valueToReturn)))) + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(value1 = valueToReturn))) + ) results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) } @@ -563,7 +583,8 @@ class FlinkProcessTestRunnerSpec val recordTrue = createTestRecord(id = "ala") val recordFalse = createTestRecord(id = "bela") - val results = testRunner.runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) val invocationResults = results.invocationResults @@ -601,7 +622,8 @@ class FlinkProcessTestRunnerSpec val recB = createTestRecord(id = "b") val recC = createTestRecord(id = "c") - val results = testRunner.runTests(process, ScenarioTestData(List(recA, recB, recC))) + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recA, recB, recC))) results.invocationResults("proc2").map(_.contextId) should contain only ( s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", @@ -658,7 +680,7 @@ class FlinkProcessTestRunnerSpec val recordC = recordA.copy(id = "c") val recordD = recordA.copy(id = "d") - val results = testRunner.runTests(process, scenarioTestData) + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, scenarioTestData) val nodeResults = results.nodeResults nodeResults("source1") shouldBe List( @@ -705,7 +727,10 @@ class FlinkProcessTestRunnerSpec .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) val results = - testRunner.runTests(process, ScenarioTestData(List(createTestRecord(sourceId = "start")))) + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(sourceId = "start"))) + ) results.invocationResults("out").map(_.value) shouldBe List( variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) @@ -725,10 +750,12 @@ class FlinkProcessTestRunnerSpec ) .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - val run = Future { - testRunner.runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) + val dictEditorException = intercept[IllegalStateException] { + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) } - val dictEditorException = intercept[IllegalStateException](Await.result(run, 10 seconds)) dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" } @@ -747,9 +774,7 @@ class FlinkProcessTestRunnerSpec ) .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - val results = runTestsWithCustomModel( - process, - ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))), + val results = prepareTestRunner( useIOMonadInInterpreter, additionalConfigsFromProvider = Map( DesignerWideComponentId("service-" + modifiedComponentName) -> ComponentAdditionalConfig( @@ -764,7 +789,7 @@ class FlinkProcessTestRunnerSpec ) ) ) - ) + ).runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) results.exceptions should have length 0 } @@ -777,7 +802,7 @@ class FlinkProcessTestRunnerSpec val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) - val results = testRunner.runTests( + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), ) @@ -792,44 +817,21 @@ class FlinkProcessTestRunnerSpec ): ScenarioTestJsonRecord = ScenarioTestJsonRecord(sourceId, Json.fromString(s"$id|$value1|2|3|4|5|6")) - private def runTestsWithCustomModel( - process: CanonicalProcess, - scenarioTestData: ScenarioTestData, + private def prepareTestRunner( useIOMonadInInterpreter: Boolean, enrichDefaultConfig: Config => Config = identity, additionalConfigsFromProvider: Map[DesignerWideComponentId, ComponentAdditionalConfig] = Map.empty - ): TestResults[_] = { + ): FlinkProcessTestRunner = { val config = enrichDefaultConfig(ConfigFactory.load("application.conf")) .withValue("globalParameters.useIOMonadInInterpreter", ConfigValueFactory.fromAnyRef(useIOMonadInInterpreter)) // We need to set context loader to avoid forking in sbt val modelData = ModelData.duringExecution( ModelConfigs(config, AdditionalModelConfigs(additionalConfigsFromProvider)), - ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround), + modelClassLoader, resolveConfigs = false ) - // TODO: reuse this instance between all test cases - val parallelism = MetaDataExtractor - .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) - .parallelism - .getOrElse(1) - new FlinkProcessTestRunner(modelData, parallelism, FlinkTestConfiguration.setupMemory(new Configuration)) - .runTests(process, scenarioTestData) - } - - private def prepareTestRunner(useIOMonadInInterpreter: Boolean) = { - val config = ConfigFactory - .load("application.conf") - .withValue("globalParameters.useIOMonadInInterpreter", ConfigValueFactory.fromAnyRef(useIOMonadInInterpreter)) - - // We need to set context loader to avoid forking in sbt - val modelData = ModelData.duringExecution( - ModelConfigs(config, AdditionalModelConfigs(Map.empty)), - ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround), - resolveConfigs = false - ) - - new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + new FlinkProcessTestRunner(modelData, Some(scenarioTestingMiniClusterWrapper)) } private def nodeResult(count: Int, vars: (String, Any)*): ResultContext[_] = diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala similarity index 89% rename from engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala rename to engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala index 49db5f3e0dc..5cbb12a32ee 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/kafka/TestFromFileSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.testsmechanism.kafka +package pl.touk.nussknacker.scenariotesting.kafka import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigValueFactory.fromAnyRef @@ -19,7 +19,10 @@ import pl.touk.nussknacker.engine.kafka.KafkaFactory.TopicParamName import pl.touk.nussknacker.engine.kafka.source.InputMeta import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator.ResultsHolders -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner +import pl.touk.nussknacker.engine.management.scenariotesting.{ + FlinkProcessTestRunner, + ScenarioTestingMiniClusterWrapperFactory +} import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.util.json.ToJsonEncoder @@ -28,7 +31,7 @@ import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigPropert import java.util.Collections -class TestFromFileSpec +class KafkaScenarioTestingSpec extends AnyFunSuite with Matchers with LazyLogging @@ -49,16 +52,22 @@ class TestFromFileSpec LocalModelData( inputConfig = config, components = List.empty, - configCreator = new KafkaSourceFactoryProcessConfigCreator(() => TestFromFileSpec.resultsHolders), + configCreator = new KafkaSourceFactoryProcessConfigCreator(() => KafkaScenarioTestingSpec.resultsHolders), modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) ) + private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( + modelData.modelClassLoader, + parallelism = 1, + streamExecutionConfig = new Configuration + ) + private val testRunner = - new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + new FlinkProcessTestRunner(modelData, Some(scenarioTestingMiniClusterWrapper)) override protected def afterAll(): Unit = { super.afterAll() - testRunner.close() + scenarioTestingMiniClusterWrapper.close() } test("Should pass correct timestamp from test data") { @@ -139,7 +148,7 @@ class TestFromFileSpec } -object TestFromFileSpec extends Serializable { +object KafkaScenarioTestingSpec extends Serializable { private val resultsHolders = new ResultsHolders diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala similarity index 91% rename from engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala rename to engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala index 318fdc43fe9..38874bc8e63 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/testsmechanism/schemedkafka/TestWithTestDataSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala @@ -1,4 +1,4 @@ -package pl.touk.nussknacker.testsmechanism.schemedkafka +package pl.touk.nussknacker.scenariotesting.schemedkafka import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigValueFactory.fromAnyRef @@ -14,21 +14,16 @@ import org.scalatest.{BeforeAndAfterAll, LoneElement, OptionValues} import pl.touk.nussknacker.engine.api.parameter.ParameterName import pl.touk.nussknacker.engine.api.test.{ScenarioTestData, ScenarioTestJsonRecord} import pl.touk.nussknacker.engine.build.ScenarioBuilder -import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.flink.test.FlinkTestConfiguration import pl.touk.nussknacker.engine.flink.util.sink.SingleValueSinkFactory.SingleValueParamName -import pl.touk.nussknacker.engine.flink.util.test.FlinkTestScenarioRunner import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.kafka.UnspecializedTopicName import pl.touk.nussknacker.engine.kafka.source.InputMeta -import pl.touk.nussknacker.engine.management.testsmechanism.FlinkProcessTestRunner +import pl.touk.nussknacker.engine.management.scenariotesting.{FlinkProcessTestRunner, ScenarioTestingMiniClusterWrapperFactory} import pl.touk.nussknacker.engine.process.helpers.TestResultsHolder import pl.touk.nussknacker.engine.schemedkafka.KafkaAvroIntegrationMockSchemaRegistry.schemaRegistryMockClient import pl.touk.nussknacker.engine.schemedkafka.KafkaAvroTestProcessConfigCreator -import pl.touk.nussknacker.engine.schemedkafka.KafkaUniversalComponentTransformer.{ - schemaVersionParamName, - topicParamName -} +import pl.touk.nussknacker.engine.schemedkafka.KafkaUniversalComponentTransformer.{schemaVersionParamName, topicParamName} import pl.touk.nussknacker.engine.schemedkafka.schema.{Address, Company} import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.confluent.ConfluentUtils import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.universal.MockSchemaRegistryClientFactory @@ -36,15 +31,14 @@ import pl.touk.nussknacker.engine.schemedkafka.schemaregistry.{SchemaRegistryCli import pl.touk.nussknacker.engine.spel.SpelExtension._ import pl.touk.nussknacker.engine.testing.LocalModelData import pl.touk.nussknacker.engine.testmode.TestProcess._ -import pl.touk.nussknacker.engine.util.ThreadUtils import pl.touk.nussknacker.engine.util.json.ToJsonEncoder import pl.touk.nussknacker.engine.util.loader.ModelClassLoader +import pl.touk.nussknacker.scenariotesting.schemedkafka.SchemedKafkaScenarioTestingSpec.sinkForInputMetaResultsHolder import pl.touk.nussknacker.test.{EitherValuesDetailedMessage, KafkaConfigProperties} -import pl.touk.nussknacker.testsmechanism.schemedkafka.TestWithTestDataSpec.sinkForInputMetaResultsHolder import java.util.Collections -class TestWithTestDataSpec +class SchemedKafkaScenarioTestingSpec extends AnyFunSuite with Matchers with LazyLogging @@ -77,12 +71,18 @@ class TestWithTestDataSpec modelClassLoader = new ModelClassLoader(getClass.getClassLoader, FlinkTestConfiguration.classpathWorkaround) ) + private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( + modelData.modelClassLoader, + parallelism = 1, + streamExecutionConfig = new Configuration + ) + private val testRunner = - new FlinkProcessTestRunner(modelData, parallelism = 1, FlinkTestConfiguration.setupMemory(new Configuration)) + new FlinkProcessTestRunner(modelData, Some(scenarioTestingMiniClusterWrapper)) override protected def afterAll(): Unit = { super.afterAll() - testRunner.close() + scenarioTestingMiniClusterWrapper.close() } test("Should pass correct timestamp from test data") { @@ -219,7 +219,7 @@ class TestWithTestDataSpec } -object TestWithTestDataSpec { +object SchemedKafkaScenarioTestingSpec { private val sinkForInputMetaResultsHolder = new TestResultsHolder[java.util.Map[String @unchecked, _]] From cd39d3b1b92906ad7c1c9a44ad52936b6bf18502 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 15:53:12 +0100 Subject: [PATCH 21/48] Verification mechanism: parallelism alignment --- .../FlinkVerificationMain.scala | 7 +++-- .../FlinkStreamingDeploymentManagerSpec.scala | 29 ++++++++++++++----- .../streaming/StatefulSampleProcess.scala | 3 +- 3 files changed, 27 insertions(+), 12 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala index 097535f9a7f..17e59cc17a0 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala @@ -33,13 +33,14 @@ class FlinkVerificationMain( try { AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback(miniClusterWrapperOpt, scenario) { miniClusterWrapper => + val alignedScenario = miniClusterWrapper.alignParallelism(scenario) val resultCollector = new TestServiceInvocationCollector(collectingListener) - val registrar = prepareRegistrar(scenario) + val registrar = prepareRegistrar(alignedScenario) val deploymentData = DeploymentData.empty - registrar.register(miniClusterWrapper.env, scenario, processVersion, deploymentData, resultCollector) + registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) miniClusterWrapper.submitJobAndCleanEnv( - scenario.name, + alignedScenario.name, SavepointRestoreSettings.forPath(savepointPath, true), modelData.modelClassLoader ) diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerSpec.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerSpec.scala index 0916e833329..0de3599cc72 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerSpec.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/FlinkStreamingDeploymentManagerSpec.scala @@ -16,6 +16,7 @@ import pl.touk.nussknacker.engine.api.deployment.{ } import pl.touk.nussknacker.engine.api.deployment.simple.SimpleStateStatus import pl.touk.nussknacker.engine.api.process.{ProcessId, ProcessName, VersionId} +import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.definition.component.Components.ComponentDefinitionExtractionMode import pl.touk.nussknacker.engine.deployment.DeploymentData import pl.touk.nussknacker.engine.util.loader.ModelClassLoader @@ -102,23 +103,35 @@ class FlinkStreamingDeploymentManagerSpec extends AnyFunSuite with Matchers with } test("save state when redeploying") { - val processName = ProcessName("redeploy") - val outTopic = s"output-$processName" - val processEmittingOneElementAfterStart = StatefulSampleProcess.prepareProcess(processName) + val processEmittingOneElementAfterStart = StatefulSampleProcess.prepareProcess(ProcessName("redeploy")) + testRedeployWithStatefulSampleProcess(processEmittingOneElementAfterStart) + } + + test("redeploy scenario with greater parallelism than configured in mini cluster") { + val processEmittingOneElementAfterStart = + StatefulSampleProcess.prepareProcess(ProcessName("redeploy-parallelism-2"), parallelism = 2) + testRedeployWithStatefulSampleProcess(processEmittingOneElementAfterStart) + } + + private def testRedeployWithStatefulSampleProcess(processEmittingOneElementAfterStart: CanonicalProcess) = { + val outTopic = s"output-${processEmittingOneElementAfterStart.name}" kafkaClient.createTopic(outTopic, 1) - deployProcessAndWaitIfRunning(processEmittingOneElementAfterStart, empty(processName)) + deployProcessAndWaitIfRunning(processEmittingOneElementAfterStart, empty(processEmittingOneElementAfterStart.name)) try { // we wait for first element to appear in kafka to be sure it's processed, before we proceed to checkpoint messagesFromTopic(outTopic, 1) shouldBe List("[One element]") - deployProcessAndWaitIfRunning(processEmittingOneElementAfterStart, empty(processName)) + deployProcessAndWaitIfRunning( + processEmittingOneElementAfterStart, + empty(processEmittingOneElementAfterStart.name) + ) val messages = messagesFromTopic(outTopic, 2) messages shouldBe List("[One element]", "[One element, One element]") } finally { - cancelProcess(processName) + cancelProcess(processEmittingOneElementAfterStart.name) } } @@ -261,8 +274,6 @@ class FlinkStreamingDeploymentManagerSpec extends AnyFunSuite with Matchers with } } - def empty(processName: ProcessName): ProcessVersion = ProcessVersion.empty.copy(processName = processName) - test("extract scenario definition") { val modelData = ModelData( processingTypeConfig = processingTypeConfig, @@ -279,6 +290,8 @@ class FlinkStreamingDeploymentManagerSpec extends AnyFunSuite with Matchers with definition.components.components.map(_.id) should contain(ComponentId(ComponentType.Service, "accountService")) } + def empty(processName: ProcessName): ProcessVersion = ProcessVersion.empty.copy(processName = processName) + private def messagesFromTopic(outTopic: String, count: Int): List[String] = kafkaClient .createConsumer() diff --git a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StatefulSampleProcess.scala b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StatefulSampleProcess.scala index 53101dd1ff2..68b2692b195 100644 --- a/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StatefulSampleProcess.scala +++ b/engine/flink/management/src/it/scala/pl/touk/nussknacker/engine/management/streaming/StatefulSampleProcess.scala @@ -8,9 +8,10 @@ object StatefulSampleProcess { import pl.touk.nussknacker.engine.spel.SpelExtension._ - def prepareProcess(name: ProcessName): CanonicalProcess = { + def prepareProcess(name: ProcessName, parallelism: Int = 1): CanonicalProcess = { ScenarioBuilder .streaming(name.value) + .parallelism(parallelism) .source("state", "oneSource") .customNode("stateful", "stateVar", "stateful", "groupBy" -> "#input".spel) .emptySink("end", "kafka-string", "Topic" -> s"'output-$name'".spel, "Value" -> "#stateVar".spel) From a09e34be55c4dd520c5fdea7e69e984e624af956 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 16:10:47 +0100 Subject: [PATCH 22/48] Docs --- docs/Changelog.md | 1 + docs/MigrationGuide.md | 3 + .../ScenarioDeploymentConfiguration.md | 70 ++++++++++++------- 3 files changed, 50 insertions(+), 24 deletions(-) diff --git a/docs/Changelog.md b/docs/Changelog.md index c008dee443c..41953483bee 100644 --- a/docs/Changelog.md +++ b/docs/Changelog.md @@ -60,6 +60,7 @@ * implementation provided for Flink DM * [#7443](https://github.com/TouK/nussknacker/pull/7443) Indexing on record is more similar to indexing on map. The change lets us access record values dynamically. For example now spel expression "{a: 5, b: 10}[#input.field]" compiles and has type "Integer" inferred from types of values of the record. This lets us access record value based on user input, for instance if user passes "{"field": "b"}" to scenario we will get value "10", whereas input {"field": "c"} would result in "null". Expression "{a: 5}["b"]" still does not compile because it is known at compile time that record does not have property "b". * [#7324](https://github.com/TouK/nussknacker/pull/7324) Fix: Passing Flink Job Global Params +* [#7458](https://github.com/TouK/nussknacker/pull/7458) Flink scenario testing mechanism and scenario state verification mechanism: mini cluster created once and reused each time ## 1.18 diff --git a/docs/MigrationGuide.md b/docs/MigrationGuide.md index 43052d7b8f0..586455cb579 100644 --- a/docs/MigrationGuide.md +++ b/docs/MigrationGuide.md @@ -73,6 +73,9 @@ To see the biggest differences please consult the [changelog](Changelog.md). shouldVerifyBeforeDeploy: true } ``` +* [#7458](https://github.com/TouK/nussknacker/pull/7458) Flink scenario testing mechanism and scenario state verification mechanism: by default mini cluster is created once and reused each time + To revert previous behaviour (creating minicluster each time), change `deploymentConfig.scenarioTesting.reuseMiniClusterForScenarioTesting` or/and + `deploymentConfig.scenarioTesting.reuseMiniClusterForScenarioStateVerification` to `false` ### Code API changes * [#7368](https://github.com/TouK/nussknacker/pull/7368) Renamed `PeriodicSourceFactory` to `SampleGeneratorSourceFactory` diff --git a/docs/configuration/ScenarioDeploymentConfiguration.md b/docs/configuration/ScenarioDeploymentConfiguration.md index 9e172b9bf85..3ada853ac4f 100644 --- a/docs/configuration/ScenarioDeploymentConfiguration.md +++ b/docs/configuration/ScenarioDeploymentConfiguration.md @@ -5,10 +5,15 @@ sidebar_position: 2 # Scenario Deployment configuration -Deployment of a scenario onto the [Engine](../about/engines/Engines.md) is managed by the Designer's extension called [Deployment Manager](../about/GLOSSARY.md#deployment-manager). -To enable a given [Deployment Manager](../about/GLOSSARY.md#deployment-manager) its jar package has to be placed in the Designer's classpath. Nussknacker is distributed with three default [Deployment Managers](../about/GLOSSARY.md#deployment-manager) (`flinkStreaming`, `lite-k8s`, `lite-embedded`). Their jars are located in the `managers` directory. +Deployment of a scenario onto the [Engine](../about/engines/Engines.md) is managed by the Designer's extension +called [Deployment Manager](../about/GLOSSARY.md#deployment-manager). +To enable a given [Deployment Manager](../about/GLOSSARY.md#deployment-manager) its jar package has to be placed in the +Designer's classpath. Nussknacker is distributed with three +default [Deployment Managers](../about/GLOSSARY.md#deployment-manager) (`flinkStreaming`, `lite-k8s`, `lite-embedded`). +Their jars are located in the `managers` directory. -Deployment specific configuration is provided in the `deploymentConfig` section of the configuration file - check [configuration areas](./index.mdx#configuration-areas) to understand the structure of the configuration file. +Deployment specific configuration is provided in the `deploymentConfig` section of the configuration file - +check [configuration areas](./index.mdx#configuration-areas) to understand the structure of the configuration file. Below is a snippet of scenario deployment configuration. @@ -23,22 +28,32 @@ deploymentConfig { ``` Parameters: -- `type` parameter determines the type of the [Deployment Manager](../about/GLOSSARY.md#deployment-manager). Possible options are: `flinkStreaming`, `lite-k8s`, `lite-embedded` -- `engineSetupName` parameter is optional. It specifies how the engine will be displayed in the GUI. If not specified, default name will be used instead (e.g. `Flink` for `flinkStreaming` Deployment Manager). + +- `type` parameter determines the type of the [Deployment Manager](../about/GLOSSARY.md#deployment-manager). Possible + options are: `flinkStreaming`, `lite-k8s`, `lite-embedded` +- `engineSetupName` parameter is optional. It specifies how the engine will be displayed in the GUI. If not specified, + default name will be used instead (e.g. `Flink` for `flinkStreaming` Deployment Manager). ## Kubernetes native Lite engine configuration -Please check high level [Lite engine description](../about/engines/LiteArchitecture.md#scenario-deployment) before proceeding to configuration details. +Please check high level [Lite engine description](../about/engines/LiteArchitecture.md#scenario-deployment) before +proceeding to configuration details. -Please note, that K8s Deployment Manager has to be run with properly configured K8s access. If you install the Designer in K8s cluster (e.g. via Helm chart) this comes out of the box. If you want to run the Designer outside the cluster, you have to configure `.kube/config` properly. +Please note, that K8s Deployment Manager has to be run with properly configured K8s access. If you install the Designer +in K8s cluster (e.g. via Helm chart) this comes out of the box. If you want to run the Designer outside the cluster, you +have to configure `.kube/config` properly. Except the `servicePort` configuration option, all remaining configuration options apply to both `streaming` and `request-response` processing modes. -The table below contains configuration options for the Lite engine. If you install Designer with Helm, you can use Helm values override mechanism to supply your own values for these [options](https://artifacthub.io/packages/helm/touk/nussknacker#configuration-in-values-yaml). As the the result of the Helm template rendering "classic" Nussknacker configuration file will be generated. +The table below contains configuration options for the Lite engine. If you install Designer with Helm, you can use Helm +values override mechanism to supply your own values for +these [options](https://artifacthub.io/packages/helm/touk/nussknacker#configuration-in-values-yaml). As the the result +of the Helm template rendering "classic" Nussknacker configuration file will be generated.   -If you install Designer outside the K8s cluster then the required changes should be applied under the `deploymentConfig` key as any other Nussknacker non K8s configuration. +If you install Designer outside the K8s cluster then the required changes should be applied under the `deploymentConfig` +key as any other Nussknacker non K8s configuration. | Parameter | Type | Default value | Description | |------------------------------------------------------|---------------------------|-----------------------------------|------------------------------------------------------------------------------------------| @@ -204,7 +219,9 @@ fixedReplicasCount configuration key; its default value is 2: `{ fixedReplicasCount: x }`. -​In the **Streaming** processing mode the scenario parallelism is set in the scenario properties; it determines the minimal number of tasks used to process events. The count of replicas, scenario parallelism and number of tasks per replica are connected with a simple formula: +​In the **Streaming** processing mode the scenario parallelism is set in the scenario properties; it determines the +minimal number of tasks used to process events. The count of replicas, scenario parallelism and number of tasks per +replica are connected with a simple formula: *scenarioParallelism = replicasCount \* tasksPerReplica* @@ -258,6 +275,7 @@ It can be configured with following options. #### Configuring custom ingress class By default, ingress resource will be created without any ingress class. If you want to use different class, you can set + ```hocon ingress { enabled: true, @@ -285,7 +303,7 @@ Deployment Manager of type `lite-embedded` has the following configuration optio | Parameter | Type | Default value | Description | |-----------------------------------------------------------|--------|-----------------|----------------------------------------------------------------------------------------------------------------------------------------------------------| -| mode | string | | Processing mode: either streaming-lite or request-response | +| mode | string | | Processing mode: either streaming-lite or request-response | | http.interface | string | 0.0.0.0 | (Request-Response only) Interface on which REST API of scenarios will be exposed | | http.port | int | 8181 | (Request-Response only) Port on which REST API of scenarios will be exposed | | request-response.definitionMetadata.servers | string | [{"url": "./"}] | (Request-Response only) Configuration of exposed servers in scenario's OpenAPI definition. When not configured, will be used server with ./ relative url | @@ -298,16 +316,20 @@ Deployment Manager of type `lite-embedded` has the following configuration optio Deployment Manager of type `flinkStreaming` has the following configuration options: -| Parameter | Type | Default value | Description | -|-------------------------------------|----------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| restUrl | string | | The only required parameter, REST API endpoint of the Flink cluster | -| jobManagerTimeout | duration | 1 minute | Timeout for communication with FLink cluster. Consider extending if e.g. you have long savepoint times etc. | -| shouldVerifyBeforeDeploy | boolean | true | By default, before redeployment of scenario with state from savepoint, verification of savepoint compatibility is performed. There are some cases when it can be too time consuming or not possible. Use this flag to disable it. | -| shouldCheckAvailableSlots | boolean | true | When set to true, Nussknacker checks if there are free slots to run new job. This check should be disabled on Flink Kubernetes Native deployments, where Taskmanager is started on demand. | -| waitForDuringDeployFinish.enabled | boolean | true | When set to true, after Flink job execution, we check if tasks were successfully started on TaskMangers, before marking version as deployed. Otherwise version is marked as deployed immediately after successful response from JobManager. | -| waitForDuringDeployFinish.maxChecks | boolean | 180 | It works when `waitForDuringDeployFinish.enabled` option is set to `true`. This parameter describe how many times we should check if tasks were successfully started on TaskMangers before notifying about deployment failure. | -| waitForDuringDeployFinish.delay | boolean | 1 second | It works when `waitForDuringDeployFinish.enabled` option is set to `true`. This parameter describe how long should be delay between checks. | -| scenarioStateCaching.enabled | boolean | true | Enables scenario state caching in scenario list view | -| scenarioStateCaching.cacheTTL | duration | 10 seconds | TimeToLeave for scenario state cache entries | -| scenarioStateRequestTimeout | duration | 3 seconds | Request timeout for fetching scenario state from Flink | -| jobConfigsCacheSize | int | 1000 | Maximum number of cached job configuration elements. | +| Parameter | Type | Default value | Description | +|--------------------------------------------------------------|----------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| restUrl | string | | The only required parameter, REST API endpoint of the Flink cluster | +| jobManagerTimeout | duration | 1 minute | Timeout for communication with FLink cluster. Consider extending if e.g. you have long savepoint times etc. | +| shouldVerifyBeforeDeploy | boolean | true | By default, before redeployment of scenario with state from savepoint, verification of savepoint compatibility is performed. There are some cases when it can be too time consuming or not possible. Use this flag to disable it. | +| shouldCheckAvailableSlots | boolean | true | When set to true, Nussknacker checks if there are free slots to run new job. This check should be disabled on Flink Kubernetes Native deployments, where Taskmanager is started on demand. | +| waitForDuringDeployFinish.enabled | boolean | true | When set to true, after Flink job execution, we check if tasks were successfully started on TaskMangers, before marking version as deployed. Otherwise version is marked as deployed immediately after successful response from JobManager. | +| waitForDuringDeployFinish.maxChecks | boolean | 180 | It works when `waitForDuringDeployFinish.enabled` option is set to `true`. This parameter describe how many times we should check if tasks were successfully started on TaskMangers before notifying about deployment failure. | +| waitForDuringDeployFinish.delay | boolean | 1 second | It works when `waitForDuringDeployFinish.enabled` option is set to `true`. This parameter describe how long should be delay between checks. | +| scenarioStateCaching.enabled | boolean | true | Enables scenario state caching in scenario list view | +| scenarioStateCaching.cacheTTL | duration | 10 seconds | TimeToLeave for scenario state cache entries | +| scenarioStateRequestTimeout | duration | 3 seconds | Request timeout for fetching scenario state from Flink | +| jobConfigsCacheSize | int | 1000 | Maximum number of cached job configuration elements. | +| scenarioTesting.reuseMiniClusterForScenarioTesting | boolean | true | Creates mini cluster once and reuses it for each scenario testing attempt | +| scenarioTesting.reuseMiniClusterForScenarioStateVerification | boolean | true | Creates mini cluster once and reuses it for each scenario state verification | +| scenarioTesting.parallelism | int | 1 | Parallelism that will be used for scenario testing and scenario state verifications mechanisms when mini cluster reusage is enabled; when mini cluster reusage is disabled, parallelism is taken from scenario properties | +| scenarioTesting.streamExecutionConfig | map of strings | [:] | Configuration that will be passed to `StreamExecutionEnvironment` for scenario testing and scenario state verifications mechanisms when mini cluster reusage is enabled; when mini cluster reusage is disabled, empty configuration will be used | From 8772153f26d30eb3859dc7a38fad3ba1a07550be Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Tue, 21 Jan 2025 16:49:40 +0100 Subject: [PATCH 23/48] debug logs --- .../AdHocMiniClusterFallbackHandler.scala | 18 ++++++--- .../scenariotesting/FlinkTestMain.scala | 37 ++++++++++--------- .../FlinkVerificationMain.scala | 29 ++++++++------- .../ScenarioTestingMiniClusterFactory.scala | 6 +-- .../ScenarioTestingMiniClusterWrapper.scala | 7 +++- 5 files changed, 58 insertions(+), 39 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala index 92126f5e063..dfc9d4dfacd 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala @@ -1,5 +1,6 @@ package pl.touk.nussknacker.engine.process.scenariotesting +import com.typesafe.scalalogging.LazyLogging import org.apache.flink.configuration.Configuration import pl.touk.nussknacker.engine.api.StreamMetaData import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess @@ -7,15 +8,22 @@ import pl.touk.nussknacker.engine.util.MetaDataExtractor // This class handles a legacy ad-hoc way to create minicluster. // After we fully switch to single mini cluster approach, it should be removed -object AdHocMiniClusterFallbackHandler { +object AdHocMiniClusterFallbackHandler extends LazyLogging { def handleAdHocMniClusterFallback[R]( reusableMiniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrapper], - scenario: CanonicalProcess + scenario: CanonicalProcess, + useCaseForDebug: String )(f: ScenarioTestingMiniClusterWrapper => R): R = { - val miniClusterWrapper = reusableMiniClusterWrapperOpt.getOrElse { - createAdHocMiniClusterWrapper(scenario) - } + val miniClusterWrapper = reusableMiniClusterWrapperOpt + .map { reusableMiniClusterWrapper => + logger.debug(s"reusableMiniClusterWrapper passed - using it for $useCaseForDebug") + reusableMiniClusterWrapper + } + .getOrElse { + logger.debug(s"reusableMiniClusterWrapper not passed - creating a new MiniCluster for $useCaseForDebug") + createAdHocMiniClusterWrapper(scenario) + } try { f(miniClusterWrapper) } finally { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala index 803a442a376..bf6f02931c4 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkTestMain.scala @@ -35,23 +35,26 @@ class FlinkTestMain(miniClusterWrapperOpt: Option[ScenarioTestingMiniClusterWrap def testScenario(scenario: CanonicalProcess, scenarioTestData: ScenarioTestData): TestResults[Json] = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener try { - AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback(miniClusterWrapperOpt, scenario) { - miniClusterWrapper => - val alignedScenario = miniClusterWrapper.alignParallelism(scenario) - val resultCollector = new TestServiceInvocationCollector(collectingListener) - // ProcessVersion can't be passed from DM because testing mechanism can be used with not saved scenario - val processVersion = ProcessVersion.empty.copy(processName = alignedScenario.name) - val deploymentData = DeploymentData.empty.copy(additionalModelConfigs = - AdditionalModelConfigs(modelData.additionalConfigsFromProvider) - ) - val registrar = prepareRegistrar(collectingListener, alignedScenario, scenarioTestData, processVersion) - registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) - miniClusterWrapper.submitJobAndCleanEnv( - alignedScenario.name, - SavepointRestoreSettings.none(), - modelData.modelClassLoader - ) - collectingListener.results + AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback( + miniClusterWrapperOpt, + scenario, + "scenario testing" + ) { miniClusterWrapper => + val alignedScenario = miniClusterWrapper.alignParallelism(scenario) + val resultCollector = new TestServiceInvocationCollector(collectingListener) + // ProcessVersion can't be passed from DM because testing mechanism can be used with not saved scenario + val processVersion = ProcessVersion.empty.copy(processName = alignedScenario.name) + val deploymentData = DeploymentData.empty.copy(additionalModelConfigs = + AdditionalModelConfigs(modelData.additionalConfigsFromProvider) + ) + val registrar = prepareRegistrar(collectingListener, alignedScenario, scenarioTestData, processVersion) + registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) + miniClusterWrapper.submitJobAndCleanEnv( + alignedScenario.name, + SavepointRestoreSettings.none(), + modelData.modelClassLoader + ) + collectingListener.results } } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala index 17e59cc17a0..5d43d07f9d6 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/FlinkVerificationMain.scala @@ -31,19 +31,22 @@ class FlinkVerificationMain( def runTest(scenario: CanonicalProcess, processVersion: ProcessVersion, savepointPath: String): Unit = { val collectingListener = ResultsCollectingListenerHolder.registerTestEngineListener try { - AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback(miniClusterWrapperOpt, scenario) { - miniClusterWrapper => - val alignedScenario = miniClusterWrapper.alignParallelism(scenario) - val resultCollector = new TestServiceInvocationCollector(collectingListener) - val registrar = prepareRegistrar(alignedScenario) - val deploymentData = DeploymentData.empty - - registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) - miniClusterWrapper.submitJobAndCleanEnv( - alignedScenario.name, - SavepointRestoreSettings.forPath(savepointPath, true), - modelData.modelClassLoader - ) + AdHocMiniClusterFallbackHandler.handleAdHocMniClusterFallback( + miniClusterWrapperOpt, + scenario, + "scenario state verification" + ) { miniClusterWrapper => + val alignedScenario = miniClusterWrapper.alignParallelism(scenario) + val resultCollector = new TestServiceInvocationCollector(collectingListener) + val registrar = prepareRegistrar(alignedScenario) + val deploymentData = DeploymentData.empty + + registrar.register(miniClusterWrapper.env, alignedScenario, processVersion, deploymentData, resultCollector) + miniClusterWrapper.submitJobAndCleanEnv( + alignedScenario.name, + SavepointRestoreSettings.forPath(savepointPath, true), + modelData.modelClassLoader + ) } } finally { collectingListener.clean() diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala index eb978ff6bca..fbdeee9fbfd 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala @@ -6,13 +6,13 @@ import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfigurati object ScenarioTestingMiniClusterFactory { - def createConfiguredMiniCluster(nomTaskSlots: Int): MiniCluster = { - val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = nomTaskSlots) + def createConfiguredMiniCluster(numTaskSlots: Int): MiniCluster = { + val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = numTaskSlots) // it is required for proper working of HadoopFileSystem FileSystem.initialize(miniClusterConfiguration, null) - createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = nomTaskSlots) + createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = numTaskSlots) } private def prepareMiniClusterConfiguration(numTaskSlots: Int) = { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala index caef70c568e..96baca04f7b 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -1,5 +1,6 @@ package pl.touk.nussknacker.engine.process.scenariotesting +import com.typesafe.scalalogging.LazyLogging import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.jobgraph.{JobGraph, SavepointRestoreSettings} import org.apache.flink.runtime.minicluster.MiniCluster @@ -67,10 +68,14 @@ final class ScenarioTestingMiniClusterWrapper( } -object ScenarioTestingMiniClusterWrapper { +object ScenarioTestingMiniClusterWrapper extends LazyLogging { def create(parallelism: Int, streamExecutionConfig: Configuration): ScenarioTestingMiniClusterWrapper = { + logger.debug(s"Creating MiniCluster with numTaskSlots = $parallelism") val miniCluster = ScenarioTestingMiniClusterFactory.createConfiguredMiniCluster(parallelism) + logger.debug( + s"Creating local StreamExecutionEnvironment with parallelism = $parallelism and configuration = $streamExecutionConfig" + ) val env = ScenarioTestingStreamExecutionEnvironmentFactory.createStreamExecutionEnvironment( parallelism, streamExecutionConfig From 0caeb4f1d1648ece962bec15f52f64afa57c0996 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 11:34:28 +0100 Subject: [PATCH 24/48] mini cluster create eagerly --- .../engine/management/FlinkDeploymentManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala index 180263c7b7c..22f30f671b3 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkDeploymentManager.scala @@ -33,18 +33,18 @@ abstract class FlinkDeploymentManager( import dependencies._ - private lazy val scenarioTestingMiniClusterWrapperOpt = + private val scenarioTestingMiniClusterWrapperOpt = ScenarioTestingMiniClusterWrapperFactory.createIfConfigured( modelData.asInvokableModelData.modelClassLoader, scenarioTestingConfig ) - private lazy val testRunner = new FlinkProcessTestRunner( + private val testRunner = new FlinkProcessTestRunner( modelData.asInvokableModelData, scenarioTestingMiniClusterWrapperOpt.filter(_ => scenarioTestingConfig.reuseMiniClusterForScenarioTesting) ) - private lazy val verification = new FlinkProcessVerifier( + private val verification = new FlinkProcessVerifier( modelData.asInvokableModelData, scenarioTestingMiniClusterWrapperOpt.filter(_ => scenarioTestingConfig.reuseMiniClusterForScenarioStateVerification) ) From 7142c51e59d591fa5845a676d721c750a85d617a Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 14:46:28 +0100 Subject: [PATCH 25/48] e2e tests fix attempt: less memory for task managers --- .../batch-data-generation/batch-customizations.conf | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf index 8fa5e63979c..bd0cec367d2 100644 --- a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf +++ b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf @@ -10,6 +10,13 @@ baseBatchConfig { deploymentConfig: { type: "flinkStreaming" restUrl: ${?FLINK_REST_URL} + scenarioTesting { + streamExecutionConfig { + taskmanager.memory.network.min: 16m + taskmanager.memory.network.max: 16m + taskmanager.memory.managed.size: 50m + } + } } modelConfig: { classPath: ["model/defaultModel.jar", "components/flink/flinkBase.jar", "components/flink-table/flinkTable.jar", "model/flinkExecutor.jar", "flink-dropwizard-metrics-deps/"] From e08b9f1d4febe77d5a724f1ca09776d0a9f66035 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 19:54:14 +0100 Subject: [PATCH 26/48] additional logs for docker-compose + temporary disable other pipeline steps --- .github/workflows/pr.yml | 845 +++++++++--------- .../src/main/resources/logback-test.xml | 1 + 2 files changed, 424 insertions(+), 422 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index 7196b1f7ee3..d9b3484f573 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -115,56 +115,56 @@ jobs: name: build-target path: target.tgz - build-fe: - name: BuildFrontend - runs-on: ubuntu-latest - needs: [ setup ] - env: - NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - name: Cache npm - id: cache-npm - uses: actions/cache@v3 - with: - path: | - **/node_modules - ~/.cache/Cypress - key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} - - name: Get Node.js version - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - name: Download node modules - if: steps.cache-npm.outputs.cache-hit != 'true' - run: | - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - - name: Build FE - run: (cd designer/client; npm run build) - - name: Build FE submodules - run: | - cd designer - cp -r client/.federated-types/nussknackerUi submodules/types/@remote - cd submodules - npm run build - - name: Tar fe artifacts - shell: bash - run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist - - name: Store dist - uses: actions/upload-artifact@v4 - with: - name: build-fe-dist - path: fe-dist.tgz - +# build-fe: +# name: BuildFrontend +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - name: Cache npm +# id: cache-npm +# uses: actions/cache@v3 +# with: +# path: | +# **/node_modules +# ~/.cache/Cypress +# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} +# - name: Get Node.js version +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - name: Download node modules +# if: steps.cache-npm.outputs.cache-hit != 'true' +# run: | +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# - name: Build FE +# run: (cd designer/client; npm run build) +# - name: Build FE submodules +# run: | +# cd designer +# cp -r client/.federated-types/nussknackerUi submodules/types/@remote +# cd submodules +# npm run build +# - name: Tar fe artifacts +# shell: bash +# run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist +# - name: Store dist +# uses: actions/upload-artifact@v4 +# with: +# name: build-fe-dist +# path: fe-dist.tgz +# tests: name: Tests runs-on: ubuntu-latest @@ -173,7 +173,8 @@ jobs: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we # have to run these tests always for every scala version. See NuDesignerApiAvailableToExposeYamlSpec - scalaVersion: [2.12, 2.13] +# scalaVersion: [2.12, 2.13] + scalaVersion: [2.12] env: NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} @@ -212,374 +213,374 @@ jobs: with: name: BackendTests-results-${{ matrix.scalaVersion }} path: '**/test-reports/*.xml' - - integrationTests: - name: IntegrationTests - needs: [ build, setup ] - strategy: - matrix: - scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} - env: - NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} - if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - runs-on: ubuntu-latest - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - uses: coursier/setup-action@v1 - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz - - uses: sbt/setup-sbt@v1 - - name: Integration tests - shell: bash - env: - dockerUpLatest: true - run: ./ciRunSbt.sh It/test - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: IntegrationTests-results-${{ matrix.scalaVersion }} - path: '**/it-reports/*.xml' - - slowTests: - name: SlowTests - runs-on: ubuntu-latest - needs: [ build, setup ] - strategy: - matrix: - scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} - env: - NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} - if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - uses: coursier/setup-action@v1 - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz - # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally - - uses: AbsaOSS/k3d-action@v2.4.0 - env: - PROJECT_ROOT: ${{ github.workspace }} - with: - cluster-name: "k3s-default" - k3d-version: "v5.6.0" - args: >- - --config=.k3d/single-cluster.yml - --trace - - name: Slow tests - env: - AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} - AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} - AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} - AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} - AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} - shell: bash - run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test - - name: docker logs - if: success() || failure() - run: | - docker logs k3d-k3s-default-server-0 - docker logs k3d-k3s-default-agent-0 - docker logs k3d-k3s-default-serverlb - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: SlowTests-results-${{ matrix.scalaVersion }} - path: '**/test-reports/*.xml' - - frontendTests: - name: FrontendTests - runs-on: ubuntu-latest - needs: [ setup ] - env: - # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. - shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} - steps: - - name: Cancel previous runs - if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - if: ${{ env.shouldPerformFrontendTests == 'true' }} - - name: Cache npm - id: cache-npm - if: ${{ env.shouldPerformFrontendTests == 'true' }} - uses: actions/cache@v3 - with: - path: | - **/node_modules - ~/.cache/Cypress - key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} - - name: Get Node.js version - if: ${{ env.shouldPerformFrontendTests == 'true' }} - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - if: ${{ env.shouldPerformFrontendTests == 'true' }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - name: Download node modules - if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} - run: | - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - - name: Test FE - if: ${{ env.shouldPerformFrontendTests == 'true' }} - run: (cd designer/client; npm run test:unit) - - name: Test Report - if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} - uses: actions/upload-artifact@v4 - with: - name: FrontendTests-results - path: designer/client/junit.xml - - cypressTests: - name: CypressTests - runs-on: self-hosted - needs: [ build, build-fe, setup ] - env: - # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests - # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we - # have this conditional logic in this step. We force building images on our "special" branches because run between merges - # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). - shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} - BE_PORT: 7251 - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all - - name: Clean all docker containers - shell: bash - run: | - c=$(docker ps -q) - [[ $c ]] && docker kill $c || echo "No container run" - - uses: actions/checkout@v3 - - name: Get Node.js version - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - uses: coursier/setup-action@v1 - if: ${{ env.shouldBuildImage == 'true' }} - with: - jvm: temurin:1.11.0.17 - - name: Download node modules - run: | - export npm_config_cache=$(mktemp -d) - npm config ls -l | grep cache - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - rm -rf $npm_config_cache - - uses: actions/download-artifact@v4 - if: ${{ env.shouldBuildImage == 'true' }} - with: - name: build-target - - name: Untar artifacts - if: ${{ env.shouldBuildImage == 'true' }} - shell: bash - run: tar xfz target.tgz - - uses: actions/download-artifact@v4 - with: - name: build-fe-dist - - name: Untar fe artifacts - shell: bash - run: tar xfz fe-dist.tgz - - name: set version - if: ${{ env.shouldBuildImage == 'true' }} - shell: bash - run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV - - name: Prepare docker - if: ${{ env.shouldBuildImage == 'true' }} - env: - addDevArtifacts: true - shell: bash - #Doc generation is rather costly, we don't want it in test image creation - run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal - - name: FE tests e2e on build docker image - if: ${{ env.shouldBuildImage == 'true' }} - env: - CYPRESS_SNAPSHOT_UPDATE: "true" - CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} - DOCKER_PULL_OPTION: never - shell: bash - run: | - cd designer/client - ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) - npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e - - name: Determine docker tag version to use - if: ${{ env.shouldBuildImage == 'false' }} - shell: bash - run: | - # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. - NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` - NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` - echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV - - name: FE tests e2e on pulled image - if: ${{ env.shouldBuildImage == 'false' }} - env: - CYPRESS_SNAPSHOT_UPDATE: "true" - shell: bash - run: | - cd designer/client - ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) - npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: CypressTests-results - path: designer/client/cypress-test-results/*.xml - - name: Store test results - if: failure() - uses: actions/upload-artifact@v4 - with: - name: e2e-test-results - path: | - docs/autoScreenshotChangeDocs/ - designer/client/cypress/**/__image_snapshots__/ - designer/client/cypress/screenshots/ - designer/client/cypress/videos/ - if-no-files-found: ignore - - name: Create Pull Request - id: update_snapshots - uses: peter-evans/create-pull-request@v7 - if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) - env: - HASH: ${{ format('#{0}', github.event.number) }} - BRANCH: ${{ needs.setup.outputs.git_source_branch }} - with: - token: ${{ secrets.GITHUB_TOKEN }} - base: ${{ env.BRANCH }} - branch: snapshots-patch/${{ env.BRANCH }} - title: Update Cypress snapshots in ${{ env.BRANCH }} - commit-message: Updated snapshots - body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} - labels: cypress - - name: Comment PR - if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} - uses: thollander/actions-comment-pull-request@v2 - with: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - message: | - ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} - :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. - comment_tag: snapshots_pr - - name: Force fail if update PR created - if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} - uses: actions/github-script@v6.3.3 - with: - script: | - core.setFailed('Snapshots updated, chceck comments!') - - publish-after-tests: - name: Publish With Tests - needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] - uses: ./.github/workflows/publish.yml - with: - should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - secrets: - nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} - git_source_branch: ${{ needs.setup.outputs.git_source_branch }} - sonatype_user: ${{ secrets.SONATYPE_USER }} - sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} - github__token: ${{ secrets.GITHUB_TOKEN }} - dockerhub_user: ${{ secrets.DOCKERHUB_USER }} - dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} - - publish-before-tests: - name: Publish Without Tests - needs: ['build', 'build-fe', 'setup'] - uses: ./.github/workflows/publish.yml - with: - should_run: ${{ github.ref == 'refs/heads/staging' }} - secrets: - nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} - git_source_branch: ${{ needs.setup.outputs.git_source_branch }} - sonatype_user: ${{ secrets.SONATYPE_USER }} - sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} - github__token: ${{ secrets.GITHUB_TOKEN }} - dockerhub_user: ${{ secrets.DOCKERHUB_USER }} - dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} - - trigger-external-build: - runs-on: ubuntu-latest - needs: [ setup, publish-before-tests ] - if: ${{ github.ref == 'refs/heads/staging' }} - strategy: - matrix: - include: - - repo: nussknacker-helm - workflow_id: helm-test-workflow.yaml - ref: main - - repo: nussknacker-quickstart - workflow_id: pr.yml - ref: staging - - repo: nussknacker-sample-components - workflow_id: pr.yml - ref: staging - - repo: nussknacker-flink-compatibility - workflow_id: test.yml - ref: staging - steps: - - uses: actions/github-script@v6 - with: - github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} - script: | - await github.rest.actions.createWorkflowDispatch({ - owner: context.repo.owner, - repo: '${{ matrix.repo }}', - workflow_id: '${{ matrix.workflow_id }}', - ref: '${{ matrix.ref }}', - inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} - }) +# +# integrationTests: +# name: IntegrationTests +# needs: [ build, setup ] +# strategy: +# matrix: +# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} +# env: +# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} +# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# runs-on: ubuntu-latest +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - uses: coursier/setup-action@v1 +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz +# - uses: sbt/setup-sbt@v1 +# - name: Integration tests +# shell: bash +# env: +# dockerUpLatest: true +# run: ./ciRunSbt.sh It/test +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: IntegrationTests-results-${{ matrix.scalaVersion }} +# path: '**/it-reports/*.xml' +# +# slowTests: +# name: SlowTests +# runs-on: ubuntu-latest +# needs: [ build, setup ] +# strategy: +# matrix: +# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} +# env: +# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} +# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - uses: coursier/setup-action@v1 +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz +# # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally +# - uses: AbsaOSS/k3d-action@v2.4.0 +# env: +# PROJECT_ROOT: ${{ github.workspace }} +# with: +# cluster-name: "k3s-default" +# k3d-version: "v5.6.0" +# args: >- +# --config=.k3d/single-cluster.yml +# --trace +# - name: Slow tests +# env: +# AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} +# AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} +# AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} +# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} +# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} +# shell: bash +# run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test +# - name: docker logs +# if: success() || failure() +# run: | +# docker logs k3d-k3s-default-server-0 +# docker logs k3d-k3s-default-agent-0 +# docker logs k3d-k3s-default-serverlb +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: SlowTests-results-${{ matrix.scalaVersion }} +# path: '**/test-reports/*.xml' +# +# frontendTests: +# name: FrontendTests +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. +# shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} +# steps: +# - name: Cancel previous runs +# if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# - name: Cache npm +# id: cache-npm +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# uses: actions/cache@v3 +# with: +# path: | +# **/node_modules +# ~/.cache/Cypress +# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} +# - name: Get Node.js version +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - name: Download node modules +# if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} +# run: | +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# - name: Test FE +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# run: (cd designer/client; npm run test:unit) +# - name: Test Report +# if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} +# uses: actions/upload-artifact@v4 +# with: +# name: FrontendTests-results +# path: designer/client/junit.xml +# +# cypressTests: +# name: CypressTests +# runs-on: self-hosted +# needs: [ build, build-fe, setup ] +# env: +# # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests +# # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we +# # have this conditional logic in this step. We force building images on our "special" branches because run between merges +# # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). +# shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} +# BE_PORT: 7251 +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all +# - name: Clean all docker containers +# shell: bash +# run: | +# c=$(docker ps -q) +# [[ $c ]] && docker kill $c || echo "No container run" +# - uses: actions/checkout@v3 +# - name: Get Node.js version +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - uses: coursier/setup-action@v1 +# if: ${{ env.shouldBuildImage == 'true' }} +# with: +# jvm: temurin:1.11.0.17 +# - name: Download node modules +# run: | +# export npm_config_cache=$(mktemp -d) +# npm config ls -l | grep cache +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# rm -rf $npm_config_cache +# - uses: actions/download-artifact@v4 +# if: ${{ env.shouldBuildImage == 'true' }} +# with: +# name: build-target +# - name: Untar artifacts +# if: ${{ env.shouldBuildImage == 'true' }} +# shell: bash +# run: tar xfz target.tgz +# - uses: actions/download-artifact@v4 +# with: +# name: build-fe-dist +# - name: Untar fe artifacts +# shell: bash +# run: tar xfz fe-dist.tgz +# - name: set version +# if: ${{ env.shouldBuildImage == 'true' }} +# shell: bash +# run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV +# - name: Prepare docker +# if: ${{ env.shouldBuildImage == 'true' }} +# env: +# addDevArtifacts: true +# shell: bash +# #Doc generation is rather costly, we don't want it in test image creation +# run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal +# - name: FE tests e2e on build docker image +# if: ${{ env.shouldBuildImage == 'true' }} +# env: +# CYPRESS_SNAPSHOT_UPDATE: "true" +# CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} +# DOCKER_PULL_OPTION: never +# shell: bash +# run: | +# cd designer/client +# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) +# npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e +# - name: Determine docker tag version to use +# if: ${{ env.shouldBuildImage == 'false' }} +# shell: bash +# run: | +# # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. +# NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` +# NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` +# echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV +# - name: FE tests e2e on pulled image +# if: ${{ env.shouldBuildImage == 'false' }} +# env: +# CYPRESS_SNAPSHOT_UPDATE: "true" +# shell: bash +# run: | +# cd designer/client +# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) +# npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: CypressTests-results +# path: designer/client/cypress-test-results/*.xml +# - name: Store test results +# if: failure() +# uses: actions/upload-artifact@v4 +# with: +# name: e2e-test-results +# path: | +# docs/autoScreenshotChangeDocs/ +# designer/client/cypress/**/__image_snapshots__/ +# designer/client/cypress/screenshots/ +# designer/client/cypress/videos/ +# if-no-files-found: ignore +# - name: Create Pull Request +# id: update_snapshots +# uses: peter-evans/create-pull-request@v7 +# if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) +# env: +# HASH: ${{ format('#{0}', github.event.number) }} +# BRANCH: ${{ needs.setup.outputs.git_source_branch }} +# with: +# token: ${{ secrets.GITHUB_TOKEN }} +# base: ${{ env.BRANCH }} +# branch: snapshots-patch/${{ env.BRANCH }} +# title: Update Cypress snapshots in ${{ env.BRANCH }} +# commit-message: Updated snapshots +# body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} +# labels: cypress +# - name: Comment PR +# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} +# uses: thollander/actions-comment-pull-request@v2 +# with: +# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} +# message: | +# ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} +# :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. +# comment_tag: snapshots_pr +# - name: Force fail if update PR created +# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} +# uses: actions/github-script@v6.3.3 +# with: +# script: | +# core.setFailed('Snapshots updated, chceck comments!') +# +# publish-after-tests: +# name: Publish With Tests +# needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] +# uses: ./.github/workflows/publish.yml +# with: +# should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# secrets: +# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} +# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} +# sonatype_user: ${{ secrets.SONATYPE_USER }} +# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} +# github__token: ${{ secrets.GITHUB_TOKEN }} +# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} +# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} +# +# publish-before-tests: +# name: Publish Without Tests +# needs: ['build', 'build-fe', 'setup'] +# uses: ./.github/workflows/publish.yml +# with: +# should_run: ${{ github.ref == 'refs/heads/staging' }} +# secrets: +# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} +# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} +# sonatype_user: ${{ secrets.SONATYPE_USER }} +# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} +# github__token: ${{ secrets.GITHUB_TOKEN }} +# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} +# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} +# +# trigger-external-build: +# runs-on: ubuntu-latest +# needs: [ setup, publish-before-tests ] +# if: ${{ github.ref == 'refs/heads/staging' }} +# strategy: +# matrix: +# include: +# - repo: nussknacker-helm +# workflow_id: helm-test-workflow.yaml +# ref: main +# - repo: nussknacker-quickstart +# workflow_id: pr.yml +# ref: staging +# - repo: nussknacker-sample-components +# workflow_id: pr.yml +# ref: staging +# - repo: nussknacker-flink-compatibility +# workflow_id: test.yml +# ref: staging +# steps: +# - uses: actions/github-script@v6 +# with: +# github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} +# script: | +# await github.rest.actions.createWorkflowDispatch({ +# owner: context.repo.owner, +# repo: '${{ matrix.repo }}', +# workflow_id: '${{ matrix.workflow_id }}', +# ref: '${{ matrix.ref }}', +# inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} +# }) diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 6211e9153da..88cfafe4b35 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -70,6 +70,7 @@ + From 16fd8a63d129f224cd02e1ae8310bb75dad50a26 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 20:27:25 +0100 Subject: [PATCH 27/48] additional logs for docker-compose --- utils/test-utils/src/main/resources/logback-test.xml | 4 ++-- .../DockerBasedNuInstallationExampleEnvironment.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 88cfafe4b35..1abfc2a94f9 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -69,8 +69,8 @@ - - + + diff --git a/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala b/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala index e2dd40a5272..e0622d89710 100644 --- a/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala +++ b/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala @@ -43,7 +43,7 @@ class DockerBasedInstallationExampleNuEnvironment( ) ), // Change to 'true' to enable logging - tailChildContainers = false + tailChildContainers = true ) { start() From 19b943c1fe9b0928c6b02b0d2ba026a081603aa2 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 23:43:10 +0100 Subject: [PATCH 28/48] additional logs for docker-compose --- .github/workflows/pr.yml | 3 ++- utils/test-utils/src/main/resources/logback-test.xml | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index d9b3484f573..fffd950e94d 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -206,7 +206,8 @@ jobs: restore-keys: sbt - name: Backend tests shell: bash - run: ./ciRunSbt.sh test +# run: ./ciRunSbt.sh test + run: ./ciRunSbt.sh e2eTests/test - name: Test Report if: success() || failure() uses: actions/upload-artifact@v4 diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 1abfc2a94f9..70bcedc88ca 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -13,7 +13,7 @@ - + From 4ba06728f02f8de5a4f3eaabcc720b21009d42d6 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Thu, 23 Jan 2025 23:52:15 +0100 Subject: [PATCH 29/48] faster feedback --- .github/workflows/pr.yml | 93 ++++++++++++++++++++-------------------- 1 file changed, 47 insertions(+), 46 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index fffd950e94d..f0f76ae36e7 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -70,51 +70,51 @@ jobs: fi - build: - name: Build - runs-on: ubuntu-latest - needs: [ setup ] - env: - # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one - shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - steps: - - name: Cancel previous runs - if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - if: ${{ env.shouldPerformBackendBuild == 'true' }} - - uses: coursier/setup-action@v1 - if: ${{ env.shouldPerformBackendBuild == 'true' }} - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - if: ${{ env.shouldPerformBackendBuild == 'true' }} - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - name: Build - if: ${{ env.shouldPerformBackendBuild == 'true' }} - shell: bash - run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile - - name: Tar artifacts - if: ${{ env.shouldPerformBackendBuild == 'true' }} - shell: bash - run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' - - name: Store target - if: ${{ env.shouldPerformBackendBuild == 'true' }} - uses: actions/upload-artifact@v4 - with: - name: build-target - path: target.tgz - +# build: +# name: Build +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one +# shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# steps: +# - name: Cancel previous runs +# if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# - uses: coursier/setup-action@v1 +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - name: Build +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# shell: bash +# run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile +# - name: Tar artifacts +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# shell: bash +# run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' +# - name: Store target +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# uses: actions/upload-artifact@v4 +# with: +# name: build-target +# path: target.tgz +# # build-fe: # name: BuildFrontend # runs-on: ubuntu-latest @@ -168,7 +168,8 @@ jobs: tests: name: Tests runs-on: ubuntu-latest - needs: [ build, setup ] +# needs: [ build, setup ] + needs: [ setup ] strategy: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we From 4d8d609550f68e2c19b9b60da706e583df39300c Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 00:06:43 +0100 Subject: [PATCH 30/48] faster feedback --- examples/installation/docker-compose.yml | 2 +- utils/test-utils/src/main/resources/logback-test.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/installation/docker-compose.yml b/examples/installation/docker-compose.yml index 030dd6a73b7..3b9302c4709 100644 --- a/examples/installation/docker-compose.yml +++ b/examples/installation/docker-compose.yml @@ -59,7 +59,7 @@ services: healthcheck: test: [ "CMD-SHELL", "curl localhost:8080/api/app/healthCheck" ] interval: 10s - retries: 10 + retries: 15 volumes: - ./designer/application-customizations.conf:/opt/nussknacker/conf/application-customizations.conf volumes_from: diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 70bcedc88ca..2ed3a655240 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -13,12 +13,12 @@ - + - + From 71fb87ced8011f89a6fa510807ec1a47a0eec578 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 00:10:40 +0100 Subject: [PATCH 31/48] faster feedback --- .github/workflows/pr.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index f0f76ae36e7..81b8b4254ac 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -189,12 +189,12 @@ jobs: - uses: coursier/setup-action@v1 with: jvm: temurin:1.11.0.17 - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz - uses: sbt/setup-sbt@v1 - name: Cache ivy packages uses: actions/cache@v3 From 8cf5955bcfcec3ee195df1670c621298e3e4c005 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 11:06:06 +0100 Subject: [PATCH 32/48] fix attempt: passing stream connfiguration to jobgraph --- .../ScenarioTestingMiniClusterWrapper.scala | 7 +- .../FlinkProcessTestRunnerSpec.scala | 1316 ++++++++--------- 2 files changed, 663 insertions(+), 660 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala index 96baca04f7b..7e2b8a7d621 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -17,7 +17,8 @@ import scala.jdk.CollectionConverters._ final class ScenarioTestingMiniClusterWrapper( miniCluster: MiniCluster, val env: StreamExecutionEnvironment, - parallelism: Int + parallelism: Int, + streamExecutionConfig: Configuration ) extends AutoCloseable { def alignParallelism(canonicalProcess: CanonicalProcess): CanonicalProcess = { @@ -59,6 +60,8 @@ final class ScenarioTestingMiniClusterWrapper( ): Unit = { jobGraph.setClasspaths(modelClassLoader.urls.asJava) jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) + // FIXME abr: Is it needed? + jobGraph.getJobConfiguration.addAll(streamExecutionConfig) } def close(): Unit = { @@ -80,7 +83,7 @@ object ScenarioTestingMiniClusterWrapper extends LazyLogging { parallelism, streamExecutionConfig ) - new ScenarioTestingMiniClusterWrapper(miniCluster, env, parallelism) + new ScenarioTestingMiniClusterWrapper(miniCluster, env, parallelism, streamExecutionConfig) } } diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala index 7f073eb8a87..1c7f89a1c81 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala @@ -95,9 +95,9 @@ class FlinkProcessTestRunnerSpec "IO monad interpreter is used" should { runTests(useIOMonadInInterpreter = true) } - "IO monad interpreter is NOT used" should { - runTests(useIOMonadInInterpreter = false) - } +// "IO monad interpreter is NOT used" should { +// runTests(useIOMonadInInterpreter = false) +// } } private def runTests(useIOMonadInInterpreter: Boolean): Unit = { @@ -165,661 +165,661 @@ class FlinkProcessTestRunnerSpec LogService.invocationsCount.get() shouldBe 0 } - "be able to run tests multiple time on the same mini cluster" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - - val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") - - val testRunner = prepareTestRunner(useIOMonadInInterpreter) - - def runTestAndVerify() = { - val results = testRunner.runTests( - process, - ScenarioTestData( - List( - ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) - ) - ) - ) - - val nodeResults = results.nodeResults - - nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input)) - nodeResults("out") shouldBe List(nodeResult(0, "input" -> input)) - - val invocationResults = results.invocationResults - - invocationResults("out") shouldBe - List(ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable(11))) - - results.externalInvocationResults("out") shouldBe List( - ExternalInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "valueMonitor", variable(11)) - ) - } - - runTestAndVerify() - runTestAndVerify() - } - - "collect results for split" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), - ) - - results.nodeResults("splitId1") shouldBe List( - nodeResult( - 0, - "input" -> - SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") - ), - nodeResult( - 1, - "input" -> - SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") - ) - ) - } - - "return correct result for custom node" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .customNode("cid", "out", "stateCustom", "groupBy" -> "#input.id".spel, "stringVal" -> "'s'".spel) - .emptySink("out", "valueMonitor", "Value" -> "#input.value1 + ' ' + #out.previous".spel) - - val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") - val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") - - val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") - val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), - ) - - val nodeResults = results.nodeResults - - nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) - nodeResults("cid") shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) - nodeResults("out") shouldBe List( - nodeResult(0, "input" -> input, "out" -> aggregate), - nodeResult(1, "input" -> input2, "out" -> aggregate2) - ) - - val invocationResults = results.invocationResults - - invocationResults("cid") shouldBe - List( - // we record only LazyParameter execution results - ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "groupBy", variable("0")), - ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "groupBy", variable("0")) - ) - - invocationResults("out") shouldBe - List( - ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable("1 0")), - ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "Value", variable("11 1")) - ) - - results.externalInvocationResults("out") shouldBe - List( - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", - "valueMonitor", - variable("1 0") - ), - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", - "valueMonitor", - variable("11 1") - ) - ) - } - - "handle large parallelism" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .parallelism(4) - .source(sourceNodeId, "input") - .emptySink("out", "monitor") - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), - ) - - val nodeResults = results.nodeResults - - nodeResults(sourceNodeId) should have length 5 - } - - "detect errors" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) - .filter("filter", "1 / #input.value1 >= 0".spel) - .emptySink("out", "monitor") - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData( - List( - createTestRecord(id = "0", value1 = 1), - createTestRecord(id = "1", value1 = 0), - createTestRecord(id = "2", value1 = 2), - createTestRecord(id = "3", value1 = 4) - ) - ), - ) - - val nodeResults = results.nodeResults - - nodeResults(sourceNodeId) should have length 4 - nodeResults("out") should have length 2 - - results.exceptions should have length 2 - - val exceptionFromExpression = results.exceptions.head - exceptionFromExpression.nodeId shouldBe Some("filter") - exceptionFromExpression.context - .variables("input") - .asInstanceOf[Json] - .hcursor - .downField("pretty") - .focus - .value - .toString() - .startsWith("SimpleJsonRecord(1") // it's not nice.. - exceptionFromExpression.throwable.getMessage shouldBe "Expression [1 / #input.value1 >= 0] evaluation failed, message: / by zero" - - val exceptionFromService = results.exceptions.last - exceptionFromService.nodeId shouldBe Some("failing") - exceptionFromService.context - .variables("input") - .asInstanceOf[Json] - .hcursor - .downField("pretty") - .focus - .value - .toString() - .startsWith("SimpleJsonRecord(2") // it's not nice.. - exceptionFromService.throwable.getMessage shouldBe "Thrown as expected" - } - - "ignore real exception handler" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) - .filter("filter", "1 / #input.value1 >= 0".spel) - .emptySink("out", "monitor") - - val exceptionConsumerId = UUID.randomUUID().toString - val results = prepareTestRunner( - useIOMonadInInterpreter, - enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) - ).runTests( - process, - scenarioTestData = ScenarioTestData( - List( - createTestRecord(id = "0", value1 = 1), - createTestRecord(id = "1", value1 = 0), - createTestRecord(id = "2", value1 = 2), - createTestRecord(id = "3", value1 = 4) - ) - ) - ) - - val nodeResults = results.nodeResults - - nodeResults(sourceNodeId) should have length 4 - nodeResults("out") should have length 2 - - results.exceptions should have length 2 - RecordingExceptionConsumer.exceptionsFor(exceptionConsumerId) shouldBe Symbol("empty") - } - - "handle transient errors" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .processor("failing", "throwingTransientService", "throw" -> "#input.value1 == 2".spel) - .emptySink("out", "monitor") - - intercept[JobExecutionException] { - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) - ) - } - } - - "handle json input" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "jsonInput") - .emptySink("out", "valueMonitor", "Value" -> "#input".spel) - val testData = ScenarioTestData( - List( - ScenarioTestJsonRecord( - sourceNodeId, - Json.obj(sourceNodeId -> Json.fromString("1"), "field" -> Json.fromString("11")) - ), - ScenarioTestJsonRecord( - sourceNodeId, - Json.obj(sourceNodeId -> Json.fromString("2"), "field" -> Json.fromString("22")) - ), - ScenarioTestJsonRecord( - sourceNodeId, - Json.obj(sourceNodeId -> Json.fromString("3"), "field" -> Json.fromString("33")) - ), - ) - ) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) - - results.nodeResults(sourceNodeId) should have size 3 - results.externalInvocationResults("out") shouldBe - List( - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", - "valueMonitor", - variable(SimpleJsonRecord("1", "11")) - ), - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", - "valueMonitor", - variable(SimpleJsonRecord("2", "22")) - ), - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2", - "valueMonitor", - variable(SimpleJsonRecord("3", "33")) - ) - ) - } - - "handle custom variables in source" in { - val process = ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "genericSourceWithCustomVariables", "elements" -> "{'abc'}".spel) - .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) - val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) - - results.nodeResults(sourceNodeId) should have size 1 - results.externalInvocationResults("out") shouldBe - List( - ExternalInvocationResult( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", - "valueMonitor", - variable("transformed:abc|3") - ) - ) - } - - "give meaningful error messages for sink errors" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) - ) - - results.exceptions should have length 1 - results.exceptions.head.nodeId shouldBe Some("out") - results.exceptions.head.throwable.getMessage should include("message: / by zero") - } - - "be able to test process with time windows" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .customNode("cid", "count", "transformWithTime", "seconds" -> "10".spel) - .emptySink("out", "monitor") - - def recordWithSeconds(duration: FiniteDuration) = - ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData( - List( - recordWithSeconds(1 second), - recordWithSeconds(2 second), - recordWithSeconds(5 second), - recordWithSeconds(9 second), - recordWithSeconds(20 second) - ) - ) - ) - - val nodeResults = results.nodeResults - - nodeResults("out").map(_.variables) shouldBe List(Map("count" -> variable(4)), Map("count" -> variable(1))) - - } - - "be able to test typed map" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source( - sourceNodeId, - "typedJsonInput", - "type" -> """{"field1": "String", "field2": "java.lang.String"}""".spel - ) - .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData( - ScenarioTestJsonRecord( - sourceNodeId, - Json.obj("field1" -> Json.fromString("abc"), "field2" -> Json.fromString("def")) - ) :: Nil - ) - ) - - results.invocationResults("out").map(_.value) shouldBe List(variable("abcdef")) - } - - "using dependent services" in { - val countToPass = 15 - val valueToReturn = 18 - - val process = ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .enricher( - "dependent", - "parsed", - "returningDependentTypeService", - "definition" -> "{'field1', 'field2'}".spel, - "toFill" -> "#input.value1.toString()".spel, - "count" -> countToPass.toString.spel - ) - .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(value1 = valueToReturn))) - ) - - results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) - } - - "switch value should be equal to variable value" in { - val process = ScenarioBuilder - .streaming(scenarioName) - .parallelism(1) - .source(sourceNodeId, "input") - .switch( - "switch", - "#input.id == 'ala'".spel, - "output", - Case( - "#output == false".spel, - GraphBuilder.emptySink("out", "valueMonitor", "Value" -> "'any'".spel) - ) - ) - - val recordTrue = createTestRecord(id = "ala") - val recordFalse = createTestRecord(id = "bela") - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) - - val invocationResults = results.invocationResults - - invocationResults("switch").filter(_.name == "expression").head.value shouldBe variable(true) - invocationResults("switch").filter(_.name == "expression").last.value shouldBe variable(false) - // first record was filtered out - invocationResults("out").head.contextId shouldBe s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1" - } - - "should handle joins for one input (diamond-like) " in { - val process = ScenarioBuilder - .streaming(scenarioName) - .sources( - GraphBuilder - .source(sourceNodeId, "input") - .split( - "split", - GraphBuilder.filter("left", "#input.id != 'a'".spel).branchEnd("end1", "join1"), - GraphBuilder.filter("right", "#input.id != 'b'".spel).branchEnd("end2", "join1") - ), - GraphBuilder - .join( - "join1", - "joinBranchExpression", - Some("input33"), - List( - "end1" -> List("value" -> "#input".spel), - "end2" -> List("value" -> "#input".spel) - ) - ) - .processorEnd("proc2", "logService", "all" -> "#input33.id".spel) - ) - - val recA = createTestRecord(id = "a") - val recB = createTestRecord(id = "b") - val recC = createTestRecord(id = "c") - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recA, recB, recC))) - - results.invocationResults("proc2").map(_.contextId) should contain only ( - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end1", - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0-end2", - s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end2" - ) - - results - .externalInvocationResults("proc2") - .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( - "b", - "a", - "c", - "c" - ).map(_ + "-collectedDuringServiceInvocation").map(variable) - } - - "should test multiple source scenario" in { - val process = ScenarioBuilder - .streaming(scenarioName) - .sources( - GraphBuilder - .source("source1", "input") - .filter("filter1", "#input.id != 'a'".spel) - .branchEnd("end1", "join"), - GraphBuilder - .source("source2", "input") - .filter("filter2", "#input.id != 'b'".spel) - .branchEnd("end2", "join"), - GraphBuilder - .join( - "join", - "joinBranchExpression", - Some("joinInput"), - List( - "end1" -> List("value" -> "#input".spel), - "end2" -> List("value" -> "#input".spel) - ) - ) - .processorEnd("proc2", "logService", "all" -> "#joinInput.id".spel) - ) - val scenarioTestData = ScenarioTestData( - List( - createTestRecord(sourceId = "source1", id = "a"), - createTestRecord(sourceId = "source2", id = "a"), - createTestRecord(sourceId = "source1", id = "d"), - createTestRecord(sourceId = "source2", id = "b"), - createTestRecord(sourceId = "source2", id = "c"), - ) - ) - val recordA = SimpleRecord("a", 1, "2", new Date(3), Some(4), 5, "6") - val recordB = recordA.copy(id = "b") - val recordC = recordA.copy(id = "c") - val recordD = recordA.copy(id = "d") - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, scenarioTestData) - - val nodeResults = results.nodeResults - nodeResults("source1") shouldBe List( - nodeResult(0, "source1", "input" -> recordA), - nodeResult(1, "source1", "input" -> recordD) - ) - nodeResults("source2") shouldBe List( - nodeResult(0, "source2", "input" -> recordA), - nodeResult(1, "source2", "input" -> recordB), - nodeResult(2, "source2", "input" -> recordC) - ) - nodeResults("filter1") shouldBe nodeResults("source1") - nodeResults("filter2") shouldBe nodeResults("source2") - nodeResults("$edge-end1-join") shouldBe List(nodeResult(1, "source1", "input" -> recordD)) - nodeResults("$edge-end2-join") shouldBe List( - nodeResult(0, "source2", "input" -> recordA), - nodeResult(2, "source2", "input" -> recordC) - ) - nodeResults("join") should contain only ( - nodeResult(1, "source1", "end1", "input" -> recordD, "joinInput" -> recordD), - nodeResult(0, "source2", "end2", "input" -> recordA, "joinInput" -> recordA), - nodeResult(2, "source2", "end2", "input" -> recordC, "joinInput" -> recordC) - ) - - results.invocationResults("proc2") should contain only ( - ExpressionInvocationResult(s"$scenarioName-source1-$firstSubtaskIndex-1-end1", "all", variable("d")), - ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-0-end2", "all", variable("a")), - ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-2-end2", "all", variable("c")) - ) - - results - .externalInvocationResults("proc2") - .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List("a", "c", "d") - .map(_ + "-collectedDuringServiceInvocation") - .map(variable) - } - - "should have correct run mode" in { - val process = ScenarioBuilder - .streaming(scenarioName) - .source("start", "input") - .enricher("componentUseCaseService", "componentUseCaseService", "returningComponentUseCaseService") - .customNode("componentUseCaseCustomNode", "componentUseCaseCustomNode", "transformerAddingComponentUseCase") - .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) - - val results = - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(sourceId = "start"))) - ) - - results.invocationResults("out").map(_.value) shouldBe List( - variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) - ) - } - - "should throw exception when parameter was modified by AdditionalUiConfigProvider with dict editor and flink wasn't provided with additional config" in { - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .processor( - "eager1", - "collectingEager", - "static" -> Expression.dictKeyWithLabel("'s'", Some("s")), - "dynamic" -> "#input.id".spel - ) - .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - - val dictEditorException = intercept[IllegalStateException] { - prepareTestRunner(useIOMonadInInterpreter).runTests( - process, - ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) - ) - } - dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" - } - - "should run correctly when parameter was modified by AdditionalUiConfigProvider with dict editor and flink was provided with additional config" in { - val modifiedComponentName = "collectingEager" - val modifiedParameterName = "static" - val process = - ScenarioBuilder - .streaming(scenarioName) - .source(sourceNodeId, "input") - .processor( - "eager1", - modifiedComponentName, - modifiedParameterName -> Expression.dictKeyWithLabel("'s'", Some("s")), - "dynamic" -> "#input.id".spel - ) - .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) - - val results = prepareTestRunner( - useIOMonadInInterpreter, - additionalConfigsFromProvider = Map( - DesignerWideComponentId("service-" + modifiedComponentName) -> ComponentAdditionalConfig( - parameterConfigs = Map( - ParameterName(modifiedParameterName) -> ParameterAdditionalUIConfig( - required = false, - initialValue = None, - hintText = None, - valueEditor = Some(ValueInputWithDictEditor("someDictId", allowOtherValue = false)), - valueCompileTimeValidation = None - ) - ) - ) - ) - ).runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) - results.exceptions should have length 0 - } - - "should not throw exception when process fragment has parameter validation defined" in { - val scenario = ScenarioBuilder - .streaming("scenario1") - .source(sourceNodeId, "input") - .fragmentOneOut("sub", fragmentWithValidationName, "output", "fragmentResult", "param" -> "'asd'".spel) - .emptySink("out", "valueMonitor", "Value" -> "1".spel) - - val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) - - val results = prepareTestRunner(useIOMonadInInterpreter).runTests( - resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, - ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), - ) - results.exceptions.length shouldBe 0 - } +// "be able to run tests multiple time on the same mini cluster" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) +// +// val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") +// +// val testRunner = prepareTestRunner(useIOMonadInInterpreter) +// +// def runTestAndVerify() = { +// val results = testRunner.runTests( +// process, +// ScenarioTestData( +// List( +// ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) +// ) +// ) +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input)) +// nodeResults("out") shouldBe List(nodeResult(0, "input" -> input)) +// +// val invocationResults = results.invocationResults +// +// invocationResults("out") shouldBe +// List(ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable(11))) +// +// results.externalInvocationResults("out") shouldBe List( +// ExternalInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "valueMonitor", variable(11)) +// ) +// } +// +// runTestAndVerify() +// runTestAndVerify() +// } +// +// "collect results for split" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), +// ) +// +// results.nodeResults("splitId1") shouldBe List( +// nodeResult( +// 0, +// "input" -> +// SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") +// ), +// nodeResult( +// 1, +// "input" -> +// SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") +// ) +// ) +// } +// +// "return correct result for custom node" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .customNode("cid", "out", "stateCustom", "groupBy" -> "#input.id".spel, "stringVal" -> "'s'".spel) +// .emptySink("out", "valueMonitor", "Value" -> "#input.value1 + ' ' + #out.previous".spel) +// +// val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") +// val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") +// +// val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") +// val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) +// nodeResults("cid") shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) +// nodeResults("out") shouldBe List( +// nodeResult(0, "input" -> input, "out" -> aggregate), +// nodeResult(1, "input" -> input2, "out" -> aggregate2) +// ) +// +// val invocationResults = results.invocationResults +// +// invocationResults("cid") shouldBe +// List( +// // we record only LazyParameter execution results +// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "groupBy", variable("0")), +// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "groupBy", variable("0")) +// ) +// +// invocationResults("out") shouldBe +// List( +// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable("1 0")), +// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "Value", variable("11 1")) +// ) +// +// results.externalInvocationResults("out") shouldBe +// List( +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", +// "valueMonitor", +// variable("1 0") +// ), +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", +// "valueMonitor", +// variable("11 1") +// ) +// ) +// } +// +// "handle large parallelism" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .parallelism(4) +// .source(sourceNodeId, "input") +// .emptySink("out", "monitor") +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults(sourceNodeId) should have length 5 +// } +// +// "detect errors" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) +// .filter("filter", "1 / #input.value1 >= 0".spel) +// .emptySink("out", "monitor") +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData( +// List( +// createTestRecord(id = "0", value1 = 1), +// createTestRecord(id = "1", value1 = 0), +// createTestRecord(id = "2", value1 = 2), +// createTestRecord(id = "3", value1 = 4) +// ) +// ), +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults(sourceNodeId) should have length 4 +// nodeResults("out") should have length 2 +// +// results.exceptions should have length 2 +// +// val exceptionFromExpression = results.exceptions.head +// exceptionFromExpression.nodeId shouldBe Some("filter") +// exceptionFromExpression.context +// .variables("input") +// .asInstanceOf[Json] +// .hcursor +// .downField("pretty") +// .focus +// .value +// .toString() +// .startsWith("SimpleJsonRecord(1") // it's not nice.. +// exceptionFromExpression.throwable.getMessage shouldBe "Expression [1 / #input.value1 >= 0] evaluation failed, message: / by zero" +// +// val exceptionFromService = results.exceptions.last +// exceptionFromService.nodeId shouldBe Some("failing") +// exceptionFromService.context +// .variables("input") +// .asInstanceOf[Json] +// .hcursor +// .downField("pretty") +// .focus +// .value +// .toString() +// .startsWith("SimpleJsonRecord(2") // it's not nice.. +// exceptionFromService.throwable.getMessage shouldBe "Thrown as expected" +// } +// +// "ignore real exception handler" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) +// .filter("filter", "1 / #input.value1 >= 0".spel) +// .emptySink("out", "monitor") +// +// val exceptionConsumerId = UUID.randomUUID().toString +// val results = prepareTestRunner( +// useIOMonadInInterpreter, +// enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) +// ).runTests( +// process, +// scenarioTestData = ScenarioTestData( +// List( +// createTestRecord(id = "0", value1 = 1), +// createTestRecord(id = "1", value1 = 0), +// createTestRecord(id = "2", value1 = 2), +// createTestRecord(id = "3", value1 = 4) +// ) +// ) +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults(sourceNodeId) should have length 4 +// nodeResults("out") should have length 2 +// +// results.exceptions should have length 2 +// RecordingExceptionConsumer.exceptionsFor(exceptionConsumerId) shouldBe Symbol("empty") +// } +// +// "handle transient errors" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .processor("failing", "throwingTransientService", "throw" -> "#input.value1 == 2".spel) +// .emptySink("out", "monitor") +// +// intercept[JobExecutionException] { +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) +// ) +// } +// } +// +// "handle json input" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "jsonInput") +// .emptySink("out", "valueMonitor", "Value" -> "#input".spel) +// val testData = ScenarioTestData( +// List( +// ScenarioTestJsonRecord( +// sourceNodeId, +// Json.obj(sourceNodeId -> Json.fromString("1"), "field" -> Json.fromString("11")) +// ), +// ScenarioTestJsonRecord( +// sourceNodeId, +// Json.obj(sourceNodeId -> Json.fromString("2"), "field" -> Json.fromString("22")) +// ), +// ScenarioTestJsonRecord( +// sourceNodeId, +// Json.obj(sourceNodeId -> Json.fromString("3"), "field" -> Json.fromString("33")) +// ), +// ) +// ) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) +// +// results.nodeResults(sourceNodeId) should have size 3 +// results.externalInvocationResults("out") shouldBe +// List( +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", +// "valueMonitor", +// variable(SimpleJsonRecord("1", "11")) +// ), +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", +// "valueMonitor", +// variable(SimpleJsonRecord("2", "22")) +// ), +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2", +// "valueMonitor", +// variable(SimpleJsonRecord("3", "33")) +// ) +// ) +// } +// +// "handle custom variables in source" in { +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "genericSourceWithCustomVariables", "elements" -> "{'abc'}".spel) +// .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) +// val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) +// +// results.nodeResults(sourceNodeId) should have size 1 +// results.externalInvocationResults("out") shouldBe +// List( +// ExternalInvocationResult( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", +// "valueMonitor", +// variable("transformed:abc|3") +// ) +// ) +// } +// +// "give meaningful error messages for sink errors" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) +// ) +// +// results.exceptions should have length 1 +// results.exceptions.head.nodeId shouldBe Some("out") +// results.exceptions.head.throwable.getMessage should include("message: / by zero") +// } +// +// "be able to test process with time windows" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .customNode("cid", "count", "transformWithTime", "seconds" -> "10".spel) +// .emptySink("out", "monitor") +// +// def recordWithSeconds(duration: FiniteDuration) = +// ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData( +// List( +// recordWithSeconds(1 second), +// recordWithSeconds(2 second), +// recordWithSeconds(5 second), +// recordWithSeconds(9 second), +// recordWithSeconds(20 second) +// ) +// ) +// ) +// +// val nodeResults = results.nodeResults +// +// nodeResults("out").map(_.variables) shouldBe List(Map("count" -> variable(4)), Map("count" -> variable(1))) +// +// } +// +// "be able to test typed map" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source( +// sourceNodeId, +// "typedJsonInput", +// "type" -> """{"field1": "String", "field2": "java.lang.String"}""".spel +// ) +// .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData( +// ScenarioTestJsonRecord( +// sourceNodeId, +// Json.obj("field1" -> Json.fromString("abc"), "field2" -> Json.fromString("def")) +// ) :: Nil +// ) +// ) +// +// results.invocationResults("out").map(_.value) shouldBe List(variable("abcdef")) +// } +// +// "using dependent services" in { +// val countToPass = 15 +// val valueToReturn = 18 +// +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .enricher( +// "dependent", +// "parsed", +// "returningDependentTypeService", +// "definition" -> "{'field1', 'field2'}".spel, +// "toFill" -> "#input.value1.toString()".spel, +// "count" -> countToPass.toString.spel +// ) +// .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(value1 = valueToReturn))) +// ) +// +// results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) +// } +// +// "switch value should be equal to variable value" in { +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .parallelism(1) +// .source(sourceNodeId, "input") +// .switch( +// "switch", +// "#input.id == 'ala'".spel, +// "output", +// Case( +// "#output == false".spel, +// GraphBuilder.emptySink("out", "valueMonitor", "Value" -> "'any'".spel) +// ) +// ) +// +// val recordTrue = createTestRecord(id = "ala") +// val recordFalse = createTestRecord(id = "bela") +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) +// +// val invocationResults = results.invocationResults +// +// invocationResults("switch").filter(_.name == "expression").head.value shouldBe variable(true) +// invocationResults("switch").filter(_.name == "expression").last.value shouldBe variable(false) +// // first record was filtered out +// invocationResults("out").head.contextId shouldBe s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1" +// } +// +// "should handle joins for one input (diamond-like) " in { +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .sources( +// GraphBuilder +// .source(sourceNodeId, "input") +// .split( +// "split", +// GraphBuilder.filter("left", "#input.id != 'a'".spel).branchEnd("end1", "join1"), +// GraphBuilder.filter("right", "#input.id != 'b'".spel).branchEnd("end2", "join1") +// ), +// GraphBuilder +// .join( +// "join1", +// "joinBranchExpression", +// Some("input33"), +// List( +// "end1" -> List("value" -> "#input".spel), +// "end2" -> List("value" -> "#input".spel) +// ) +// ) +// .processorEnd("proc2", "logService", "all" -> "#input33.id".spel) +// ) +// +// val recA = createTestRecord(id = "a") +// val recB = createTestRecord(id = "b") +// val recC = createTestRecord(id = "c") +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recA, recB, recC))) +// +// results.invocationResults("proc2").map(_.contextId) should contain only ( +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end1", +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0-end2", +// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end2" +// ) +// +// results +// .externalInvocationResults("proc2") +// .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( +// "b", +// "a", +// "c", +// "c" +// ).map(_ + "-collectedDuringServiceInvocation").map(variable) +// } +// +// "should test multiple source scenario" in { +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .sources( +// GraphBuilder +// .source("source1", "input") +// .filter("filter1", "#input.id != 'a'".spel) +// .branchEnd("end1", "join"), +// GraphBuilder +// .source("source2", "input") +// .filter("filter2", "#input.id != 'b'".spel) +// .branchEnd("end2", "join"), +// GraphBuilder +// .join( +// "join", +// "joinBranchExpression", +// Some("joinInput"), +// List( +// "end1" -> List("value" -> "#input".spel), +// "end2" -> List("value" -> "#input".spel) +// ) +// ) +// .processorEnd("proc2", "logService", "all" -> "#joinInput.id".spel) +// ) +// val scenarioTestData = ScenarioTestData( +// List( +// createTestRecord(sourceId = "source1", id = "a"), +// createTestRecord(sourceId = "source2", id = "a"), +// createTestRecord(sourceId = "source1", id = "d"), +// createTestRecord(sourceId = "source2", id = "b"), +// createTestRecord(sourceId = "source2", id = "c"), +// ) +// ) +// val recordA = SimpleRecord("a", 1, "2", new Date(3), Some(4), 5, "6") +// val recordB = recordA.copy(id = "b") +// val recordC = recordA.copy(id = "c") +// val recordD = recordA.copy(id = "d") +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, scenarioTestData) +// +// val nodeResults = results.nodeResults +// nodeResults("source1") shouldBe List( +// nodeResult(0, "source1", "input" -> recordA), +// nodeResult(1, "source1", "input" -> recordD) +// ) +// nodeResults("source2") shouldBe List( +// nodeResult(0, "source2", "input" -> recordA), +// nodeResult(1, "source2", "input" -> recordB), +// nodeResult(2, "source2", "input" -> recordC) +// ) +// nodeResults("filter1") shouldBe nodeResults("source1") +// nodeResults("filter2") shouldBe nodeResults("source2") +// nodeResults("$edge-end1-join") shouldBe List(nodeResult(1, "source1", "input" -> recordD)) +// nodeResults("$edge-end2-join") shouldBe List( +// nodeResult(0, "source2", "input" -> recordA), +// nodeResult(2, "source2", "input" -> recordC) +// ) +// nodeResults("join") should contain only ( +// nodeResult(1, "source1", "end1", "input" -> recordD, "joinInput" -> recordD), +// nodeResult(0, "source2", "end2", "input" -> recordA, "joinInput" -> recordA), +// nodeResult(2, "source2", "end2", "input" -> recordC, "joinInput" -> recordC) +// ) +// +// results.invocationResults("proc2") should contain only ( +// ExpressionInvocationResult(s"$scenarioName-source1-$firstSubtaskIndex-1-end1", "all", variable("d")), +// ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-0-end2", "all", variable("a")), +// ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-2-end2", "all", variable("c")) +// ) +// +// results +// .externalInvocationResults("proc2") +// .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List("a", "c", "d") +// .map(_ + "-collectedDuringServiceInvocation") +// .map(variable) +// } +// +// "should have correct run mode" in { +// val process = ScenarioBuilder +// .streaming(scenarioName) +// .source("start", "input") +// .enricher("componentUseCaseService", "componentUseCaseService", "returningComponentUseCaseService") +// .customNode("componentUseCaseCustomNode", "componentUseCaseCustomNode", "transformerAddingComponentUseCase") +// .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) +// +// val results = +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(sourceId = "start"))) +// ) +// +// results.invocationResults("out").map(_.value) shouldBe List( +// variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) +// ) +// } +// +// "should throw exception when parameter was modified by AdditionalUiConfigProvider with dict editor and flink wasn't provided with additional config" in { +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .processor( +// "eager1", +// "collectingEager", +// "static" -> Expression.dictKeyWithLabel("'s'", Some("s")), +// "dynamic" -> "#input.id".spel +// ) +// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) +// +// val dictEditorException = intercept[IllegalStateException] { +// prepareTestRunner(useIOMonadInInterpreter).runTests( +// process, +// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) +// ) +// } +// dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" +// } +// +// "should run correctly when parameter was modified by AdditionalUiConfigProvider with dict editor and flink was provided with additional config" in { +// val modifiedComponentName = "collectingEager" +// val modifiedParameterName = "static" +// val process = +// ScenarioBuilder +// .streaming(scenarioName) +// .source(sourceNodeId, "input") +// .processor( +// "eager1", +// modifiedComponentName, +// modifiedParameterName -> Expression.dictKeyWithLabel("'s'", Some("s")), +// "dynamic" -> "#input.id".spel +// ) +// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) +// +// val results = prepareTestRunner( +// useIOMonadInInterpreter, +// additionalConfigsFromProvider = Map( +// DesignerWideComponentId("service-" + modifiedComponentName) -> ComponentAdditionalConfig( +// parameterConfigs = Map( +// ParameterName(modifiedParameterName) -> ParameterAdditionalUIConfig( +// required = false, +// initialValue = None, +// hintText = None, +// valueEditor = Some(ValueInputWithDictEditor("someDictId", allowOtherValue = false)), +// valueCompileTimeValidation = None +// ) +// ) +// ) +// ) +// ).runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) +// results.exceptions should have length 0 +// } +// +// "should not throw exception when process fragment has parameter validation defined" in { +// val scenario = ScenarioBuilder +// .streaming("scenario1") +// .source(sourceNodeId, "input") +// .fragmentOneOut("sub", fragmentWithValidationName, "output", "fragmentResult", "param" -> "'asd'".spel) +// .emptySink("out", "valueMonitor", "Value" -> "1".spel) +// +// val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) +// +// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( +// resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, +// ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), +// ) +// results.exceptions.length shouldBe 0 +// } } private def createTestRecord( From 09c602ae2e33d92c120aa25a8050e92b46aabe82 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 11:17:19 +0100 Subject: [PATCH 33/48] Revert "fix attempt: passing stream connfiguration to jobgraph" This reverts commit 8cf5955bcfcec3ee195df1670c621298e3e4c005. --- .../ScenarioTestingMiniClusterWrapper.scala | 7 +- .../FlinkProcessTestRunnerSpec.scala | 1316 ++++++++--------- 2 files changed, 660 insertions(+), 663 deletions(-) diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala index 7e2b8a7d621..96baca04f7b 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -17,8 +17,7 @@ import scala.jdk.CollectionConverters._ final class ScenarioTestingMiniClusterWrapper( miniCluster: MiniCluster, val env: StreamExecutionEnvironment, - parallelism: Int, - streamExecutionConfig: Configuration + parallelism: Int ) extends AutoCloseable { def alignParallelism(canonicalProcess: CanonicalProcess): CanonicalProcess = { @@ -60,8 +59,6 @@ final class ScenarioTestingMiniClusterWrapper( ): Unit = { jobGraph.setClasspaths(modelClassLoader.urls.asJava) jobGraph.setSavepointRestoreSettings(savepointRestoreSettings) - // FIXME abr: Is it needed? - jobGraph.getJobConfiguration.addAll(streamExecutionConfig) } def close(): Unit = { @@ -83,7 +80,7 @@ object ScenarioTestingMiniClusterWrapper extends LazyLogging { parallelism, streamExecutionConfig ) - new ScenarioTestingMiniClusterWrapper(miniCluster, env, parallelism, streamExecutionConfig) + new ScenarioTestingMiniClusterWrapper(miniCluster, env, parallelism) } } diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala index 1c7f89a1c81..7f073eb8a87 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala @@ -95,9 +95,9 @@ class FlinkProcessTestRunnerSpec "IO monad interpreter is used" should { runTests(useIOMonadInInterpreter = true) } -// "IO monad interpreter is NOT used" should { -// runTests(useIOMonadInInterpreter = false) -// } + "IO monad interpreter is NOT used" should { + runTests(useIOMonadInInterpreter = false) + } } private def runTests(useIOMonadInInterpreter: Boolean): Unit = { @@ -165,661 +165,661 @@ class FlinkProcessTestRunnerSpec LogService.invocationsCount.get() shouldBe 0 } -// "be able to run tests multiple time on the same mini cluster" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) -// -// val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") -// -// val testRunner = prepareTestRunner(useIOMonadInInterpreter) -// -// def runTestAndVerify() = { -// val results = testRunner.runTests( -// process, -// ScenarioTestData( -// List( -// ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) -// ) -// ) -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input)) -// nodeResults("out") shouldBe List(nodeResult(0, "input" -> input)) -// -// val invocationResults = results.invocationResults -// -// invocationResults("out") shouldBe -// List(ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable(11))) -// -// results.externalInvocationResults("out") shouldBe List( -// ExternalInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "valueMonitor", variable(11)) -// ) -// } -// -// runTestAndVerify() -// runTestAndVerify() -// } -// -// "collect results for split" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), -// ) -// -// results.nodeResults("splitId1") shouldBe List( -// nodeResult( -// 0, -// "input" -> -// SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") -// ), -// nodeResult( -// 1, -// "input" -> -// SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") -// ) -// ) -// } -// -// "return correct result for custom node" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .customNode("cid", "out", "stateCustom", "groupBy" -> "#input.id".spel, "stringVal" -> "'s'".spel) -// .emptySink("out", "valueMonitor", "Value" -> "#input.value1 + ' ' + #out.previous".spel) -// -// val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") -// val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") -// -// val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") -// val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) -// nodeResults("cid") shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) -// nodeResults("out") shouldBe List( -// nodeResult(0, "input" -> input, "out" -> aggregate), -// nodeResult(1, "input" -> input2, "out" -> aggregate2) -// ) -// -// val invocationResults = results.invocationResults -// -// invocationResults("cid") shouldBe -// List( -// // we record only LazyParameter execution results -// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "groupBy", variable("0")), -// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "groupBy", variable("0")) -// ) -// -// invocationResults("out") shouldBe -// List( -// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable("1 0")), -// ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "Value", variable("11 1")) -// ) -// -// results.externalInvocationResults("out") shouldBe -// List( -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", -// "valueMonitor", -// variable("1 0") -// ), -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", -// "valueMonitor", -// variable("11 1") -// ) -// ) -// } -// -// "handle large parallelism" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .parallelism(4) -// .source(sourceNodeId, "input") -// .emptySink("out", "monitor") -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults(sourceNodeId) should have length 5 -// } -// -// "detect errors" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) -// .filter("filter", "1 / #input.value1 >= 0".spel) -// .emptySink("out", "monitor") -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData( -// List( -// createTestRecord(id = "0", value1 = 1), -// createTestRecord(id = "1", value1 = 0), -// createTestRecord(id = "2", value1 = 2), -// createTestRecord(id = "3", value1 = 4) -// ) -// ), -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults(sourceNodeId) should have length 4 -// nodeResults("out") should have length 2 -// -// results.exceptions should have length 2 -// -// val exceptionFromExpression = results.exceptions.head -// exceptionFromExpression.nodeId shouldBe Some("filter") -// exceptionFromExpression.context -// .variables("input") -// .asInstanceOf[Json] -// .hcursor -// .downField("pretty") -// .focus -// .value -// .toString() -// .startsWith("SimpleJsonRecord(1") // it's not nice.. -// exceptionFromExpression.throwable.getMessage shouldBe "Expression [1 / #input.value1 >= 0] evaluation failed, message: / by zero" -// -// val exceptionFromService = results.exceptions.last -// exceptionFromService.nodeId shouldBe Some("failing") -// exceptionFromService.context -// .variables("input") -// .asInstanceOf[Json] -// .hcursor -// .downField("pretty") -// .focus -// .value -// .toString() -// .startsWith("SimpleJsonRecord(2") // it's not nice.. -// exceptionFromService.throwable.getMessage shouldBe "Thrown as expected" -// } -// -// "ignore real exception handler" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) -// .filter("filter", "1 / #input.value1 >= 0".spel) -// .emptySink("out", "monitor") -// -// val exceptionConsumerId = UUID.randomUUID().toString -// val results = prepareTestRunner( -// useIOMonadInInterpreter, -// enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) -// ).runTests( -// process, -// scenarioTestData = ScenarioTestData( -// List( -// createTestRecord(id = "0", value1 = 1), -// createTestRecord(id = "1", value1 = 0), -// createTestRecord(id = "2", value1 = 2), -// createTestRecord(id = "3", value1 = 4) -// ) -// ) -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults(sourceNodeId) should have length 4 -// nodeResults("out") should have length 2 -// -// results.exceptions should have length 2 -// RecordingExceptionConsumer.exceptionsFor(exceptionConsumerId) shouldBe Symbol("empty") -// } -// -// "handle transient errors" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .processor("failing", "throwingTransientService", "throw" -> "#input.value1 == 2".spel) -// .emptySink("out", "monitor") -// -// intercept[JobExecutionException] { -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) -// ) -// } -// } -// -// "handle json input" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "jsonInput") -// .emptySink("out", "valueMonitor", "Value" -> "#input".spel) -// val testData = ScenarioTestData( -// List( -// ScenarioTestJsonRecord( -// sourceNodeId, -// Json.obj(sourceNodeId -> Json.fromString("1"), "field" -> Json.fromString("11")) -// ), -// ScenarioTestJsonRecord( -// sourceNodeId, -// Json.obj(sourceNodeId -> Json.fromString("2"), "field" -> Json.fromString("22")) -// ), -// ScenarioTestJsonRecord( -// sourceNodeId, -// Json.obj(sourceNodeId -> Json.fromString("3"), "field" -> Json.fromString("33")) -// ), -// ) -// ) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) -// -// results.nodeResults(sourceNodeId) should have size 3 -// results.externalInvocationResults("out") shouldBe -// List( -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", -// "valueMonitor", -// variable(SimpleJsonRecord("1", "11")) -// ), -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", -// "valueMonitor", -// variable(SimpleJsonRecord("2", "22")) -// ), -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2", -// "valueMonitor", -// variable(SimpleJsonRecord("3", "33")) -// ) -// ) -// } -// -// "handle custom variables in source" in { -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "genericSourceWithCustomVariables", "elements" -> "{'abc'}".spel) -// .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) -// val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) -// -// results.nodeResults(sourceNodeId) should have size 1 -// results.externalInvocationResults("out") shouldBe -// List( -// ExternalInvocationResult( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", -// "valueMonitor", -// variable("transformed:abc|3") -// ) -// ) -// } -// -// "give meaningful error messages for sink errors" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) -// ) -// -// results.exceptions should have length 1 -// results.exceptions.head.nodeId shouldBe Some("out") -// results.exceptions.head.throwable.getMessage should include("message: / by zero") -// } -// -// "be able to test process with time windows" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .customNode("cid", "count", "transformWithTime", "seconds" -> "10".spel) -// .emptySink("out", "monitor") -// -// def recordWithSeconds(duration: FiniteDuration) = -// ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData( -// List( -// recordWithSeconds(1 second), -// recordWithSeconds(2 second), -// recordWithSeconds(5 second), -// recordWithSeconds(9 second), -// recordWithSeconds(20 second) -// ) -// ) -// ) -// -// val nodeResults = results.nodeResults -// -// nodeResults("out").map(_.variables) shouldBe List(Map("count" -> variable(4)), Map("count" -> variable(1))) -// -// } -// -// "be able to test typed map" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source( -// sourceNodeId, -// "typedJsonInput", -// "type" -> """{"field1": "String", "field2": "java.lang.String"}""".spel -// ) -// .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData( -// ScenarioTestJsonRecord( -// sourceNodeId, -// Json.obj("field1" -> Json.fromString("abc"), "field2" -> Json.fromString("def")) -// ) :: Nil -// ) -// ) -// -// results.invocationResults("out").map(_.value) shouldBe List(variable("abcdef")) -// } -// -// "using dependent services" in { -// val countToPass = 15 -// val valueToReturn = 18 -// -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .enricher( -// "dependent", -// "parsed", -// "returningDependentTypeService", -// "definition" -> "{'field1', 'field2'}".spel, -// "toFill" -> "#input.value1.toString()".spel, -// "count" -> countToPass.toString.spel -// ) -// .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(value1 = valueToReturn))) -// ) -// -// results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) -// } -// -// "switch value should be equal to variable value" in { -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .parallelism(1) -// .source(sourceNodeId, "input") -// .switch( -// "switch", -// "#input.id == 'ala'".spel, -// "output", -// Case( -// "#output == false".spel, -// GraphBuilder.emptySink("out", "valueMonitor", "Value" -> "'any'".spel) -// ) -// ) -// -// val recordTrue = createTestRecord(id = "ala") -// val recordFalse = createTestRecord(id = "bela") -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) -// -// val invocationResults = results.invocationResults -// -// invocationResults("switch").filter(_.name == "expression").head.value shouldBe variable(true) -// invocationResults("switch").filter(_.name == "expression").last.value shouldBe variable(false) -// // first record was filtered out -// invocationResults("out").head.contextId shouldBe s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1" -// } -// -// "should handle joins for one input (diamond-like) " in { -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .sources( -// GraphBuilder -// .source(sourceNodeId, "input") -// .split( -// "split", -// GraphBuilder.filter("left", "#input.id != 'a'".spel).branchEnd("end1", "join1"), -// GraphBuilder.filter("right", "#input.id != 'b'".spel).branchEnd("end2", "join1") -// ), -// GraphBuilder -// .join( -// "join1", -// "joinBranchExpression", -// Some("input33"), -// List( -// "end1" -> List("value" -> "#input".spel), -// "end2" -> List("value" -> "#input".spel) -// ) -// ) -// .processorEnd("proc2", "logService", "all" -> "#input33.id".spel) -// ) -// -// val recA = createTestRecord(id = "a") -// val recB = createTestRecord(id = "b") -// val recC = createTestRecord(id = "c") -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recA, recB, recC))) -// -// results.invocationResults("proc2").map(_.contextId) should contain only ( -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end1", -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0-end2", -// s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end2" -// ) -// -// results -// .externalInvocationResults("proc2") -// .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( -// "b", -// "a", -// "c", -// "c" -// ).map(_ + "-collectedDuringServiceInvocation").map(variable) -// } -// -// "should test multiple source scenario" in { -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .sources( -// GraphBuilder -// .source("source1", "input") -// .filter("filter1", "#input.id != 'a'".spel) -// .branchEnd("end1", "join"), -// GraphBuilder -// .source("source2", "input") -// .filter("filter2", "#input.id != 'b'".spel) -// .branchEnd("end2", "join"), -// GraphBuilder -// .join( -// "join", -// "joinBranchExpression", -// Some("joinInput"), -// List( -// "end1" -> List("value" -> "#input".spel), -// "end2" -> List("value" -> "#input".spel) -// ) -// ) -// .processorEnd("proc2", "logService", "all" -> "#joinInput.id".spel) -// ) -// val scenarioTestData = ScenarioTestData( -// List( -// createTestRecord(sourceId = "source1", id = "a"), -// createTestRecord(sourceId = "source2", id = "a"), -// createTestRecord(sourceId = "source1", id = "d"), -// createTestRecord(sourceId = "source2", id = "b"), -// createTestRecord(sourceId = "source2", id = "c"), -// ) -// ) -// val recordA = SimpleRecord("a", 1, "2", new Date(3), Some(4), 5, "6") -// val recordB = recordA.copy(id = "b") -// val recordC = recordA.copy(id = "c") -// val recordD = recordA.copy(id = "d") -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, scenarioTestData) -// -// val nodeResults = results.nodeResults -// nodeResults("source1") shouldBe List( -// nodeResult(0, "source1", "input" -> recordA), -// nodeResult(1, "source1", "input" -> recordD) -// ) -// nodeResults("source2") shouldBe List( -// nodeResult(0, "source2", "input" -> recordA), -// nodeResult(1, "source2", "input" -> recordB), -// nodeResult(2, "source2", "input" -> recordC) -// ) -// nodeResults("filter1") shouldBe nodeResults("source1") -// nodeResults("filter2") shouldBe nodeResults("source2") -// nodeResults("$edge-end1-join") shouldBe List(nodeResult(1, "source1", "input" -> recordD)) -// nodeResults("$edge-end2-join") shouldBe List( -// nodeResult(0, "source2", "input" -> recordA), -// nodeResult(2, "source2", "input" -> recordC) -// ) -// nodeResults("join") should contain only ( -// nodeResult(1, "source1", "end1", "input" -> recordD, "joinInput" -> recordD), -// nodeResult(0, "source2", "end2", "input" -> recordA, "joinInput" -> recordA), -// nodeResult(2, "source2", "end2", "input" -> recordC, "joinInput" -> recordC) -// ) -// -// results.invocationResults("proc2") should contain only ( -// ExpressionInvocationResult(s"$scenarioName-source1-$firstSubtaskIndex-1-end1", "all", variable("d")), -// ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-0-end2", "all", variable("a")), -// ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-2-end2", "all", variable("c")) -// ) -// -// results -// .externalInvocationResults("proc2") -// .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List("a", "c", "d") -// .map(_ + "-collectedDuringServiceInvocation") -// .map(variable) -// } -// -// "should have correct run mode" in { -// val process = ScenarioBuilder -// .streaming(scenarioName) -// .source("start", "input") -// .enricher("componentUseCaseService", "componentUseCaseService", "returningComponentUseCaseService") -// .customNode("componentUseCaseCustomNode", "componentUseCaseCustomNode", "transformerAddingComponentUseCase") -// .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) -// -// val results = -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(sourceId = "start"))) -// ) -// -// results.invocationResults("out").map(_.value) shouldBe List( -// variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) -// ) -// } -// -// "should throw exception when parameter was modified by AdditionalUiConfigProvider with dict editor and flink wasn't provided with additional config" in { -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .processor( -// "eager1", -// "collectingEager", -// "static" -> Expression.dictKeyWithLabel("'s'", Some("s")), -// "dynamic" -> "#input.id".spel -// ) -// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) -// -// val dictEditorException = intercept[IllegalStateException] { -// prepareTestRunner(useIOMonadInInterpreter).runTests( -// process, -// ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) -// ) -// } -// dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" -// } -// -// "should run correctly when parameter was modified by AdditionalUiConfigProvider with dict editor and flink was provided with additional config" in { -// val modifiedComponentName = "collectingEager" -// val modifiedParameterName = "static" -// val process = -// ScenarioBuilder -// .streaming(scenarioName) -// .source(sourceNodeId, "input") -// .processor( -// "eager1", -// modifiedComponentName, -// modifiedParameterName -> Expression.dictKeyWithLabel("'s'", Some("s")), -// "dynamic" -> "#input.id".spel -// ) -// .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) -// -// val results = prepareTestRunner( -// useIOMonadInInterpreter, -// additionalConfigsFromProvider = Map( -// DesignerWideComponentId("service-" + modifiedComponentName) -> ComponentAdditionalConfig( -// parameterConfigs = Map( -// ParameterName(modifiedParameterName) -> ParameterAdditionalUIConfig( -// required = false, -// initialValue = None, -// hintText = None, -// valueEditor = Some(ValueInputWithDictEditor("someDictId", allowOtherValue = false)), -// valueCompileTimeValidation = None -// ) -// ) -// ) -// ) -// ).runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) -// results.exceptions should have length 0 -// } -// -// "should not throw exception when process fragment has parameter validation defined" in { -// val scenario = ScenarioBuilder -// .streaming("scenario1") -// .source(sourceNodeId, "input") -// .fragmentOneOut("sub", fragmentWithValidationName, "output", "fragmentResult", "param" -> "'asd'".spel) -// .emptySink("out", "valueMonitor", "Value" -> "1".spel) -// -// val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) -// -// val results = prepareTestRunner(useIOMonadInInterpreter).runTests( -// resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, -// ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), -// ) -// results.exceptions.length shouldBe 0 -// } + "be able to run tests multiple time on the same mini cluster" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) + + val input = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") + + val testRunner = prepareTestRunner(useIOMonadInInterpreter) + + def runTestAndVerify() = { + val results = testRunner.runTests( + process, + ScenarioTestData( + List( + ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|11|2|3|4|5|6")) + ) + ) + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input)) + nodeResults("out") shouldBe List(nodeResult(0, "input" -> input)) + + val invocationResults = results.invocationResults + + invocationResults("out") shouldBe + List(ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable(11))) + + results.externalInvocationResults("out") shouldBe List( + ExternalInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "valueMonitor", variable(11)) + ) + } + + runTestAndVerify() + runTestAndVerify() + } + + "collect results for split" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .split("splitId1", GraphBuilder.emptySink("out1", "monitor"), GraphBuilder.emptySink("out2", "monitor")) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), + ) + + results.nodeResults("splitId1") shouldBe List( + nodeResult( + 0, + "input" -> + SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") + ), + nodeResult( + 1, + "input" -> + SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") + ) + ) + } + + "return correct result for custom node" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .customNode("cid", "out", "stateCustom", "groupBy" -> "#input.id".spel, "stringVal" -> "'s'".spel) + .emptySink("out", "valueMonitor", "Value" -> "#input.value1 + ' ' + #out.previous".spel) + + val input = SimpleRecord("0", 1, "2", new Date(3), Some(4), 5, "6") + val input2 = SimpleRecord("0", 11, "2", new Date(3), Some(4), 5, "6") + + val aggregate = SimpleRecordWithPreviousValue(input, 0, "s") + val aggregate2 = SimpleRecordWithPreviousValue(input2, 1, "s") + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(), createTestRecord(value1 = 11))), + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) + nodeResults("cid") shouldBe List(nodeResult(0, "input" -> input), nodeResult(1, "input" -> input2)) + nodeResults("out") shouldBe List( + nodeResult(0, "input" -> input, "out" -> aggregate), + nodeResult(1, "input" -> input2, "out" -> aggregate2) + ) + + val invocationResults = results.invocationResults + + invocationResults("cid") shouldBe + List( + // we record only LazyParameter execution results + ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "groupBy", variable("0")), + ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "groupBy", variable("0")) + ) + + invocationResults("out") shouldBe + List( + ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", "Value", variable("1 0")), + ExpressionInvocationResult(s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", "Value", variable("11 1")) + ) + + results.externalInvocationResults("out") shouldBe + List( + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", + "valueMonitor", + variable("1 0") + ), + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", + "valueMonitor", + variable("11 1") + ) + ) + } + + "handle large parallelism" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .parallelism(4) + .source(sourceNodeId, "input") + .emptySink("out", "monitor") + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(createTestRecord() :: List.fill(4)(createTestRecord(value1 = 11))), + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) should have length 5 + } + + "detect errors" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) + .filter("filter", "1 / #input.value1 >= 0".spel) + .emptySink("out", "monitor") + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData( + List( + createTestRecord(id = "0", value1 = 1), + createTestRecord(id = "1", value1 = 0), + createTestRecord(id = "2", value1 = 2), + createTestRecord(id = "3", value1 = 4) + ) + ), + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) should have length 4 + nodeResults("out") should have length 2 + + results.exceptions should have length 2 + + val exceptionFromExpression = results.exceptions.head + exceptionFromExpression.nodeId shouldBe Some("filter") + exceptionFromExpression.context + .variables("input") + .asInstanceOf[Json] + .hcursor + .downField("pretty") + .focus + .value + .toString() + .startsWith("SimpleJsonRecord(1") // it's not nice.. + exceptionFromExpression.throwable.getMessage shouldBe "Expression [1 / #input.value1 >= 0] evaluation failed, message: / by zero" + + val exceptionFromService = results.exceptions.last + exceptionFromService.nodeId shouldBe Some("failing") + exceptionFromService.context + .variables("input") + .asInstanceOf[Json] + .hcursor + .downField("pretty") + .focus + .value + .toString() + .startsWith("SimpleJsonRecord(2") // it's not nice.. + exceptionFromService.throwable.getMessage shouldBe "Thrown as expected" + } + + "ignore real exception handler" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .processor("failing", "throwingService", "throw" -> "#input.value1 == 2".spel) + .filter("filter", "1 / #input.value1 >= 0".spel) + .emptySink("out", "monitor") + + val exceptionConsumerId = UUID.randomUUID().toString + val results = prepareTestRunner( + useIOMonadInInterpreter, + enrichDefaultConfig = RecordingExceptionConsumerProvider.configWithProvider(_, exceptionConsumerId) + ).runTests( + process, + scenarioTestData = ScenarioTestData( + List( + createTestRecord(id = "0", value1 = 1), + createTestRecord(id = "1", value1 = 0), + createTestRecord(id = "2", value1 = 2), + createTestRecord(id = "3", value1 = 4) + ) + ) + ) + + val nodeResults = results.nodeResults + + nodeResults(sourceNodeId) should have length 4 + nodeResults("out") should have length 2 + + results.exceptions should have length 2 + RecordingExceptionConsumer.exceptionsFor(exceptionConsumerId) shouldBe Symbol("empty") + } + + "handle transient errors" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .processor("failing", "throwingTransientService", "throw" -> "#input.value1 == 2".spel) + .emptySink("out", "monitor") + + intercept[JobExecutionException] { + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) + } + } + + "handle json input" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "jsonInput") + .emptySink("out", "valueMonitor", "Value" -> "#input".spel) + val testData = ScenarioTestData( + List( + ScenarioTestJsonRecord( + sourceNodeId, + Json.obj(sourceNodeId -> Json.fromString("1"), "field" -> Json.fromString("11")) + ), + ScenarioTestJsonRecord( + sourceNodeId, + Json.obj(sourceNodeId -> Json.fromString("2"), "field" -> Json.fromString("22")) + ), + ScenarioTestJsonRecord( + sourceNodeId, + Json.obj(sourceNodeId -> Json.fromString("3"), "field" -> Json.fromString("33")) + ), + ) + ) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) + + results.nodeResults(sourceNodeId) should have size 3 + results.externalInvocationResults("out") shouldBe + List( + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", + "valueMonitor", + variable(SimpleJsonRecord("1", "11")) + ), + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1", + "valueMonitor", + variable(SimpleJsonRecord("2", "22")) + ), + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2", + "valueMonitor", + variable(SimpleJsonRecord("3", "33")) + ) + ) + } + + "handle custom variables in source" in { + val process = ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "genericSourceWithCustomVariables", "elements" -> "{'abc'}".spel) + .emptySink("out", "valueMonitor", "Value" -> "#additionalOne + '|' + #additionalTwo".spel) + val testData = ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("abc")))) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, testData) + + results.nodeResults(sourceNodeId) should have size 1 + results.externalInvocationResults("out") shouldBe + List( + ExternalInvocationResult( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0", + "valueMonitor", + variable("transformed:abc|3") + ) + ) + } + + "give meaningful error messages for sink errors" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .emptySink("out", "sinkForInts", "Value" -> "15 / {0, 1}[0]".spel) + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) + + results.exceptions should have length 1 + results.exceptions.head.nodeId shouldBe Some("out") + results.exceptions.head.throwable.getMessage should include("message: / by zero") + } + + "be able to test process with time windows" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .customNode("cid", "count", "transformWithTime", "seconds" -> "10".spel) + .emptySink("out", "monitor") + + def recordWithSeconds(duration: FiniteDuration) = + ScenarioTestJsonRecord(sourceNodeId, Json.fromString(s"0|0|0|${duration.toMillis}|0|0|0")) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData( + List( + recordWithSeconds(1 second), + recordWithSeconds(2 second), + recordWithSeconds(5 second), + recordWithSeconds(9 second), + recordWithSeconds(20 second) + ) + ) + ) + + val nodeResults = results.nodeResults + + nodeResults("out").map(_.variables) shouldBe List(Map("count" -> variable(4)), Map("count" -> variable(1))) + + } + + "be able to test typed map" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source( + sourceNodeId, + "typedJsonInput", + "type" -> """{"field1": "String", "field2": "java.lang.String"}""".spel + ) + .emptySink("out", "valueMonitor", "Value" -> "#input.field1 + #input.field2".spel) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData( + ScenarioTestJsonRecord( + sourceNodeId, + Json.obj("field1" -> Json.fromString("abc"), "field2" -> Json.fromString("def")) + ) :: Nil + ) + ) + + results.invocationResults("out").map(_.value) shouldBe List(variable("abcdef")) + } + + "using dependent services" in { + val countToPass = 15 + val valueToReturn = 18 + + val process = ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .enricher( + "dependent", + "parsed", + "returningDependentTypeService", + "definition" -> "{'field1', 'field2'}".spel, + "toFill" -> "#input.value1.toString()".spel, + "count" -> countToPass.toString.spel + ) + .emptySink("out", "valueMonitor", "Value" -> "#parsed.size + ' ' + #parsed[0].field2".spel) + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(value1 = valueToReturn))) + ) + + results.invocationResults("out").map(_.value) shouldBe List(variable(s"$countToPass $valueToReturn")) + } + + "switch value should be equal to variable value" in { + val process = ScenarioBuilder + .streaming(scenarioName) + .parallelism(1) + .source(sourceNodeId, "input") + .switch( + "switch", + "#input.id == 'ala'".spel, + "output", + Case( + "#output == false".spel, + GraphBuilder.emptySink("out", "valueMonitor", "Value" -> "'any'".spel) + ) + ) + + val recordTrue = createTestRecord(id = "ala") + val recordFalse = createTestRecord(id = "bela") + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recordTrue, recordFalse))) + + val invocationResults = results.invocationResults + + invocationResults("switch").filter(_.name == "expression").head.value shouldBe variable(true) + invocationResults("switch").filter(_.name == "expression").last.value shouldBe variable(false) + // first record was filtered out + invocationResults("out").head.contextId shouldBe s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1" + } + + "should handle joins for one input (diamond-like) " in { + val process = ScenarioBuilder + .streaming(scenarioName) + .sources( + GraphBuilder + .source(sourceNodeId, "input") + .split( + "split", + GraphBuilder.filter("left", "#input.id != 'a'".spel).branchEnd("end1", "join1"), + GraphBuilder.filter("right", "#input.id != 'b'".spel).branchEnd("end2", "join1") + ), + GraphBuilder + .join( + "join1", + "joinBranchExpression", + Some("input33"), + List( + "end1" -> List("value" -> "#input".spel), + "end2" -> List("value" -> "#input".spel) + ) + ) + .processorEnd("proc2", "logService", "all" -> "#input33.id".spel) + ) + + val recA = createTestRecord(id = "a") + val recB = createTestRecord(id = "b") + val recC = createTestRecord(id = "c") + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests(process, ScenarioTestData(List(recA, recB, recC))) + + results.invocationResults("proc2").map(_.contextId) should contain only ( + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-1-end1", + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end1", + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-0-end2", + s"$scenarioName-$sourceNodeId-$firstSubtaskIndex-2-end2" + ) + + results + .externalInvocationResults("proc2") + .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List( + "b", + "a", + "c", + "c" + ).map(_ + "-collectedDuringServiceInvocation").map(variable) + } + + "should test multiple source scenario" in { + val process = ScenarioBuilder + .streaming(scenarioName) + .sources( + GraphBuilder + .source("source1", "input") + .filter("filter1", "#input.id != 'a'".spel) + .branchEnd("end1", "join"), + GraphBuilder + .source("source2", "input") + .filter("filter2", "#input.id != 'b'".spel) + .branchEnd("end2", "join"), + GraphBuilder + .join( + "join", + "joinBranchExpression", + Some("joinInput"), + List( + "end1" -> List("value" -> "#input".spel), + "end2" -> List("value" -> "#input".spel) + ) + ) + .processorEnd("proc2", "logService", "all" -> "#joinInput.id".spel) + ) + val scenarioTestData = ScenarioTestData( + List( + createTestRecord(sourceId = "source1", id = "a"), + createTestRecord(sourceId = "source2", id = "a"), + createTestRecord(sourceId = "source1", id = "d"), + createTestRecord(sourceId = "source2", id = "b"), + createTestRecord(sourceId = "source2", id = "c"), + ) + ) + val recordA = SimpleRecord("a", 1, "2", new Date(3), Some(4), 5, "6") + val recordB = recordA.copy(id = "b") + val recordC = recordA.copy(id = "c") + val recordD = recordA.copy(id = "d") + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests(process, scenarioTestData) + + val nodeResults = results.nodeResults + nodeResults("source1") shouldBe List( + nodeResult(0, "source1", "input" -> recordA), + nodeResult(1, "source1", "input" -> recordD) + ) + nodeResults("source2") shouldBe List( + nodeResult(0, "source2", "input" -> recordA), + nodeResult(1, "source2", "input" -> recordB), + nodeResult(2, "source2", "input" -> recordC) + ) + nodeResults("filter1") shouldBe nodeResults("source1") + nodeResults("filter2") shouldBe nodeResults("source2") + nodeResults("$edge-end1-join") shouldBe List(nodeResult(1, "source1", "input" -> recordD)) + nodeResults("$edge-end2-join") shouldBe List( + nodeResult(0, "source2", "input" -> recordA), + nodeResult(2, "source2", "input" -> recordC) + ) + nodeResults("join") should contain only ( + nodeResult(1, "source1", "end1", "input" -> recordD, "joinInput" -> recordD), + nodeResult(0, "source2", "end2", "input" -> recordA, "joinInput" -> recordA), + nodeResult(2, "source2", "end2", "input" -> recordC, "joinInput" -> recordC) + ) + + results.invocationResults("proc2") should contain only ( + ExpressionInvocationResult(s"$scenarioName-source1-$firstSubtaskIndex-1-end1", "all", variable("d")), + ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-0-end2", "all", variable("a")), + ExpressionInvocationResult(s"$scenarioName-source2-$firstSubtaskIndex-2-end2", "all", variable("c")) + ) + + results + .externalInvocationResults("proc2") + .map(_.value.asInstanceOf[Json]) should contain theSameElementsAs List("a", "c", "d") + .map(_ + "-collectedDuringServiceInvocation") + .map(variable) + } + + "should have correct run mode" in { + val process = ScenarioBuilder + .streaming(scenarioName) + .source("start", "input") + .enricher("componentUseCaseService", "componentUseCaseService", "returningComponentUseCaseService") + .customNode("componentUseCaseCustomNode", "componentUseCaseCustomNode", "transformerAddingComponentUseCase") + .emptySink("out", "valueMonitor", "Value" -> "{#componentUseCaseService, #componentUseCaseCustomNode}".spel) + + val results = + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(sourceId = "start"))) + ) + + results.invocationResults("out").map(_.value) shouldBe List( + variable(List(ComponentUseCase.TestRuntime, ComponentUseCase.TestRuntime)) + ) + } + + "should throw exception when parameter was modified by AdditionalUiConfigProvider with dict editor and flink wasn't provided with additional config" in { + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .processor( + "eager1", + "collectingEager", + "static" -> Expression.dictKeyWithLabel("'s'", Some("s")), + "dynamic" -> "#input.id".spel + ) + .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) + + val dictEditorException = intercept[IllegalStateException] { + prepareTestRunner(useIOMonadInInterpreter).runTests( + process, + ScenarioTestData(List(createTestRecord(id = "2", value1 = 2))) + ) + } + dictEditorException.getMessage shouldBe "DictKeyWithLabel expression can only be used with DictParameterEditor, got Some(DualParameterEditor(StringParameterEditor,RAW))" + } + + "should run correctly when parameter was modified by AdditionalUiConfigProvider with dict editor and flink was provided with additional config" in { + val modifiedComponentName = "collectingEager" + val modifiedParameterName = "static" + val process = + ScenarioBuilder + .streaming(scenarioName) + .source(sourceNodeId, "input") + .processor( + "eager1", + modifiedComponentName, + modifiedParameterName -> Expression.dictKeyWithLabel("'s'", Some("s")), + "dynamic" -> "#input.id".spel + ) + .emptySink("out", "valueMonitor", "Value" -> "#input.value1".spel) + + val results = prepareTestRunner( + useIOMonadInInterpreter, + additionalConfigsFromProvider = Map( + DesignerWideComponentId("service-" + modifiedComponentName) -> ComponentAdditionalConfig( + parameterConfigs = Map( + ParameterName(modifiedParameterName) -> ParameterAdditionalUIConfig( + required = false, + initialValue = None, + hintText = None, + valueEditor = Some(ValueInputWithDictEditor("someDictId", allowOtherValue = false)), + valueCompileTimeValidation = None + ) + ) + ) + ) + ).runTests(process, ScenarioTestData(List(createTestRecord(id = "2", value1 = 2)))) + results.exceptions should have length 0 + } + + "should not throw exception when process fragment has parameter validation defined" in { + val scenario = ScenarioBuilder + .streaming("scenario1") + .source(sourceNodeId, "input") + .fragmentOneOut("sub", fragmentWithValidationName, "output", "fragmentResult", "param" -> "'asd'".spel) + .emptySink("out", "valueMonitor", "Value" -> "1".spel) + + val resolved = FragmentResolver(List(processWithFragmentParameterValidation)).resolve(scenario) + + val results = prepareTestRunner(useIOMonadInInterpreter).runTests( + resolved.valueOr { _ => throw new IllegalArgumentException("Won't happen") }, + ScenarioTestData(List(ScenarioTestJsonRecord(sourceNodeId, Json.fromString("0|1|2|3|4|5|6")))), + ) + results.exceptions.length shouldBe 0 + } } private def createTestRecord( From efae190061b8becf35bc2731751b2028f7bd47f7 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 11:21:43 +0100 Subject: [PATCH 34/48] fix attempt: removed Xmx override --- e2e-tests/src/test/resources/debuggable-nu-designer.override.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/e2e-tests/src/test/resources/debuggable-nu-designer.override.yml b/e2e-tests/src/test/resources/debuggable-nu-designer.override.yml index c478a56d6c0..205a1e6c611 100644 --- a/e2e-tests/src/test/resources/debuggable-nu-designer.override.yml +++ b/e2e-tests/src/test/resources/debuggable-nu-designer.override.yml @@ -5,5 +5,4 @@ services: - "5005:5005" environment: JAVA_TOOL_OPTIONS: -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=*:5005 - JDK_JAVA_OPTIONS: "-Xmx2048M" NUSSKNACKER_LOG_LEVEL: debug From 1d63b5d9c0180e7b9c52a65b87a3e6b0f74720bf Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 11:44:00 +0100 Subject: [PATCH 35/48] fix attempt: memory properties fix --- .../batch-data-generation/batch-customizations.conf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf index bd0cec367d2..96ccc0f41e9 100644 --- a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf +++ b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf @@ -12,9 +12,9 @@ baseBatchConfig { restUrl: ${?FLINK_REST_URL} scenarioTesting { streamExecutionConfig { - taskmanager.memory.network.min: 16m - taskmanager.memory.network.max: 16m - taskmanager.memory.managed.size: 50m + "taskmanager.memory.network.min": 16m + "taskmanager.memory.network.max": 16m + "taskmanager.memory.managed.size": 50m } } } From 848fa3934fa33e53e117b219185e68eb1f6e7100 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 11:48:56 +0100 Subject: [PATCH 36/48] removed memory tweaks --- .../batch-data-generation/batch-customizations.conf | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf index 96ccc0f41e9..d7926e125a8 100644 --- a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf +++ b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf @@ -11,11 +11,11 @@ baseBatchConfig { type: "flinkStreaming" restUrl: ${?FLINK_REST_URL} scenarioTesting { - streamExecutionConfig { - "taskmanager.memory.network.min": 16m - "taskmanager.memory.network.max": 16m - "taskmanager.memory.managed.size": 50m - } +// streamExecutionConfig { +// "taskmanager.memory.network.min": 64m +// "taskmanager.memory.network.max": 64m +// "taskmanager.memory.managed.size": 50m +// } } } modelConfig: { From 0d2f5c8232e7e206e6b5a65ea7ff293defe0659a Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 12:15:45 +0100 Subject: [PATCH 37/48] in buffers to 200MB --- .../batch-data-generation/batch-customizations.conf | 10 +++++----- examples/installation/docker-compose.yml | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf index d7926e125a8..ad14fc416c2 100644 --- a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf +++ b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf @@ -11,11 +11,11 @@ baseBatchConfig { type: "flinkStreaming" restUrl: ${?FLINK_REST_URL} scenarioTesting { -// streamExecutionConfig { -// "taskmanager.memory.network.min": 64m -// "taskmanager.memory.network.max": 64m -// "taskmanager.memory.managed.size": 50m -// } + streamExecutionConfig { + "taskmanager.memory.network.min": 64m + "taskmanager.memory.network.max": 64m + "taskmanager.memory.managed.size": 50m + } } } modelConfig: { diff --git a/examples/installation/docker-compose.yml b/examples/installation/docker-compose.yml index 3b9302c4709..fa009c0be50 100644 --- a/examples/installation/docker-compose.yml +++ b/examples/installation/docker-compose.yml @@ -41,7 +41,7 @@ services: SCHEMA_REGISTRY_URL: "http://schema-registry:8081" INFLUXDB_URL: "http://influxdb:8086" FLINK_REST_URL: "http://flink-jobmanager:8081" - JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=300M -XX:MaxDirectMemorySize=100M" + JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=300M -XX:MaxDirectMemorySize=200M" USAGE_REPORTS_SOURCE: "example-installation-docker-compose" depends_on: postgres: From 57f86588f21e0b49204b6f50fe2863889c11dd33 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 13:13:53 +0100 Subject: [PATCH 38/48] propert batch customization --- .../batch-data-generation/batch-customizations.conf | 7 ------- .../src/test/resources/batch-nu-designer.override.yml | 5 +++++ examples/installation/docker-compose.yml | 4 ++-- utils/test-utils/src/main/resources/logback-test.xml | 6 +++++- .../DockerBasedNuInstallationExampleEnvironment.scala | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf index ad14fc416c2..8fa5e63979c 100644 --- a/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf +++ b/e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf @@ -10,13 +10,6 @@ baseBatchConfig { deploymentConfig: { type: "flinkStreaming" restUrl: ${?FLINK_REST_URL} - scenarioTesting { - streamExecutionConfig { - "taskmanager.memory.network.min": 64m - "taskmanager.memory.network.max": 64m - "taskmanager.memory.managed.size": 50m - } - } } modelConfig: { classPath: ["model/defaultModel.jar", "components/flink/flinkBase.jar", "components/flink-table/flinkTable.jar", "model/flinkExecutor.jar", "flink-dropwizard-metrics-deps/"] diff --git a/e2e-tests/src/test/resources/batch-nu-designer.override.yml b/e2e-tests/src/test/resources/batch-nu-designer.override.yml index 4819e8a33f9..c16f4d402a7 100644 --- a/e2e-tests/src/test/resources/batch-nu-designer.override.yml +++ b/e2e-tests/src/test/resources/batch-nu-designer.override.yml @@ -4,7 +4,12 @@ services: environment: CONFIG_FILE: "/opt/nussknacker/conf/application.conf,/opt/nussknacker/conf/application-customizations.conf,/opt/nussknacker/conf/batch-customizations.conf" TABLES_DEFINITION_FILE: "/opt/nussknacker/conf/tables-definition.sql" + JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=500M -XX:MaxDirectMemorySize=300M" volumes: - ../../e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf:/opt/nussknacker/conf/batch-customizations.conf - ../../e2e-tests/src/test/resources/batch-data-generation/transactions:/transactions - ../../e2e-tests/src/test/resources/batch-data-generation/tables-definition.sql:/opt/nussknacker/conf/tables-definition.sql + deploy: + resources: + limits: + memory: 1500M diff --git a/examples/installation/docker-compose.yml b/examples/installation/docker-compose.yml index fa009c0be50..d155975c2fd 100644 --- a/examples/installation/docker-compose.yml +++ b/examples/installation/docker-compose.yml @@ -41,7 +41,7 @@ services: SCHEMA_REGISTRY_URL: "http://schema-registry:8081" INFLUXDB_URL: "http://influxdb:8086" FLINK_REST_URL: "http://flink-jobmanager:8081" - JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=300M -XX:MaxDirectMemorySize=200M" + JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=300M -XX:MaxDirectMemorySize=100M" USAGE_REPORTS_SOURCE: "example-installation-docker-compose" depends_on: postgres: @@ -67,7 +67,7 @@ services: deploy: resources: limits: - memory: 1024M + memory: 1000M postgres: image: postgres:13 diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 2ed3a655240..20dba8eed5f 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -13,7 +13,7 @@ - + @@ -66,6 +66,10 @@ + + + + diff --git a/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala b/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala index e0622d89710..e2dd40a5272 100644 --- a/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala +++ b/utils/test-utils/src/main/scala/pl/touk/nussknacker/test/installationexample/DockerBasedNuInstallationExampleEnvironment.scala @@ -43,7 +43,7 @@ class DockerBasedInstallationExampleNuEnvironment( ) ), // Change to 'true' to enable logging - tailChildContainers = true + tailChildContainers = false ) { start() From 8f0806a9883858bf686e70e06328270fe80dcf2c Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 13:28:57 +0100 Subject: [PATCH 39/48] revert pipeline changes --- .github/workflows/pr.yml | 952 +++++++++++++++++++-------------------- 1 file changed, 474 insertions(+), 478 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index 81b8b4254ac..50d56647349 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -69,113 +69,110 @@ jobs: echo "scala_version_matrix=[\"2.13\"]" >> $GITHUB_OUTPUT fi + build: + name: Build + runs-on: ubuntu-latest + needs: [ setup ] + env: + # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one + shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + steps: + - name: Cancel previous runs + if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + if: ${{ env.shouldPerformBackendBuild == 'true' }} + - uses: coursier/setup-action@v1 + if: ${{ env.shouldPerformBackendBuild == 'true' }} + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + if: ${{ env.shouldPerformBackendBuild == 'true' }} + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - name: Build + if: ${{ env.shouldPerformBackendBuild == 'true' }} + shell: bash + run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile + - name: Tar artifacts + if: ${{ env.shouldPerformBackendBuild == 'true' }} + shell: bash + run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' + - name: Store target + if: ${{ env.shouldPerformBackendBuild == 'true' }} + uses: actions/upload-artifact@v4 + with: + name: build-target + path: target.tgz + + build-fe: + name: BuildFrontend + runs-on: ubuntu-latest + needs: [ setup ] + env: + NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - name: Cache npm + id: cache-npm + uses: actions/cache@v3 + with: + path: | + **/node_modules + ~/.cache/Cypress + key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} + - name: Get Node.js version + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - name: Download node modules + if: steps.cache-npm.outputs.cache-hit != 'true' + run: | + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + - name: Build FE + run: (cd designer/client; npm run build) + - name: Build FE submodules + run: | + cd designer + cp -r client/.federated-types/nussknackerUi submodules/types/@remote + cd submodules + npm run build + - name: Tar fe artifacts + shell: bash + run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist + - name: Store dist + uses: actions/upload-artifact@v4 + with: + name: build-fe-dist + path: fe-dist.tgz -# build: -# name: Build -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one -# shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# steps: -# - name: Cancel previous runs -# if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# - uses: coursier/setup-action@v1 -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - name: Build -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# shell: bash -# run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile -# - name: Tar artifacts -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# shell: bash -# run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' -# - name: Store target -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# uses: actions/upload-artifact@v4 -# with: -# name: build-target -# path: target.tgz -# -# build-fe: -# name: BuildFrontend -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - name: Cache npm -# id: cache-npm -# uses: actions/cache@v3 -# with: -# path: | -# **/node_modules -# ~/.cache/Cypress -# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} -# - name: Get Node.js version -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - name: Download node modules -# if: steps.cache-npm.outputs.cache-hit != 'true' -# run: | -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# - name: Build FE -# run: (cd designer/client; npm run build) -# - name: Build FE submodules -# run: | -# cd designer -# cp -r client/.federated-types/nussknackerUi submodules/types/@remote -# cd submodules -# npm run build -# - name: Tar fe artifacts -# shell: bash -# run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist -# - name: Store dist -# uses: actions/upload-artifact@v4 -# with: -# name: build-fe-dist -# path: fe-dist.tgz -# tests: name: Tests runs-on: ubuntu-latest -# needs: [ build, setup ] - needs: [ setup ] + needs: [ build, setup ] strategy: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we # have to run these tests always for every scala version. See NuDesignerApiAvailableToExposeYamlSpec -# scalaVersion: [2.12, 2.13] - scalaVersion: [2.12] + scalaVersion: [2.12, 2.13] env: NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} @@ -189,12 +186,12 @@ jobs: - uses: coursier/setup-action@v1 with: jvm: temurin:1.11.0.17 -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz - uses: sbt/setup-sbt@v1 - name: Cache ivy packages uses: actions/cache@v3 @@ -207,382 +204,381 @@ jobs: restore-keys: sbt - name: Backend tests shell: bash -# run: ./ciRunSbt.sh test - run: ./ciRunSbt.sh e2eTests/test + run: ./ciRunSbt.sh test - name: Test Report if: success() || failure() uses: actions/upload-artifact@v4 with: name: BackendTests-results-${{ matrix.scalaVersion }} path: '**/test-reports/*.xml' -# -# integrationTests: -# name: IntegrationTests -# needs: [ build, setup ] -# strategy: -# matrix: -# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} -# env: -# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} -# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# runs-on: ubuntu-latest -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - uses: coursier/setup-action@v1 -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz -# - uses: sbt/setup-sbt@v1 -# - name: Integration tests -# shell: bash -# env: -# dockerUpLatest: true -# run: ./ciRunSbt.sh It/test -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: IntegrationTests-results-${{ matrix.scalaVersion }} -# path: '**/it-reports/*.xml' -# -# slowTests: -# name: SlowTests -# runs-on: ubuntu-latest -# needs: [ build, setup ] -# strategy: -# matrix: -# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} -# env: -# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} -# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - uses: coursier/setup-action@v1 -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz -# # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally -# - uses: AbsaOSS/k3d-action@v2.4.0 -# env: -# PROJECT_ROOT: ${{ github.workspace }} -# with: -# cluster-name: "k3s-default" -# k3d-version: "v5.6.0" -# args: >- -# --config=.k3d/single-cluster.yml -# --trace -# - name: Slow tests -# env: -# AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} -# AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} -# AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} -# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} -# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} -# shell: bash -# run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test -# - name: docker logs -# if: success() || failure() -# run: | -# docker logs k3d-k3s-default-server-0 -# docker logs k3d-k3s-default-agent-0 -# docker logs k3d-k3s-default-serverlb -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: SlowTests-results-${{ matrix.scalaVersion }} -# path: '**/test-reports/*.xml' -# -# frontendTests: -# name: FrontendTests -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. -# shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} -# steps: -# - name: Cancel previous runs -# if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# - name: Cache npm -# id: cache-npm -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# uses: actions/cache@v3 -# with: -# path: | -# **/node_modules -# ~/.cache/Cypress -# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} -# - name: Get Node.js version -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - name: Download node modules -# if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} -# run: | -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# - name: Test FE -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# run: (cd designer/client; npm run test:unit) -# - name: Test Report -# if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} -# uses: actions/upload-artifact@v4 -# with: -# name: FrontendTests-results -# path: designer/client/junit.xml -# -# cypressTests: -# name: CypressTests -# runs-on: self-hosted -# needs: [ build, build-fe, setup ] -# env: -# # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests -# # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we -# # have this conditional logic in this step. We force building images on our "special" branches because run between merges -# # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). -# shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} -# BE_PORT: 7251 -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all -# - name: Clean all docker containers -# shell: bash -# run: | -# c=$(docker ps -q) -# [[ $c ]] && docker kill $c || echo "No container run" -# - uses: actions/checkout@v3 -# - name: Get Node.js version -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - uses: coursier/setup-action@v1 -# if: ${{ env.shouldBuildImage == 'true' }} -# with: -# jvm: temurin:1.11.0.17 -# - name: Download node modules -# run: | -# export npm_config_cache=$(mktemp -d) -# npm config ls -l | grep cache -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# rm -rf $npm_config_cache -# - uses: actions/download-artifact@v4 -# if: ${{ env.shouldBuildImage == 'true' }} -# with: -# name: build-target -# - name: Untar artifacts -# if: ${{ env.shouldBuildImage == 'true' }} -# shell: bash -# run: tar xfz target.tgz -# - uses: actions/download-artifact@v4 -# with: -# name: build-fe-dist -# - name: Untar fe artifacts -# shell: bash -# run: tar xfz fe-dist.tgz -# - name: set version -# if: ${{ env.shouldBuildImage == 'true' }} -# shell: bash -# run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV -# - name: Prepare docker -# if: ${{ env.shouldBuildImage == 'true' }} -# env: -# addDevArtifacts: true -# shell: bash -# #Doc generation is rather costly, we don't want it in test image creation -# run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal -# - name: FE tests e2e on build docker image -# if: ${{ env.shouldBuildImage == 'true' }} -# env: -# CYPRESS_SNAPSHOT_UPDATE: "true" -# CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} -# DOCKER_PULL_OPTION: never -# shell: bash -# run: | -# cd designer/client -# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) -# npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e -# - name: Determine docker tag version to use -# if: ${{ env.shouldBuildImage == 'false' }} -# shell: bash -# run: | -# # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. -# NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` -# NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` -# echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV -# - name: FE tests e2e on pulled image -# if: ${{ env.shouldBuildImage == 'false' }} -# env: -# CYPRESS_SNAPSHOT_UPDATE: "true" -# shell: bash -# run: | -# cd designer/client -# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) -# npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: CypressTests-results -# path: designer/client/cypress-test-results/*.xml -# - name: Store test results -# if: failure() -# uses: actions/upload-artifact@v4 -# with: -# name: e2e-test-results -# path: | -# docs/autoScreenshotChangeDocs/ -# designer/client/cypress/**/__image_snapshots__/ -# designer/client/cypress/screenshots/ -# designer/client/cypress/videos/ -# if-no-files-found: ignore -# - name: Create Pull Request -# id: update_snapshots -# uses: peter-evans/create-pull-request@v7 -# if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) -# env: -# HASH: ${{ format('#{0}', github.event.number) }} -# BRANCH: ${{ needs.setup.outputs.git_source_branch }} -# with: -# token: ${{ secrets.GITHUB_TOKEN }} -# base: ${{ env.BRANCH }} -# branch: snapshots-patch/${{ env.BRANCH }} -# title: Update Cypress snapshots in ${{ env.BRANCH }} -# commit-message: Updated snapshots -# body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} -# labels: cypress -# - name: Comment PR -# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} -# uses: thollander/actions-comment-pull-request@v2 -# with: -# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} -# message: | -# ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} -# :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. -# comment_tag: snapshots_pr -# - name: Force fail if update PR created -# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} -# uses: actions/github-script@v6.3.3 -# with: -# script: | -# core.setFailed('Snapshots updated, chceck comments!') -# -# publish-after-tests: -# name: Publish With Tests -# needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] -# uses: ./.github/workflows/publish.yml -# with: -# should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# secrets: -# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} -# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} -# sonatype_user: ${{ secrets.SONATYPE_USER }} -# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} -# github__token: ${{ secrets.GITHUB_TOKEN }} -# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} -# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} -# -# publish-before-tests: -# name: Publish Without Tests -# needs: ['build', 'build-fe', 'setup'] -# uses: ./.github/workflows/publish.yml -# with: -# should_run: ${{ github.ref == 'refs/heads/staging' }} -# secrets: -# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} -# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} -# sonatype_user: ${{ secrets.SONATYPE_USER }} -# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} -# github__token: ${{ secrets.GITHUB_TOKEN }} -# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} -# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} -# -# trigger-external-build: -# runs-on: ubuntu-latest -# needs: [ setup, publish-before-tests ] -# if: ${{ github.ref == 'refs/heads/staging' }} -# strategy: -# matrix: -# include: -# - repo: nussknacker-helm -# workflow_id: helm-test-workflow.yaml -# ref: main -# - repo: nussknacker-quickstart -# workflow_id: pr.yml -# ref: staging -# - repo: nussknacker-sample-components -# workflow_id: pr.yml -# ref: staging -# - repo: nussknacker-flink-compatibility -# workflow_id: test.yml -# ref: staging -# steps: -# - uses: actions/github-script@v6 -# with: -# github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} -# script: | -# await github.rest.actions.createWorkflowDispatch({ -# owner: context.repo.owner, -# repo: '${{ matrix.repo }}', -# workflow_id: '${{ matrix.workflow_id }}', -# ref: '${{ matrix.ref }}', -# inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} -# }) + + integrationTests: + name: IntegrationTests + needs: [ build, setup ] + strategy: + matrix: + scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} + env: + NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} + if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + runs-on: ubuntu-latest + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - uses: coursier/setup-action@v1 + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz + - uses: sbt/setup-sbt@v1 + - name: Integration tests + shell: bash + env: + dockerUpLatest: true + run: ./ciRunSbt.sh It/test + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: IntegrationTests-results-${{ matrix.scalaVersion }} + path: '**/it-reports/*.xml' + + slowTests: + name: SlowTests + runs-on: ubuntu-latest + needs: [ build, setup ] + strategy: + matrix: + scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} + env: + NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} + if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - uses: coursier/setup-action@v1 + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz + # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally + - uses: AbsaOSS/k3d-action@v2.4.0 + env: + PROJECT_ROOT: ${{ github.workspace }} + with: + cluster-name: "k3s-default" + k3d-version: "v5.6.0" + args: >- + --config=.k3d/single-cluster.yml + --trace + - name: Slow tests + env: + AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} + AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} + AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} + AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} + AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} + shell: bash + run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test + - name: docker logs + if: success() || failure() + run: | + docker logs k3d-k3s-default-server-0 + docker logs k3d-k3s-default-agent-0 + docker logs k3d-k3s-default-serverlb + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: SlowTests-results-${{ matrix.scalaVersion }} + path: '**/test-reports/*.xml' + + frontendTests: + name: FrontendTests + runs-on: ubuntu-latest + needs: [ setup ] + env: + # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. + shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} + steps: + - name: Cancel previous runs + if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + if: ${{ env.shouldPerformFrontendTests == 'true' }} + - name: Cache npm + id: cache-npm + if: ${{ env.shouldPerformFrontendTests == 'true' }} + uses: actions/cache@v3 + with: + path: | + **/node_modules + ~/.cache/Cypress + key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} + - name: Get Node.js version + if: ${{ env.shouldPerformFrontendTests == 'true' }} + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + if: ${{ env.shouldPerformFrontendTests == 'true' }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - name: Download node modules + if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} + run: | + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + - name: Test FE + if: ${{ env.shouldPerformFrontendTests == 'true' }} + run: (cd designer/client; npm run test:unit) + - name: Test Report + if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} + uses: actions/upload-artifact@v4 + with: + name: FrontendTests-results + path: designer/client/junit.xml + + cypressTests: + name: CypressTests + runs-on: self-hosted + needs: [ build, build-fe, setup ] + env: + # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests + # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we + # have this conditional logic in this step. We force building images on our "special" branches because run between merges + # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). + shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} + BE_PORT: 7251 + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all + - name: Clean all docker containers + shell: bash + run: | + c=$(docker ps -q) + [[ $c ]] && docker kill $c || echo "No container run" + - uses: actions/checkout@v3 + - name: Get Node.js version + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - uses: coursier/setup-action@v1 + if: ${{ env.shouldBuildImage == 'true' }} + with: + jvm: temurin:1.11.0.17 + - name: Download node modules + run: | + export npm_config_cache=$(mktemp -d) + npm config ls -l | grep cache + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + rm -rf $npm_config_cache + - uses: actions/download-artifact@v4 + if: ${{ env.shouldBuildImage == 'true' }} + with: + name: build-target + - name: Untar artifacts + if: ${{ env.shouldBuildImage == 'true' }} + shell: bash + run: tar xfz target.tgz + - uses: actions/download-artifact@v4 + with: + name: build-fe-dist + - name: Untar fe artifacts + shell: bash + run: tar xfz fe-dist.tgz + - name: set version + if: ${{ env.shouldBuildImage == 'true' }} + shell: bash + run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV + - name: Prepare docker + if: ${{ env.shouldBuildImage == 'true' }} + env: + addDevArtifacts: true + shell: bash + #Doc generation is rather costly, we don't want it in test image creation + run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal + - name: FE tests e2e on build docker image + if: ${{ env.shouldBuildImage == 'true' }} + env: + CYPRESS_SNAPSHOT_UPDATE: "true" + CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} + DOCKER_PULL_OPTION: never + shell: bash + run: | + cd designer/client + ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) + npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e + - name: Determine docker tag version to use + if: ${{ env.shouldBuildImage == 'false' }} + shell: bash + run: | + # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. + NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` + NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` + echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV + - name: FE tests e2e on pulled image + if: ${{ env.shouldBuildImage == 'false' }} + env: + CYPRESS_SNAPSHOT_UPDATE: "true" + shell: bash + run: | + cd designer/client + ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) + npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: CypressTests-results + path: designer/client/cypress-test-results/*.xml + - name: Store test results + if: failure() + uses: actions/upload-artifact@v4 + with: + name: e2e-test-results + path: | + docs/autoScreenshotChangeDocs/ + designer/client/cypress/**/__image_snapshots__/ + designer/client/cypress/screenshots/ + designer/client/cypress/videos/ + if-no-files-found: ignore + - name: Create Pull Request + id: update_snapshots + uses: peter-evans/create-pull-request@v7 + if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) + env: + HASH: ${{ format('#{0}', github.event.number) }} + BRANCH: ${{ needs.setup.outputs.git_source_branch }} + with: + token: ${{ secrets.GITHUB_TOKEN }} + base: ${{ env.BRANCH }} + branch: snapshots-patch/${{ env.BRANCH }} + title: Update Cypress snapshots in ${{ env.BRANCH }} + commit-message: Updated snapshots + body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} + labels: cypress + - name: Comment PR + if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} + uses: thollander/actions-comment-pull-request@v2 + with: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + message: | + ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} + :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. + comment_tag: snapshots_pr + - name: Force fail if update PR created + if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} + uses: actions/github-script@v6.3.3 + with: + script: | + core.setFailed('Snapshots updated, chceck comments!') + + publish-after-tests: + name: Publish With Tests + needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] + uses: ./.github/workflows/publish.yml + with: + should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + secrets: + nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} + git_source_branch: ${{ needs.setup.outputs.git_source_branch }} + sonatype_user: ${{ secrets.SONATYPE_USER }} + sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} + github__token: ${{ secrets.GITHUB_TOKEN }} + dockerhub_user: ${{ secrets.DOCKERHUB_USER }} + dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} + + publish-before-tests: + name: Publish Without Tests + needs: ['build', 'build-fe', 'setup'] + uses: ./.github/workflows/publish.yml + with: + should_run: ${{ github.ref == 'refs/heads/staging' }} + secrets: + nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} + git_source_branch: ${{ needs.setup.outputs.git_source_branch }} + sonatype_user: ${{ secrets.SONATYPE_USER }} + sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} + github__token: ${{ secrets.GITHUB_TOKEN }} + dockerhub_user: ${{ secrets.DOCKERHUB_USER }} + dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} + + trigger-external-build: + runs-on: ubuntu-latest + needs: [ setup, publish-before-tests ] + if: ${{ github.ref == 'refs/heads/staging' }} + strategy: + matrix: + include: + - repo: nussknacker-helm + workflow_id: helm-test-workflow.yaml + ref: main + - repo: nussknacker-quickstart + workflow_id: pr.yml + ref: staging + - repo: nussknacker-sample-components + workflow_id: pr.yml + ref: staging + - repo: nussknacker-flink-compatibility + workflow_id: test.yml + ref: staging + steps: + - uses: actions/github-script@v6 + with: + github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} + script: | + await github.rest.actions.createWorkflowDispatch({ + owner: context.repo.owner, + repo: '${{ matrix.repo }}', + workflow_id: '${{ matrix.workflow_id }}', + ref: '${{ matrix.ref }}', + inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} + }) From 272d13928dca015c7107aad4f4fe98b652d6e972 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 13:30:19 +0100 Subject: [PATCH 40/48] revert compos changes --- examples/installation/docker-compose.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/installation/docker-compose.yml b/examples/installation/docker-compose.yml index d155975c2fd..030dd6a73b7 100644 --- a/examples/installation/docker-compose.yml +++ b/examples/installation/docker-compose.yml @@ -59,7 +59,7 @@ services: healthcheck: test: [ "CMD-SHELL", "curl localhost:8080/api/app/healthCheck" ] interval: 10s - retries: 15 + retries: 10 volumes: - ./designer/application-customizations.conf:/opt/nussknacker/conf/application-customizations.conf volumes_from: @@ -67,7 +67,7 @@ services: deploy: resources: limits: - memory: 1000M + memory: 1024M postgres: image: postgres:13 From bbb2eba23ab5e6b4799cc81b4d9b36d745678aee Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 13:31:47 +0100 Subject: [PATCH 41/48] revert logback changes --- utils/test-utils/src/main/resources/logback-test.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 20dba8eed5f..961b23b4298 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -18,7 +18,7 @@ - + @@ -73,6 +73,7 @@ + From c01a5695145d6390fef4b724df22b7aa25fd9af1 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 17:41:11 +0100 Subject: [PATCH 42/48] less memory used by mini clusters --- .../resources/batch-nu-designer.override.yml | 2 ++ .../AdHocMiniClusterFallbackHandler.scala | 2 +- .../ScenarioTestingMiniClusterFactory.scala | 16 +++++----------- .../ScenarioTestingMiniClusterWrapper.scala | 8 ++++++-- .../engine/management/FlinkConfig.scala | 16 +++++++++++++++- ...FlinkStreamingDeploymentManagerProvider.scala | 4 +++- ...cenarioTestingMiniClusterWrapperFactory.scala | 5 +++-- .../FlinkProcessTestRunnerSpec.scala | 2 ++ .../kafka/KafkaScenarioTestingSpec.scala | 2 ++ .../SchemedKafkaScenarioTestingSpec.scala | 2 ++ .../src/main/resources/logback-test.xml | 2 +- 11 files changed, 42 insertions(+), 19 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-nu-designer.override.yml b/e2e-tests/src/test/resources/batch-nu-designer.override.yml index c16f4d402a7..ff0b35fe551 100644 --- a/e2e-tests/src/test/resources/batch-nu-designer.override.yml +++ b/e2e-tests/src/test/resources/batch-nu-designer.override.yml @@ -4,6 +4,8 @@ services: environment: CONFIG_FILE: "/opt/nussknacker/conf/application.conf,/opt/nussknacker/conf/application-customizations.conf,/opt/nussknacker/conf/batch-customizations.conf" TABLES_DEFINITION_FILE: "/opt/nussknacker/conf/tables-definition.sql" + # Batch adds 2 new processing types, each of them add MiniCluster for scenario testing purpose. Also batch components itself use MiniCluster. + # It increases memory footprint significantly JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=500M -XX:MaxDirectMemorySize=300M" volumes: - ../../e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf:/opt/nussknacker/conf/batch-customizations.conf diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala index dfc9d4dfacd..a1f2f9a906f 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala @@ -38,7 +38,7 @@ object AdHocMiniClusterFallbackHandler extends LazyLogging { .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) .parallelism .getOrElse(1) - ScenarioTestingMiniClusterWrapper.create(scenarioParallelism, new Configuration()) + ScenarioTestingMiniClusterWrapper.create(scenarioParallelism, new Configuration(), new Configuration()) } } diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala index fbdeee9fbfd..030e324e986 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterFactory.scala @@ -6,23 +6,17 @@ import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfigurati object ScenarioTestingMiniClusterFactory { - def createConfiguredMiniCluster(numTaskSlots: Int): MiniCluster = { - val miniClusterConfiguration = prepareMiniClusterConfiguration(numTaskSlots = numTaskSlots) + def createConfiguredMiniCluster(numTaskSlots: Int, miniClusterConfig: Configuration): MiniCluster = { + adjustConfiguration(miniClusterConfig, numTaskSlots = numTaskSlots) // it is required for proper working of HadoopFileSystem - FileSystem.initialize(miniClusterConfiguration, null) + FileSystem.initialize(miniClusterConfig, null) - createMiniCluster(miniClusterConfiguration, numSlotsPerTaskManager = numTaskSlots) + createMiniCluster(miniClusterConfig, numSlotsPerTaskManager = numTaskSlots) } - private def prepareMiniClusterConfiguration(numTaskSlots: Int) = { - val configuration: Configuration = new Configuration + private def adjustConfiguration(configuration: Configuration, numTaskSlots: Int): Unit = { configuration.set[Integer](TaskManagerOptions.NUM_TASK_SLOTS, numTaskSlots) - configuration.set[Integer](RestOptions.PORT, 0) - - // FIXME: reversing flink default order - configuration.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") - configuration } private def createMiniCluster(configuration: Configuration, numSlotsPerTaskManager: Int) = { diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala index 96baca04f7b..8c8789a0667 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -70,9 +70,13 @@ final class ScenarioTestingMiniClusterWrapper( object ScenarioTestingMiniClusterWrapper extends LazyLogging { - def create(parallelism: Int, streamExecutionConfig: Configuration): ScenarioTestingMiniClusterWrapper = { + def create( + parallelism: Int, + miniClusterConfig: Configuration, + streamExecutionConfig: Configuration + ): ScenarioTestingMiniClusterWrapper = { logger.debug(s"Creating MiniCluster with numTaskSlots = $parallelism") - val miniCluster = ScenarioTestingMiniClusterFactory.createConfiguredMiniCluster(parallelism) + val miniCluster = ScenarioTestingMiniClusterFactory.createConfiguredMiniCluster(parallelism, miniClusterConfig) logger.debug( s"Creating local StreamExecutionEnvironment with parallelism = $parallelism and configuration = $streamExecutionConfig" ) diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala index b8ef97670d7..0de1c02fbc0 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkConfig.scala @@ -2,7 +2,7 @@ package pl.touk.nussknacker.engine.management import net.ceedubs.ficus.Ficus import net.ceedubs.ficus.readers.ValueReader -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.{Configuration, CoreOptions, MemorySize, RestOptions, TaskManagerOptions} import scala.concurrent.duration.{DurationInt, FiniteDuration} import scala.jdk.CollectionConverters._ @@ -61,6 +61,7 @@ final case class ScenarioTestingConfig( reuseMiniClusterForScenarioTesting: Boolean = true, reuseMiniClusterForScenarioStateVerification: Boolean = true, parallelism: Int = 1, + miniClusterConfig: Configuration = ScenarioTestingConfig.defaultMiniClusterConfig, streamExecutionConfig: Configuration = new Configuration ) @@ -71,4 +72,17 @@ object ScenarioTestingConfig { implicit val flinkConfigurationValueReader: ValueReader[Configuration] = Ficus.mapValueReader[String].map(map => Configuration.fromMap(map.asJava)) + private[nussknacker] val defaultMiniClusterConfig: Configuration = { + val config = new Configuration + config.set[Integer](RestOptions.PORT, 0) + // FIXME: reversing flink default order + config.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") + // In some setups we create a few Flink DMs. Each of them creates its own mini cluster. + // To reduce footprint we decrease off-heap memory buffers size and managed memory + config.set(TaskManagerOptions.NETWORK_MEMORY_MIN, MemorySize.parse("16m")) + config.set(TaskManagerOptions.NETWORK_MEMORY_MAX, MemorySize.parse("16m")) + config.set(TaskManagerOptions.MANAGED_MEMORY_SIZE, MemorySize.parse("50m")) + config + } + } diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala index b6ae35d6be3..c0dce580579 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/FlinkStreamingDeploymentManagerProvider.scala @@ -2,6 +2,7 @@ package pl.touk.nussknacker.engine.management import cats.data.ValidatedNel import com.typesafe.config.Config +import com.typesafe.scalalogging.LazyLogging import pl.touk.nussknacker.engine._ import pl.touk.nussknacker.engine.api.StreamMetaData import pl.touk.nussknacker.engine.api.component.ScenarioPropertyConfig @@ -15,7 +16,7 @@ import pl.touk.nussknacker.engine.management.rest.FlinkClient import scala.concurrent.duration.FiniteDuration import scala.util.Try -class FlinkStreamingDeploymentManagerProvider extends DeploymentManagerProvider { +class FlinkStreamingDeploymentManagerProvider extends DeploymentManagerProvider with LazyLogging { import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ArbitraryTypeReader._ @@ -28,6 +29,7 @@ class FlinkStreamingDeploymentManagerProvider extends DeploymentManagerProvider deploymentConfig: Config, scenarioStateCacheTTL: Option[FiniteDuration] ): ValidatedNel[String, DeploymentManager] = { + logger.info("Creating FlinkStreamingDeploymentManager") import dependencies._ val flinkConfig = deploymentConfig.rootAs[FlinkConfig] FlinkClient.create(flinkConfig, scenarioStateCacheTTL).map { client => diff --git a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala index b4009e0bc4a..36a2dc8ccfe 100644 --- a/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala +++ b/engine/flink/management/src/main/scala/pl/touk/nussknacker/engine/management/scenariotesting/ScenarioTestingMiniClusterWrapperFactory.scala @@ -12,7 +12,7 @@ object ScenarioTestingMiniClusterWrapperFactory { // Other option would be to add flinkExecutor.jar to classpath from which Flink DM is loaded def createIfConfigured(modelClassLoader: ModelClassLoader, config: ScenarioTestingConfig): Option[AutoCloseable] = { if (config.reuseMiniClusterForScenarioTesting || config.reuseMiniClusterForScenarioStateVerification) { - Some(create(modelClassLoader, config.parallelism, config.streamExecutionConfig)) + Some(create(modelClassLoader, config.parallelism, config.miniClusterConfig, config.streamExecutionConfig)) } else { None } @@ -21,6 +21,7 @@ object ScenarioTestingMiniClusterWrapperFactory { private[nussknacker] def create( modelClassLoader: ModelClassLoader, parallelism: Int, + miniClusterConfig: Configuration, streamExecutionConfig: Configuration ): AutoCloseable = { val methodInvoker = new ReflectiveMethodInvoker[AutoCloseable]( @@ -28,7 +29,7 @@ object ScenarioTestingMiniClusterWrapperFactory { "pl.touk.nussknacker.engine.process.scenariotesting.ScenarioTestingMiniClusterWrapper", "create" ) - methodInvoker.invokeStaticMethod(parallelism, streamExecutionConfig) + methodInvoker.invokeStaticMethod(parallelism, miniClusterConfig, streamExecutionConfig) } } diff --git a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala index 7f073eb8a87..bfb1153446d 100644 --- a/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala +++ b/engine/flink/management/src/test/scala/pl/touk/nussknacker/engine/management/scenariotesting/FlinkProcessTestRunnerSpec.scala @@ -34,6 +34,7 @@ import pl.touk.nussknacker.engine.flink.test.{ import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.graph.node.FragmentInputDefinition.{FragmentClazzRef, FragmentParameter} import pl.touk.nussknacker.engine.graph.node.{Case, FragmentInputDefinition, FragmentOutputDefinition} +import pl.touk.nussknacker.engine.management.ScenarioTestingConfig import pl.touk.nussknacker.engine.management.scenariotesting.FlinkProcessTestRunnerSpec.{ fragmentWithValidationName, processWithFragmentParameterValidation @@ -76,6 +77,7 @@ class FlinkProcessTestRunnerSpec private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( modelClassLoader, parallelism = 1, + miniClusterConfig = ScenarioTestingConfig.defaultMiniClusterConfig, streamExecutionConfig = new Configuration ) diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala index 78e0ce37087..7cce8d298ba 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/kafka/KafkaScenarioTestingSpec.scala @@ -20,6 +20,7 @@ import pl.touk.nussknacker.engine.kafka.KafkaFactory.TopicParamName import pl.touk.nussknacker.engine.kafka.source.InputMeta import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator import pl.touk.nussknacker.engine.kafka.source.flink.KafkaSourceFactoryProcessConfigCreator.ResultsHolders +import pl.touk.nussknacker.engine.management.ScenarioTestingConfig import pl.touk.nussknacker.engine.management.scenariotesting.{ FlinkProcessTestRunner, ScenarioTestingMiniClusterWrapperFactory @@ -67,6 +68,7 @@ class KafkaScenarioTestingSpec private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( modelData.modelClassLoader, parallelism = 1, + miniClusterConfig = ScenarioTestingConfig.defaultMiniClusterConfig, streamExecutionConfig = new Configuration ) diff --git a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala index 5416256b76b..3f8744d9c6b 100644 --- a/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala +++ b/engine/flink/tests/src/test/scala/pl/touk/nussknacker/scenariotesting/schemedkafka/SchemedKafkaScenarioTestingSpec.scala @@ -20,6 +20,7 @@ import pl.touk.nussknacker.engine.flink.util.sink.SingleValueSinkFactory.SingleV import pl.touk.nussknacker.engine.graph.expression.Expression import pl.touk.nussknacker.engine.kafka.UnspecializedTopicName import pl.touk.nussknacker.engine.kafka.source.InputMeta +import pl.touk.nussknacker.engine.management.ScenarioTestingConfig import pl.touk.nussknacker.engine.management.scenariotesting.{ FlinkProcessTestRunner, ScenarioTestingMiniClusterWrapperFactory @@ -88,6 +89,7 @@ class SchemedKafkaScenarioTestingSpec private val scenarioTestingMiniClusterWrapper = ScenarioTestingMiniClusterWrapperFactory.create( modelData.modelClassLoader, parallelism = 1, + miniClusterConfig = ScenarioTestingConfig.defaultMiniClusterConfig, streamExecutionConfig = new Configuration ) diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 961b23b4298..b06a86a4b74 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -18,7 +18,7 @@ - + From 9e65761e0aa2654161685a39816e3c97a14c6395 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 17:44:11 +0100 Subject: [PATCH 43/48] faster feedback --- .github/workflows/pr.yml | 946 ++++++++++++++++++++------------------- 1 file changed, 474 insertions(+), 472 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index 50d56647349..fded02a2da3 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -69,110 +69,112 @@ jobs: echo "scala_version_matrix=[\"2.13\"]" >> $GITHUB_OUTPUT fi - build: - name: Build - runs-on: ubuntu-latest - needs: [ setup ] - env: - # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one - shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - steps: - - name: Cancel previous runs - if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - if: ${{ env.shouldPerformBackendBuild == 'true' }} - - uses: coursier/setup-action@v1 - if: ${{ env.shouldPerformBackendBuild == 'true' }} - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - if: ${{ env.shouldPerformBackendBuild == 'true' }} - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - name: Build - if: ${{ env.shouldPerformBackendBuild == 'true' }} - shell: bash - run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile - - name: Tar artifacts - if: ${{ env.shouldPerformBackendBuild == 'true' }} - shell: bash - run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' - - name: Store target - if: ${{ env.shouldPerformBackendBuild == 'true' }} - uses: actions/upload-artifact@v4 - with: - name: build-target - path: target.tgz - - build-fe: - name: BuildFrontend - runs-on: ubuntu-latest - needs: [ setup ] - env: - NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - name: Cache npm - id: cache-npm - uses: actions/cache@v3 - with: - path: | - **/node_modules - ~/.cache/Cypress - key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} - - name: Get Node.js version - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - name: Download node modules - if: steps.cache-npm.outputs.cache-hit != 'true' - run: | - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - - name: Build FE - run: (cd designer/client; npm run build) - - name: Build FE submodules - run: | - cd designer - cp -r client/.federated-types/nussknackerUi submodules/types/@remote - cd submodules - npm run build - - name: Tar fe artifacts - shell: bash - run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist - - name: Store dist - uses: actions/upload-artifact@v4 - with: - name: build-fe-dist - path: fe-dist.tgz +# build: +# name: Build +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one +# shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# steps: +# - name: Cancel previous runs +# if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# - uses: coursier/setup-action@v1 +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - name: Build +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# shell: bash +# run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile +# - name: Tar artifacts +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# shell: bash +# run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' +# - name: Store target +# if: ${{ env.shouldPerformBackendBuild == 'true' }} +# uses: actions/upload-artifact@v4 +# with: +# name: build-target +# path: target.tgz +# +# build-fe: +# name: BuildFrontend +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - name: Cache npm +# id: cache-npm +# uses: actions/cache@v3 +# with: +# path: | +# **/node_modules +# ~/.cache/Cypress +# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} +# - name: Get Node.js version +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - name: Download node modules +# if: steps.cache-npm.outputs.cache-hit != 'true' +# run: | +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# - name: Build FE +# run: (cd designer/client; npm run build) +# - name: Build FE submodules +# run: | +# cd designer +# cp -r client/.federated-types/nussknackerUi submodules/types/@remote +# cd submodules +# npm run build +# - name: Tar fe artifacts +# shell: bash +# run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist +# - name: Store dist +# uses: actions/upload-artifact@v4 +# with: +# name: build-fe-dist +# path: fe-dist.tgz tests: name: Tests runs-on: ubuntu-latest - needs: [ build, setup ] +# needs: [ build, setup ] + needs: [ setup ] strategy: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we # have to run these tests always for every scala version. See NuDesignerApiAvailableToExposeYamlSpec - scalaVersion: [2.12, 2.13] +# scalaVersion: [2.12, 2.13] + scalaVersion: [2.12] env: NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} @@ -186,12 +188,12 @@ jobs: - uses: coursier/setup-action@v1 with: jvm: temurin:1.11.0.17 - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz - uses: sbt/setup-sbt@v1 - name: Cache ivy packages uses: actions/cache@v3 @@ -212,373 +214,373 @@ jobs: name: BackendTests-results-${{ matrix.scalaVersion }} path: '**/test-reports/*.xml' - integrationTests: - name: IntegrationTests - needs: [ build, setup ] - strategy: - matrix: - scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} - env: - NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} - if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - runs-on: ubuntu-latest - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - uses: coursier/setup-action@v1 - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz - - uses: sbt/setup-sbt@v1 - - name: Integration tests - shell: bash - env: - dockerUpLatest: true - run: ./ciRunSbt.sh It/test - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: IntegrationTests-results-${{ matrix.scalaVersion }} - path: '**/it-reports/*.xml' - - slowTests: - name: SlowTests - runs-on: ubuntu-latest - needs: [ build, setup ] - strategy: - matrix: - scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} - env: - NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} - if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - - uses: coursier/setup-action@v1 - with: - jvm: temurin:1.11.0.17 - - uses: sbt/setup-sbt@v1 - - name: Cache ivy packages - uses: actions/cache@v3 - with: - path: | - ~/.ivy2/cache - ~/.cache/coursier - ~/.sbt - key: sbt-cache-${{ hashFiles('**/*.sbt') }} - restore-keys: sbt - - uses: actions/download-artifact@v4 - with: - name: build-target - - name: Untar artifacts - shell: bash - run: tar xfz target.tgz - # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally - - uses: AbsaOSS/k3d-action@v2.4.0 - env: - PROJECT_ROOT: ${{ github.workspace }} - with: - cluster-name: "k3s-default" - k3d-version: "v5.6.0" - args: >- - --config=.k3d/single-cluster.yml - --trace - - name: Slow tests - env: - AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} - AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} - AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} - AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} - AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} - shell: bash - run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test - - name: docker logs - if: success() || failure() - run: | - docker logs k3d-k3s-default-server-0 - docker logs k3d-k3s-default-agent-0 - docker logs k3d-k3s-default-serverlb - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: SlowTests-results-${{ matrix.scalaVersion }} - path: '**/test-reports/*.xml' - - frontendTests: - name: FrontendTests - runs-on: ubuntu-latest - needs: [ setup ] - env: - # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. - shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} - steps: - - name: Cancel previous runs - if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - - uses: actions/checkout@v3 - if: ${{ env.shouldPerformFrontendTests == 'true' }} - - name: Cache npm - id: cache-npm - if: ${{ env.shouldPerformFrontendTests == 'true' }} - uses: actions/cache@v3 - with: - path: | - **/node_modules - ~/.cache/Cypress - key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} - - name: Get Node.js version - if: ${{ env.shouldPerformFrontendTests == 'true' }} - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - if: ${{ env.shouldPerformFrontendTests == 'true' }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - name: Download node modules - if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} - run: | - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - - name: Test FE - if: ${{ env.shouldPerformFrontendTests == 'true' }} - run: (cd designer/client; npm run test:unit) - - name: Test Report - if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} - uses: actions/upload-artifact@v4 - with: - name: FrontendTests-results - path: designer/client/junit.xml - - cypressTests: - name: CypressTests - runs-on: self-hosted - needs: [ build, build-fe, setup ] - env: - # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests - # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we - # have this conditional logic in this step. We force building images on our "special" branches because run between merges - # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). - shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} - BE_PORT: 7251 - steps: - - name: Cancel previous runs - if: github.event_name != 'push' - uses: styfle/cancel-workflow-action@0.11.0 - with: - access_token: ${{ secrets.GITHUB_TOKEN }} - # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all - - name: Clean all docker containers - shell: bash - run: | - c=$(docker ps -q) - [[ $c ]] && docker kill $c || echo "No container run" - - uses: actions/checkout@v3 - - name: Get Node.js version - id: nvm - run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT - - name: Use Node.js ${{ matrix.node-version }} - uses: actions/setup-node@v3 - with: - node-version: ${{ steps.nvm.outputs.NODE_VERSION }} - - uses: coursier/setup-action@v1 - if: ${{ env.shouldBuildImage == 'true' }} - with: - jvm: temurin:1.11.0.17 - - name: Download node modules - run: | - export npm_config_cache=$(mktemp -d) - npm config ls -l | grep cache - npm ci --no-audit --prefix designer/client - npm ci --no-audit --prefix designer/submodules - rm -rf $npm_config_cache - - uses: actions/download-artifact@v4 - if: ${{ env.shouldBuildImage == 'true' }} - with: - name: build-target - - name: Untar artifacts - if: ${{ env.shouldBuildImage == 'true' }} - shell: bash - run: tar xfz target.tgz - - uses: actions/download-artifact@v4 - with: - name: build-fe-dist - - name: Untar fe artifacts - shell: bash - run: tar xfz fe-dist.tgz - - name: set version - if: ${{ env.shouldBuildImage == 'true' }} - shell: bash - run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV - - name: Prepare docker - if: ${{ env.shouldBuildImage == 'true' }} - env: - addDevArtifacts: true - shell: bash - #Doc generation is rather costly, we don't want it in test image creation - run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal - - name: FE tests e2e on build docker image - if: ${{ env.shouldBuildImage == 'true' }} - env: - CYPRESS_SNAPSHOT_UPDATE: "true" - CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} - DOCKER_PULL_OPTION: never - shell: bash - run: | - cd designer/client - ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) - npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e - - name: Determine docker tag version to use - if: ${{ env.shouldBuildImage == 'false' }} - shell: bash - run: | - # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. - NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` - NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` - echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV - - name: FE tests e2e on pulled image - if: ${{ env.shouldBuildImage == 'false' }} - env: - CYPRESS_SNAPSHOT_UPDATE: "true" - shell: bash - run: | - cd designer/client - ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) - npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e - - name: Test Report - if: success() || failure() - uses: actions/upload-artifact@v4 - with: - name: CypressTests-results - path: designer/client/cypress-test-results/*.xml - - name: Store test results - if: failure() - uses: actions/upload-artifact@v4 - with: - name: e2e-test-results - path: | - docs/autoScreenshotChangeDocs/ - designer/client/cypress/**/__image_snapshots__/ - designer/client/cypress/screenshots/ - designer/client/cypress/videos/ - if-no-files-found: ignore - - name: Create Pull Request - id: update_snapshots - uses: peter-evans/create-pull-request@v7 - if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) - env: - HASH: ${{ format('#{0}', github.event.number) }} - BRANCH: ${{ needs.setup.outputs.git_source_branch }} - with: - token: ${{ secrets.GITHUB_TOKEN }} - base: ${{ env.BRANCH }} - branch: snapshots-patch/${{ env.BRANCH }} - title: Update Cypress snapshots in ${{ env.BRANCH }} - commit-message: Updated snapshots - body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} - labels: cypress - - name: Comment PR - if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} - uses: thollander/actions-comment-pull-request@v2 - with: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - message: | - ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} - :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. - comment_tag: snapshots_pr - - name: Force fail if update PR created - if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} - uses: actions/github-script@v6.3.3 - with: - script: | - core.setFailed('Snapshots updated, chceck comments!') - - publish-after-tests: - name: Publish With Tests - needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] - uses: ./.github/workflows/publish.yml - with: - should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} - secrets: - nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} - git_source_branch: ${{ needs.setup.outputs.git_source_branch }} - sonatype_user: ${{ secrets.SONATYPE_USER }} - sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} - github__token: ${{ secrets.GITHUB_TOKEN }} - dockerhub_user: ${{ secrets.DOCKERHUB_USER }} - dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} - - publish-before-tests: - name: Publish Without Tests - needs: ['build', 'build-fe', 'setup'] - uses: ./.github/workflows/publish.yml - with: - should_run: ${{ github.ref == 'refs/heads/staging' }} - secrets: - nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} - git_source_branch: ${{ needs.setup.outputs.git_source_branch }} - sonatype_user: ${{ secrets.SONATYPE_USER }} - sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} - github__token: ${{ secrets.GITHUB_TOKEN }} - dockerhub_user: ${{ secrets.DOCKERHUB_USER }} - dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} - - trigger-external-build: - runs-on: ubuntu-latest - needs: [ setup, publish-before-tests ] - if: ${{ github.ref == 'refs/heads/staging' }} - strategy: - matrix: - include: - - repo: nussknacker-helm - workflow_id: helm-test-workflow.yaml - ref: main - - repo: nussknacker-quickstart - workflow_id: pr.yml - ref: staging - - repo: nussknacker-sample-components - workflow_id: pr.yml - ref: staging - - repo: nussknacker-flink-compatibility - workflow_id: test.yml - ref: staging - steps: - - uses: actions/github-script@v6 - with: - github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} - script: | - await github.rest.actions.createWorkflowDispatch({ - owner: context.repo.owner, - repo: '${{ matrix.repo }}', - workflow_id: '${{ matrix.workflow_id }}', - ref: '${{ matrix.ref }}', - inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} - }) +# integrationTests: +# name: IntegrationTests +# needs: [ build, setup ] +# strategy: +# matrix: +# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} +# env: +# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} +# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# runs-on: ubuntu-latest +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - uses: coursier/setup-action@v1 +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz +# - uses: sbt/setup-sbt@v1 +# - name: Integration tests +# shell: bash +# env: +# dockerUpLatest: true +# run: ./ciRunSbt.sh It/test +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: IntegrationTests-results-${{ matrix.scalaVersion }} +# path: '**/it-reports/*.xml' +# +# slowTests: +# name: SlowTests +# runs-on: ubuntu-latest +# needs: [ build, setup ] +# strategy: +# matrix: +# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} +# env: +# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} +# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# - uses: coursier/setup-action@v1 +# with: +# jvm: temurin:1.11.0.17 +# - uses: sbt/setup-sbt@v1 +# - name: Cache ivy packages +# uses: actions/cache@v3 +# with: +# path: | +# ~/.ivy2/cache +# ~/.cache/coursier +# ~/.sbt +# key: sbt-cache-${{ hashFiles('**/*.sbt') }} +# restore-keys: sbt +# - uses: actions/download-artifact@v4 +# with: +# name: build-target +# - name: Untar artifacts +# shell: bash +# run: tar xfz target.tgz +# # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally +# - uses: AbsaOSS/k3d-action@v2.4.0 +# env: +# PROJECT_ROOT: ${{ github.workspace }} +# with: +# cluster-name: "k3s-default" +# k3d-version: "v5.6.0" +# args: >- +# --config=.k3d/single-cluster.yml +# --trace +# - name: Slow tests +# env: +# AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} +# AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} +# AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} +# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} +# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} +# shell: bash +# run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test +# - name: docker logs +# if: success() || failure() +# run: | +# docker logs k3d-k3s-default-server-0 +# docker logs k3d-k3s-default-agent-0 +# docker logs k3d-k3s-default-serverlb +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: SlowTests-results-${{ matrix.scalaVersion }} +# path: '**/test-reports/*.xml' +# +# frontendTests: +# name: FrontendTests +# runs-on: ubuntu-latest +# needs: [ setup ] +# env: +# # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. +# shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} +# steps: +# - name: Cancel previous runs +# if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# - uses: actions/checkout@v3 +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# - name: Cache npm +# id: cache-npm +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# uses: actions/cache@v3 +# with: +# path: | +# **/node_modules +# ~/.cache/Cypress +# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} +# - name: Get Node.js version +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - name: Download node modules +# if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} +# run: | +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# - name: Test FE +# if: ${{ env.shouldPerformFrontendTests == 'true' }} +# run: (cd designer/client; npm run test:unit) +# - name: Test Report +# if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} +# uses: actions/upload-artifact@v4 +# with: +# name: FrontendTests-results +# path: designer/client/junit.xml +# +# cypressTests: +# name: CypressTests +# runs-on: self-hosted +# needs: [ build, build-fe, setup ] +# env: +# # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests +# # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we +# # have this conditional logic in this step. We force building images on our "special" branches because run between merges +# # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). +# shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} +# BE_PORT: 7251 +# steps: +# - name: Cancel previous runs +# if: github.event_name != 'push' +# uses: styfle/cancel-workflow-action@0.11.0 +# with: +# access_token: ${{ secrets.GITHUB_TOKEN }} +# # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all +# - name: Clean all docker containers +# shell: bash +# run: | +# c=$(docker ps -q) +# [[ $c ]] && docker kill $c || echo "No container run" +# - uses: actions/checkout@v3 +# - name: Get Node.js version +# id: nvm +# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT +# - name: Use Node.js ${{ matrix.node-version }} +# uses: actions/setup-node@v3 +# with: +# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} +# - uses: coursier/setup-action@v1 +# if: ${{ env.shouldBuildImage == 'true' }} +# with: +# jvm: temurin:1.11.0.17 +# - name: Download node modules +# run: | +# export npm_config_cache=$(mktemp -d) +# npm config ls -l | grep cache +# npm ci --no-audit --prefix designer/client +# npm ci --no-audit --prefix designer/submodules +# rm -rf $npm_config_cache +# - uses: actions/download-artifact@v4 +# if: ${{ env.shouldBuildImage == 'true' }} +# with: +# name: build-target +# - name: Untar artifacts +# if: ${{ env.shouldBuildImage == 'true' }} +# shell: bash +# run: tar xfz target.tgz +# - uses: actions/download-artifact@v4 +# with: +# name: build-fe-dist +# - name: Untar fe artifacts +# shell: bash +# run: tar xfz fe-dist.tgz +# - name: set version +# if: ${{ env.shouldBuildImage == 'true' }} +# shell: bash +# run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV +# - name: Prepare docker +# if: ${{ env.shouldBuildImage == 'true' }} +# env: +# addDevArtifacts: true +# shell: bash +# #Doc generation is rather costly, we don't want it in test image creation +# run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal +# - name: FE tests e2e on build docker image +# if: ${{ env.shouldBuildImage == 'true' }} +# env: +# CYPRESS_SNAPSHOT_UPDATE: "true" +# CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} +# DOCKER_PULL_OPTION: never +# shell: bash +# run: | +# cd designer/client +# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) +# npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e +# - name: Determine docker tag version to use +# if: ${{ env.shouldBuildImage == 'false' }} +# shell: bash +# run: | +# # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. +# NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` +# NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` +# echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV +# - name: FE tests e2e on pulled image +# if: ${{ env.shouldBuildImage == 'false' }} +# env: +# CYPRESS_SNAPSHOT_UPDATE: "true" +# shell: bash +# run: | +# cd designer/client +# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) +# npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e +# - name: Test Report +# if: success() || failure() +# uses: actions/upload-artifact@v4 +# with: +# name: CypressTests-results +# path: designer/client/cypress-test-results/*.xml +# - name: Store test results +# if: failure() +# uses: actions/upload-artifact@v4 +# with: +# name: e2e-test-results +# path: | +# docs/autoScreenshotChangeDocs/ +# designer/client/cypress/**/__image_snapshots__/ +# designer/client/cypress/screenshots/ +# designer/client/cypress/videos/ +# if-no-files-found: ignore +# - name: Create Pull Request +# id: update_snapshots +# uses: peter-evans/create-pull-request@v7 +# if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) +# env: +# HASH: ${{ format('#{0}', github.event.number) }} +# BRANCH: ${{ needs.setup.outputs.git_source_branch }} +# with: +# token: ${{ secrets.GITHUB_TOKEN }} +# base: ${{ env.BRANCH }} +# branch: snapshots-patch/${{ env.BRANCH }} +# title: Update Cypress snapshots in ${{ env.BRANCH }} +# commit-message: Updated snapshots +# body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} +# labels: cypress +# - name: Comment PR +# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} +# uses: thollander/actions-comment-pull-request@v2 +# with: +# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} +# message: | +# ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} +# :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. +# comment_tag: snapshots_pr +# - name: Force fail if update PR created +# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} +# uses: actions/github-script@v6.3.3 +# with: +# script: | +# core.setFailed('Snapshots updated, chceck comments!') +# +# publish-after-tests: +# name: Publish With Tests +# needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] +# uses: ./.github/workflows/publish.yml +# with: +# should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} +# secrets: +# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} +# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} +# sonatype_user: ${{ secrets.SONATYPE_USER }} +# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} +# github__token: ${{ secrets.GITHUB_TOKEN }} +# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} +# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} +# +# publish-before-tests: +# name: Publish Without Tests +# needs: ['build', 'build-fe', 'setup'] +# uses: ./.github/workflows/publish.yml +# with: +# should_run: ${{ github.ref == 'refs/heads/staging' }} +# secrets: +# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} +# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} +# sonatype_user: ${{ secrets.SONATYPE_USER }} +# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} +# github__token: ${{ secrets.GITHUB_TOKEN }} +# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} +# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} +# +# trigger-external-build: +# runs-on: ubuntu-latest +# needs: [ setup, publish-before-tests ] +# if: ${{ github.ref == 'refs/heads/staging' }} +# strategy: +# matrix: +# include: +# - repo: nussknacker-helm +# workflow_id: helm-test-workflow.yaml +# ref: main +# - repo: nussknacker-quickstart +# workflow_id: pr.yml +# ref: staging +# - repo: nussknacker-sample-components +# workflow_id: pr.yml +# ref: staging +# - repo: nussknacker-flink-compatibility +# workflow_id: test.yml +# ref: staging +# steps: +# - uses: actions/github-script@v6 +# with: +# github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} +# script: | +# await github.rest.actions.createWorkflowDispatch({ +# owner: context.repo.owner, +# repo: '${{ matrix.repo }}', +# workflow_id: '${{ matrix.workflow_id }}', +# ref: '${{ matrix.ref }}', +# inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} +# }) From 05b1f161616128701ab58c77c54bac81eca58b4d Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 17:47:12 +0100 Subject: [PATCH 44/48] docs + reverted legacy config --- docs/configuration/ScenarioDeploymentConfiguration.md | 1 + .../scenariotesting/AdHocMiniClusterFallbackHandler.scala | 8 ++++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/configuration/ScenarioDeploymentConfiguration.md b/docs/configuration/ScenarioDeploymentConfiguration.md index 3ada853ac4f..9ae7cb2b1ce 100644 --- a/docs/configuration/ScenarioDeploymentConfiguration.md +++ b/docs/configuration/ScenarioDeploymentConfiguration.md @@ -332,4 +332,5 @@ Deployment Manager of type `flinkStreaming` has the following configuration opti | scenarioTesting.reuseMiniClusterForScenarioTesting | boolean | true | Creates mini cluster once and reuses it for each scenario testing attempt | | scenarioTesting.reuseMiniClusterForScenarioStateVerification | boolean | true | Creates mini cluster once and reuses it for each scenario state verification | | scenarioTesting.parallelism | int | 1 | Parallelism that will be used for scenario testing and scenario state verifications mechanisms when mini cluster reusage is enabled; when mini cluster reusage is disabled, parallelism is taken from scenario properties | +| scenarioTesting.miniClusterConfig | map of strings | [:] | Configuration that will be passed to `MiniCluster` for scenario testing and scenario state verifications mechanisms when mini cluster reusage is enabled; when mini cluster reusage is disabled, empty configuration will be used | | scenarioTesting.streamExecutionConfig | map of strings | [:] | Configuration that will be passed to `StreamExecutionEnvironment` for scenario testing and scenario state verifications mechanisms when mini cluster reusage is enabled; when mini cluster reusage is disabled, empty configuration will be used | diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala index a1f2f9a906f..d278ca5ae11 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/AdHocMiniClusterFallbackHandler.scala @@ -1,7 +1,7 @@ package pl.touk.nussknacker.engine.process.scenariotesting import com.typesafe.scalalogging.LazyLogging -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.{Configuration, CoreOptions, RestOptions} import pl.touk.nussknacker.engine.api.StreamMetaData import pl.touk.nussknacker.engine.canonicalgraph.CanonicalProcess import pl.touk.nussknacker.engine.util.MetaDataExtractor @@ -38,7 +38,11 @@ object AdHocMiniClusterFallbackHandler extends LazyLogging { .extractTypeSpecificDataOrDefault[StreamMetaData](process.metaData, StreamMetaData()) .parallelism .getOrElse(1) - ScenarioTestingMiniClusterWrapper.create(scenarioParallelism, new Configuration(), new Configuration()) + val legacyMiniClusterConfig = new Configuration + legacyMiniClusterConfig.set[Integer](RestOptions.PORT, 0) + // FIXME: reversing flink default order + legacyMiniClusterConfig.set(CoreOptions.CLASSLOADER_RESOLVE_ORDER, "parent-first") + ScenarioTestingMiniClusterWrapper.create(scenarioParallelism, legacyMiniClusterConfig, new Configuration()) } } From 4c8a4ba68cb71c511fcde5c7b0491b1889774a4a Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 18:27:04 +0100 Subject: [PATCH 45/48] memory inc --- e2e-tests/src/test/resources/batch-nu-designer.override.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e-tests/src/test/resources/batch-nu-designer.override.yml b/e2e-tests/src/test/resources/batch-nu-designer.override.yml index ff0b35fe551..e42504c5f1a 100644 --- a/e2e-tests/src/test/resources/batch-nu-designer.override.yml +++ b/e2e-tests/src/test/resources/batch-nu-designer.override.yml @@ -6,7 +6,7 @@ services: TABLES_DEFINITION_FILE: "/opt/nussknacker/conf/tables-definition.sql" # Batch adds 2 new processing types, each of them add MiniCluster for scenario testing purpose. Also batch components itself use MiniCluster. # It increases memory footprint significantly - JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=500M -XX:MaxDirectMemorySize=300M" + JDK_JAVA_OPTIONS: "-Xmx400M -XX:MaxMetaspaceSize=500M -XX:MaxDirectMemorySize=200M" volumes: - ../../e2e-tests/src/test/resources/batch-data-generation/batch-customizations.conf:/opt/nussknacker/conf/batch-customizations.conf - ../../e2e-tests/src/test/resources/batch-data-generation/transactions:/transactions @@ -14,4 +14,4 @@ services: deploy: resources: limits: - memory: 1500M + memory: 1300M From 3e1c83a5db2824221691c6992015af9b54da0642 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 19:16:09 +0100 Subject: [PATCH 46/48] e2eTests only --- .github/workflows/pr.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index fded02a2da3..1f6b8a7b182 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -206,7 +206,8 @@ jobs: restore-keys: sbt - name: Backend tests shell: bash - run: ./ciRunSbt.sh test +# run: ./ciRunSbt.sh test + run: ./ciRunSbt.sh e2eTests/test - name: Test Report if: success() || failure() uses: actions/upload-artifact@v4 From 5a5e0afa44659ed79cb2a130b7e674330caaa034 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 20:24:50 +0100 Subject: [PATCH 47/48] more tests --- .github/workflows/pr.yml | 107 +++++++++--------- .../ScenarioTestingMiniClusterWrapper.scala | 1 - .../src/main/resources/logback-test.xml | 2 +- 3 files changed, 54 insertions(+), 56 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index 1f6b8a7b182..98dfe9add1a 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -69,50 +69,50 @@ jobs: echo "scala_version_matrix=[\"2.13\"]" >> $GITHUB_OUTPUT fi -# build: -# name: Build -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one -# shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# steps: -# - name: Cancel previous runs -# if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# - uses: coursier/setup-action@v1 -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - name: Build -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# shell: bash -# run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile -# - name: Tar artifacts -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# shell: bash -# run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' -# - name: Store target -# if: ${{ env.shouldPerformBackendBuild == 'true' }} -# uses: actions/upload-artifact@v4 -# with: -# name: build-target -# path: target.tgz + build: + name: Build + runs-on: ubuntu-latest + needs: [ setup ] + env: + # We can't just use conditional jobs mechanism ('if' directive) because 'cypressTests' job depends on this one + shouldPerformBackendBuild: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + steps: + - name: Cancel previous runs + if: ${{ env.shouldPerformBackendBuild == 'true' && github.event_name != 'push' }} + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + if: ${{ env.shouldPerformBackendBuild == 'true' }} + - uses: coursier/setup-action@v1 + if: ${{ env.shouldPerformBackendBuild == 'true' }} + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + if: ${{ env.shouldPerformBackendBuild == 'true' }} + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - name: Build + if: ${{ env.shouldPerformBackendBuild == 'true' }} + shell: bash + run: CROSS_BUILD=true ./ciRunSbt.sh Compile/compile Test/compile + - name: Tar artifacts + if: ${{ env.shouldPerformBackendBuild == 'true' }} + shell: bash + run: find . -wholename "**/target/**/*" -printf '%P\0' | tar --null -C '.' --files-from=- -czf 'target.tgz' + - name: Store target + if: ${{ env.shouldPerformBackendBuild == 'true' }} + uses: actions/upload-artifact@v4 + with: + name: build-target + path: target.tgz # # build-fe: # name: BuildFrontend @@ -167,8 +167,7 @@ jobs: tests: name: Tests runs-on: ubuntu-latest -# needs: [ build, setup ] - needs: [ setup ] + needs: [ build, setup ] strategy: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we @@ -188,12 +187,12 @@ jobs: - uses: coursier/setup-action@v1 with: jvm: temurin:1.11.0.17 -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz - uses: sbt/setup-sbt@v1 - name: Cache ivy packages uses: actions/cache@v3 @@ -206,8 +205,8 @@ jobs: restore-keys: sbt - name: Backend tests shell: bash -# run: ./ciRunSbt.sh test - run: ./ciRunSbt.sh e2eTests/test + run: ./ciRunSbt.sh test +# run: ./ciRunSbt.sh e2eTests/test - name: Test Report if: success() || failure() uses: actions/upload-artifact@v4 diff --git a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala index 8c8789a0667..dbacb86325e 100644 --- a/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala +++ b/engine/flink/executor/src/main/scala/pl/touk/nussknacker/engine/process/scenariotesting/ScenarioTestingMiniClusterWrapper.scala @@ -13,7 +13,6 @@ import pl.touk.nussknacker.engine.util.loader.ModelClassLoader import scala.jdk.CollectionConverters._ -// We use MiniCluster directly, instead of LocalExecutionEnvironment, to be able to pass own classpath... final class ScenarioTestingMiniClusterWrapper( miniCluster: MiniCluster, val env: StreamExecutionEnvironment, diff --git a/utils/test-utils/src/main/resources/logback-test.xml b/utils/test-utils/src/main/resources/logback-test.xml index 9f58c3839e4..79fb9f1fc41 100644 --- a/utils/test-utils/src/main/resources/logback-test.xml +++ b/utils/test-utils/src/main/resources/logback-test.xml @@ -18,7 +18,7 @@ - + From 6934bb418db2209787feb0a76eb82908610713b5 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Fri, 24 Jan 2025 20:57:52 +0100 Subject: [PATCH 48/48] tests revert --- .github/workflows/pr.yml | 844 +++++++++++++++++++-------------------- 1 file changed, 421 insertions(+), 423 deletions(-) diff --git a/.github/workflows/pr.yml b/.github/workflows/pr.yml index 98dfe9add1a..50d56647349 100644 --- a/.github/workflows/pr.yml +++ b/.github/workflows/pr.yml @@ -113,56 +113,56 @@ jobs: with: name: build-target path: target.tgz -# -# build-fe: -# name: BuildFrontend -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - name: Cache npm -# id: cache-npm -# uses: actions/cache@v3 -# with: -# path: | -# **/node_modules -# ~/.cache/Cypress -# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} -# - name: Get Node.js version -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - name: Download node modules -# if: steps.cache-npm.outputs.cache-hit != 'true' -# run: | -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# - name: Build FE -# run: (cd designer/client; npm run build) -# - name: Build FE submodules -# run: | -# cd designer -# cp -r client/.federated-types/nussknackerUi submodules/types/@remote -# cd submodules -# npm run build -# - name: Tar fe artifacts -# shell: bash -# run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist -# - name: Store dist -# uses: actions/upload-artifact@v4 -# with: -# name: build-fe-dist -# path: fe-dist.tgz + + build-fe: + name: BuildFrontend + runs-on: ubuntu-latest + needs: [ setup ] + env: + NUSSKNACKER_VERSION: ${{ needs.setup.outputs.nk_snapshot_version }} + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - name: Cache npm + id: cache-npm + uses: actions/cache@v3 + with: + path: | + **/node_modules + ~/.cache/Cypress + key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} + - name: Get Node.js version + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - name: Download node modules + if: steps.cache-npm.outputs.cache-hit != 'true' + run: | + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + - name: Build FE + run: (cd designer/client; npm run build) + - name: Build FE submodules + run: | + cd designer + cp -r client/.federated-types/nussknackerUi submodules/types/@remote + cd submodules + npm run build + - name: Tar fe artifacts + shell: bash + run: tar -czf 'fe-dist.tgz' designer/client/dist designer/submodules/dist + - name: Store dist + uses: actions/upload-artifact@v4 + with: + name: build-fe-dist + path: fe-dist.tgz tests: name: Tests @@ -172,8 +172,7 @@ jobs: matrix: # Due to the different behaviour of Nussknacker's OpenAPI generation on different versions of scala we # have to run these tests always for every scala version. See NuDesignerApiAvailableToExposeYamlSpec -# scalaVersion: [2.12, 2.13] - scalaVersion: [2.12] + scalaVersion: [2.12, 2.13] env: NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} @@ -206,7 +205,6 @@ jobs: - name: Backend tests shell: bash run: ./ciRunSbt.sh test -# run: ./ciRunSbt.sh e2eTests/test - name: Test Report if: success() || failure() uses: actions/upload-artifact@v4 @@ -214,373 +212,373 @@ jobs: name: BackendTests-results-${{ matrix.scalaVersion }} path: '**/test-reports/*.xml' -# integrationTests: -# name: IntegrationTests -# needs: [ build, setup ] -# strategy: -# matrix: -# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} -# env: -# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} -# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# runs-on: ubuntu-latest -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - uses: coursier/setup-action@v1 -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz -# - uses: sbt/setup-sbt@v1 -# - name: Integration tests -# shell: bash -# env: -# dockerUpLatest: true -# run: ./ciRunSbt.sh It/test -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: IntegrationTests-results-${{ matrix.scalaVersion }} -# path: '**/it-reports/*.xml' -# -# slowTests: -# name: SlowTests -# runs-on: ubuntu-latest -# needs: [ build, setup ] -# strategy: -# matrix: -# scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} -# env: -# NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} -# if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# - uses: coursier/setup-action@v1 -# with: -# jvm: temurin:1.11.0.17 -# - uses: sbt/setup-sbt@v1 -# - name: Cache ivy packages -# uses: actions/cache@v3 -# with: -# path: | -# ~/.ivy2/cache -# ~/.cache/coursier -# ~/.sbt -# key: sbt-cache-${{ hashFiles('**/*.sbt') }} -# restore-keys: sbt -# - uses: actions/download-artifact@v4 -# with: -# name: build-target -# - name: Untar artifacts -# shell: bash -# run: tar xfz target.tgz -# # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally -# - uses: AbsaOSS/k3d-action@v2.4.0 -# env: -# PROJECT_ROOT: ${{ github.workspace }} -# with: -# cluster-name: "k3s-default" -# k3d-version: "v5.6.0" -# args: >- -# --config=.k3d/single-cluster.yml -# --trace -# - name: Slow tests -# env: -# AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} -# AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} -# AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} -# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} -# AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} -# shell: bash -# run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test -# - name: docker logs -# if: success() || failure() -# run: | -# docker logs k3d-k3s-default-server-0 -# docker logs k3d-k3s-default-agent-0 -# docker logs k3d-k3s-default-serverlb -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: SlowTests-results-${{ matrix.scalaVersion }} -# path: '**/test-reports/*.xml' -# -# frontendTests: -# name: FrontendTests -# runs-on: ubuntu-latest -# needs: [ setup ] -# env: -# # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. -# shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} -# steps: -# - name: Cancel previous runs -# if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# - uses: actions/checkout@v3 -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# - name: Cache npm -# id: cache-npm -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# uses: actions/cache@v3 -# with: -# path: | -# **/node_modules -# ~/.cache/Cypress -# key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} -# - name: Get Node.js version -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - name: Download node modules -# if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} -# run: | -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# - name: Test FE -# if: ${{ env.shouldPerformFrontendTests == 'true' }} -# run: (cd designer/client; npm run test:unit) -# - name: Test Report -# if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} -# uses: actions/upload-artifact@v4 -# with: -# name: FrontendTests-results -# path: designer/client/junit.xml -# -# cypressTests: -# name: CypressTests -# runs-on: self-hosted -# needs: [ build, build-fe, setup ] -# env: -# # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests -# # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we -# # have this conditional logic in this step. We force building images on our "special" branches because run between merges -# # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). -# shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} -# BE_PORT: 7251 -# steps: -# - name: Cancel previous runs -# if: github.event_name != 'push' -# uses: styfle/cancel-workflow-action@0.11.0 -# with: -# access_token: ${{ secrets.GITHUB_TOKEN }} -# # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all -# - name: Clean all docker containers -# shell: bash -# run: | -# c=$(docker ps -q) -# [[ $c ]] && docker kill $c || echo "No container run" -# - uses: actions/checkout@v3 -# - name: Get Node.js version -# id: nvm -# run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT -# - name: Use Node.js ${{ matrix.node-version }} -# uses: actions/setup-node@v3 -# with: -# node-version: ${{ steps.nvm.outputs.NODE_VERSION }} -# - uses: coursier/setup-action@v1 -# if: ${{ env.shouldBuildImage == 'true' }} -# with: -# jvm: temurin:1.11.0.17 -# - name: Download node modules -# run: | -# export npm_config_cache=$(mktemp -d) -# npm config ls -l | grep cache -# npm ci --no-audit --prefix designer/client -# npm ci --no-audit --prefix designer/submodules -# rm -rf $npm_config_cache -# - uses: actions/download-artifact@v4 -# if: ${{ env.shouldBuildImage == 'true' }} -# with: -# name: build-target -# - name: Untar artifacts -# if: ${{ env.shouldBuildImage == 'true' }} -# shell: bash -# run: tar xfz target.tgz -# - uses: actions/download-artifact@v4 -# with: -# name: build-fe-dist -# - name: Untar fe artifacts -# shell: bash -# run: tar xfz fe-dist.tgz -# - name: set version -# if: ${{ env.shouldBuildImage == 'true' }} -# shell: bash -# run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV -# - name: Prepare docker -# if: ${{ env.shouldBuildImage == 'true' }} -# env: -# addDevArtifacts: true -# shell: bash -# #Doc generation is rather costly, we don't want it in test image creation -# run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal -# - name: FE tests e2e on build docker image -# if: ${{ env.shouldBuildImage == 'true' }} -# env: -# CYPRESS_SNAPSHOT_UPDATE: "true" -# CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} -# DOCKER_PULL_OPTION: never -# shell: bash -# run: | -# cd designer/client -# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) -# npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e -# - name: Determine docker tag version to use -# if: ${{ env.shouldBuildImage == 'false' }} -# shell: bash -# run: | -# # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. -# NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` -# NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` -# echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV -# - name: FE tests e2e on pulled image -# if: ${{ env.shouldBuildImage == 'false' }} -# env: -# CYPRESS_SNAPSHOT_UPDATE: "true" -# shell: bash -# run: | -# cd designer/client -# ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) -# npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e -# - name: Test Report -# if: success() || failure() -# uses: actions/upload-artifact@v4 -# with: -# name: CypressTests-results -# path: designer/client/cypress-test-results/*.xml -# - name: Store test results -# if: failure() -# uses: actions/upload-artifact@v4 -# with: -# name: e2e-test-results -# path: | -# docs/autoScreenshotChangeDocs/ -# designer/client/cypress/**/__image_snapshots__/ -# designer/client/cypress/screenshots/ -# designer/client/cypress/videos/ -# if-no-files-found: ignore -# - name: Create Pull Request -# id: update_snapshots -# uses: peter-evans/create-pull-request@v7 -# if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) -# env: -# HASH: ${{ format('#{0}', github.event.number) }} -# BRANCH: ${{ needs.setup.outputs.git_source_branch }} -# with: -# token: ${{ secrets.GITHUB_TOKEN }} -# base: ${{ env.BRANCH }} -# branch: snapshots-patch/${{ env.BRANCH }} -# title: Update Cypress snapshots in ${{ env.BRANCH }} -# commit-message: Updated snapshots -# body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} -# labels: cypress -# - name: Comment PR -# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} -# uses: thollander/actions-comment-pull-request@v2 -# with: -# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} -# message: | -# ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} -# :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. -# comment_tag: snapshots_pr -# - name: Force fail if update PR created -# if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} -# uses: actions/github-script@v6.3.3 -# with: -# script: | -# core.setFailed('Snapshots updated, chceck comments!') -# -# publish-after-tests: -# name: Publish With Tests -# needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] -# uses: ./.github/workflows/publish.yml -# with: -# should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} -# secrets: -# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} -# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} -# sonatype_user: ${{ secrets.SONATYPE_USER }} -# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} -# github__token: ${{ secrets.GITHUB_TOKEN }} -# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} -# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} -# -# publish-before-tests: -# name: Publish Without Tests -# needs: ['build', 'build-fe', 'setup'] -# uses: ./.github/workflows/publish.yml -# with: -# should_run: ${{ github.ref == 'refs/heads/staging' }} -# secrets: -# nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} -# git_source_branch: ${{ needs.setup.outputs.git_source_branch }} -# sonatype_user: ${{ secrets.SONATYPE_USER }} -# sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} -# github__token: ${{ secrets.GITHUB_TOKEN }} -# dockerhub_user: ${{ secrets.DOCKERHUB_USER }} -# dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} -# -# trigger-external-build: -# runs-on: ubuntu-latest -# needs: [ setup, publish-before-tests ] -# if: ${{ github.ref == 'refs/heads/staging' }} -# strategy: -# matrix: -# include: -# - repo: nussknacker-helm -# workflow_id: helm-test-workflow.yaml -# ref: main -# - repo: nussknacker-quickstart -# workflow_id: pr.yml -# ref: staging -# - repo: nussknacker-sample-components -# workflow_id: pr.yml -# ref: staging -# - repo: nussknacker-flink-compatibility -# workflow_id: test.yml -# ref: staging -# steps: -# - uses: actions/github-script@v6 -# with: -# github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} -# script: | -# await github.rest.actions.createWorkflowDispatch({ -# owner: context.repo.owner, -# repo: '${{ matrix.repo }}', -# workflow_id: '${{ matrix.workflow_id }}', -# ref: '${{ matrix.ref }}', -# inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} -# }) + integrationTests: + name: IntegrationTests + needs: [ build, setup ] + strategy: + matrix: + scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} + env: + NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} + if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + runs-on: ubuntu-latest + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - uses: coursier/setup-action@v1 + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz + - uses: sbt/setup-sbt@v1 + - name: Integration tests + shell: bash + env: + dockerUpLatest: true + run: ./ciRunSbt.sh It/test + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: IntegrationTests-results-${{ matrix.scalaVersion }} + path: '**/it-reports/*.xml' + + slowTests: + name: SlowTests + runs-on: ubuntu-latest + needs: [ build, setup ] + strategy: + matrix: + scalaVersion: ${{fromJson(needs.setup.outputs.scala_version_matrix)}} + env: + NUSSKNACKER_SCALA_VERSION: ${{ matrix.scalaVersion }} + if: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + - uses: coursier/setup-action@v1 + with: + jvm: temurin:1.11.0.17 + - uses: sbt/setup-sbt@v1 + - name: Cache ivy packages + uses: actions/cache@v3 + with: + path: | + ~/.ivy2/cache + ~/.cache/coursier + ~/.sbt + key: sbt-cache-${{ hashFiles('**/*.sbt') }} + restore-keys: sbt + - uses: actions/download-artifact@v4 + with: + name: build-target + - name: Untar artifacts + shell: bash + run: tar xfz target.tgz + # We run k8s tests here instead of in integrationTests job because it causes RAM overhead and some other docker tests failing occasionally + - uses: AbsaOSS/k3d-action@v2.4.0 + env: + PROJECT_ROOT: ${{ github.workspace }} + with: + cluster-name: "k3s-default" + k3d-version: "v5.6.0" + args: >- + --config=.k3d/single-cluster.yml + --trace + - name: Slow tests + env: + AZURE_CLIENT_ID: ${{ secrets.AZURE_CLIENT_ID }} + AZURE_CLIENT_SECRET: ${{ secrets.AZURE_CLIENT_SECRET }} + AZURE_TENANT_ID: ${{ secrets.AZURE_TENANT_ID }} + AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY_NAME }} + AZURE_EVENT_HUBS_SHARED_ACCESS_KEY: ${{ secrets.AZURE_EVENT_HUBS_SHARED_ACCESS_KEY }} + shell: bash + run: ./ciRunSbt.sh designer/Slow/test liteK8sDeploymentManager/ExternalDepsTests/test schemedKafkaComponentsUtils/ExternalDepsTests/test liteKafkaComponentsTests/ExternalDepsTests/test + - name: docker logs + if: success() || failure() + run: | + docker logs k3d-k3s-default-server-0 + docker logs k3d-k3s-default-agent-0 + docker logs k3d-k3s-default-serverlb + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: SlowTests-results-${{ matrix.scalaVersion }} + path: '**/test-reports/*.xml' + + frontendTests: + name: FrontendTests + runs-on: ubuntu-latest + needs: [ setup ] + env: + # We can't just use conditional jobs mechanism ('if' directive) because 'publish' job depends on this one. + shouldPerformFrontendTests: ${{ needs.setup.outputs.fe_changes_count > 0 }} + steps: + - name: Cancel previous runs + if: ${{ env.shouldPerformFrontendTests == 'true' && github.event_name != 'push' }} + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + - uses: actions/checkout@v3 + if: ${{ env.shouldPerformFrontendTests == 'true' }} + - name: Cache npm + id: cache-npm + if: ${{ env.shouldPerformFrontendTests == 'true' }} + uses: actions/cache@v3 + with: + path: | + **/node_modules + ~/.cache/Cypress + key: ${{ hashFiles('designer/client/package-lock.json', 'designer/client/.nvmrc', 'designer/submodules/package-lock.json', 'designer/submodules/.nvmrc') }} + - name: Get Node.js version + if: ${{ env.shouldPerformFrontendTests == 'true' }} + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + if: ${{ env.shouldPerformFrontendTests == 'true' }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - name: Download node modules + if: ${{ env.shouldPerformFrontendTests == 'true' && steps.cache-npm.outputs.cache-hit != 'true' }} + run: | + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + - name: Test FE + if: ${{ env.shouldPerformFrontendTests == 'true' }} + run: (cd designer/client; npm run test:unit) + - name: Test Report + if: ${{ env.shouldPerformFrontendTests == 'true' && (success() || failure()) }} + uses: actions/upload-artifact@v4 + with: + name: FrontendTests-results + path: designer/client/junit.xml + + cypressTests: + name: CypressTests + runs-on: self-hosted + needs: [ build, build-fe, setup ] + env: + # We skip docker build for fe-only changes. Would be more clean to split this step into two steps: build image and run tests + # e.g. by using ishworkh/docker-image-artifact-upload/download but it caused ~3min overhead for the whole pipeline so we + # have this conditional logic in this step. We force building images on our "special" branches because run between merges + # could cause that cypress tests will be run at stale image (because of cancel-workflow-action). + shouldBuildImage: ${{ needs.setup.outputs.fe_changes_count != needs.setup.outputs.all_changes_count || github.ref == 'refs/heads/staging' || github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + GIT_SOURCE_BRANCH: ${{ needs.setup.outputs.git_source_branch }} + BE_PORT: 7251 + steps: + - name: Cancel previous runs + if: github.event_name != 'push' + uses: styfle/cancel-workflow-action@0.11.0 + with: + access_token: ${{ secrets.GITHUB_TOKEN }} + # On self-hosted runners, docker can have some obsolete containers. Because of that, we clean them all + - name: Clean all docker containers + shell: bash + run: | + c=$(docker ps -q) + [[ $c ]] && docker kill $c || echo "No container run" + - uses: actions/checkout@v3 + - name: Get Node.js version + id: nvm + run: echo "NODE_VERSION=$(cat designer/client/.nvmrc)" >> $GITHUB_OUTPUT + - name: Use Node.js ${{ matrix.node-version }} + uses: actions/setup-node@v3 + with: + node-version: ${{ steps.nvm.outputs.NODE_VERSION }} + - uses: coursier/setup-action@v1 + if: ${{ env.shouldBuildImage == 'true' }} + with: + jvm: temurin:1.11.0.17 + - name: Download node modules + run: | + export npm_config_cache=$(mktemp -d) + npm config ls -l | grep cache + npm ci --no-audit --prefix designer/client + npm ci --no-audit --prefix designer/submodules + rm -rf $npm_config_cache + - uses: actions/download-artifact@v4 + if: ${{ env.shouldBuildImage == 'true' }} + with: + name: build-target + - name: Untar artifacts + if: ${{ env.shouldBuildImage == 'true' }} + shell: bash + run: tar xfz target.tgz + - uses: actions/download-artifact@v4 + with: + name: build-fe-dist + - name: Untar fe artifacts + shell: bash + run: tar xfz fe-dist.tgz + - name: set version + if: ${{ env.shouldBuildImage == 'true' }} + shell: bash + run: echo "NUSSKNACKER_VERSION=${{ needs.setup.outputs.nk_snapshot_version }}" >> $GITHUB_ENV + - name: Prepare docker + if: ${{ env.shouldBuildImage == 'true' }} + env: + addDevArtifacts: true + shell: bash + #Doc generation is rather costly, we don't want it in test image creation + run: sbt "set ThisBuild / version := \"$NUSSKNACKER_VERSION\"; set ThisBuild / packageDoc / publishArtifact := false; set Compile / doc / sources := Seq.empty" dist/Docker/publishLocal + - name: FE tests e2e on build docker image + if: ${{ env.shouldBuildImage == 'true' }} + env: + CYPRESS_SNAPSHOT_UPDATE: "true" + CYPRESS_BASE_URL: http://localhost:${{ env.BE_PORT }} + DOCKER_PULL_OPTION: never + shell: bash + run: | + cd designer/client + ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) + npx start-server-and-test backend:docker ${{ env.BE_PORT }} test:e2e + - name: Determine docker tag version to use + if: ${{ env.shouldBuildImage == 'false' }} + shell: bash + run: | + # Take a look at build.sbt commonDockerSettings to see how this tag is determined. Thanks to fact that we publish all changes pushed to our "special" branches it should work quite correctly. + NK_REF_VERSION=`[ "${GITHUB_REF}" != "" ] && echo "${GITHUB_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` + NK_BASE_REF_VERSION=`[ "${NK_REF_VERSION}" != "" ] && echo "${NK_REF_VERSION}" || [ "${GITHUB_BASE_REF}" != "" ] && echo "${GITHUB_BASE_REF}" | sed -e 's/refs\/heads\///g' -e 's/[^a-zA-Z0-9._-]/\_/g' -e 's/$/-latest/' | xargs -I VER sh -c 'docker pull touk/nussknacker:VER > /dev/null && echo VER || echo ""'` + echo "NUSSKNACKER_VERSION=`[ \"${NK_BASE_REF_VERSION}\" != \"\" ] && echo \"${NK_BASE_REF_VERSION}\" || echo staging-latest`" >> $GITHUB_ENV + - name: FE tests e2e on pulled image + if: ${{ env.shouldBuildImage == 'false' }} + env: + CYPRESS_SNAPSHOT_UPDATE: "true" + shell: bash + run: | + cd designer/client + ! npx wait-on -t 250 tcp:localhost:${{ env.BE_PORT }} 2> /dev/null || (echo "Port: ${BE_PORT} already in use!" && exit 1) + npx start-server-and-test backend:docker ${{ env.BE_PORT }} start-prod 3000 test:e2e + - name: Test Report + if: success() || failure() + uses: actions/upload-artifact@v4 + with: + name: CypressTests-results + path: designer/client/cypress-test-results/*.xml + - name: Store test results + if: failure() + uses: actions/upload-artifact@v4 + with: + name: e2e-test-results + path: | + docs/autoScreenshotChangeDocs/ + designer/client/cypress/**/__image_snapshots__/ + designer/client/cypress/screenshots/ + designer/client/cypress/videos/ + if-no-files-found: ignore + - name: Create Pull Request + id: update_snapshots + uses: peter-evans/create-pull-request@v7 + if: always() && (github.event_name == 'push' || github.event.pull_request.head.repo.full_name == github.repository) + env: + HASH: ${{ format('#{0}', github.event.number) }} + BRANCH: ${{ needs.setup.outputs.git_source_branch }} + with: + token: ${{ secrets.GITHUB_TOKEN }} + base: ${{ env.BRANCH }} + branch: snapshots-patch/${{ env.BRANCH }} + title: Update Cypress snapshots in ${{ env.BRANCH }} + commit-message: Updated snapshots + body: Updated snapshots in ${{ github.event_name == 'pull_request' && env.HASH || env.BRANCH}} + labels: cypress + - name: Comment PR + if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} + uses: thollander/actions-comment-pull-request@v2 + with: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + message: | + ${{ steps.update_snapshots.outputs.pull-request-operation }}: #${{ steps.update_snapshots.outputs.pull-request-number }} + :warning: Be careful! Snapshot changes are not necessarily the cause of the error. Check the logs. + comment_tag: snapshots_pr + - name: Force fail if update PR created + if: ${{ always() && github.event_name == 'pull_request' && steps.update_snapshots.outputs.pull-request-number }} + uses: actions/github-script@v6.3.3 + with: + script: | + core.setFailed('Snapshots updated, chceck comments!') + + publish-after-tests: + name: Publish With Tests + needs: ['build', 'build-fe', 'setup', 'tests', 'integrationTests', 'slowTests', 'frontendTests', 'cypressTests'] + uses: ./.github/workflows/publish.yml + with: + should_run: ${{ github.ref == 'refs/heads/master' || github.ref == 'refs/heads/demo' || startsWith(github.ref, 'refs/heads/preview') || startsWith(github.ref, 'refs/heads/release') }} + secrets: + nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} + git_source_branch: ${{ needs.setup.outputs.git_source_branch }} + sonatype_user: ${{ secrets.SONATYPE_USER }} + sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} + github__token: ${{ secrets.GITHUB_TOKEN }} + dockerhub_user: ${{ secrets.DOCKERHUB_USER }} + dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} + + publish-before-tests: + name: Publish Without Tests + needs: ['build', 'build-fe', 'setup'] + uses: ./.github/workflows/publish.yml + with: + should_run: ${{ github.ref == 'refs/heads/staging' }} + secrets: + nussknacker_version: ${{ needs.setup.outputs.nk_snapshot_version }} + git_source_branch: ${{ needs.setup.outputs.git_source_branch }} + sonatype_user: ${{ secrets.SONATYPE_USER }} + sonatype_password: ${{ secrets.SONATYPE_PASSWORD }} + github__token: ${{ secrets.GITHUB_TOKEN }} + dockerhub_user: ${{ secrets.DOCKERHUB_USER }} + dockerhub_token: ${{ secrets.DOCKERHUB_TOKEN }} + + trigger-external-build: + runs-on: ubuntu-latest + needs: [ setup, publish-before-tests ] + if: ${{ github.ref == 'refs/heads/staging' }} + strategy: + matrix: + include: + - repo: nussknacker-helm + workflow_id: helm-test-workflow.yaml + ref: main + - repo: nussknacker-quickstart + workflow_id: pr.yml + ref: staging + - repo: nussknacker-sample-components + workflow_id: pr.yml + ref: staging + - repo: nussknacker-flink-compatibility + workflow_id: test.yml + ref: staging + steps: + - uses: actions/github-script@v6 + with: + github-token: ${{ secrets.ORG_TRIGGER_PAT_TOKEN }} + script: | + await github.rest.actions.createWorkflowDispatch({ + owner: context.repo.owner, + repo: '${{ matrix.repo }}', + workflow_id: '${{ matrix.workflow_id }}', + ref: '${{ matrix.ref }}', + inputs: {"nussknacker_version": "${{ needs.setup.outputs.nk_snapshot_version }}"} + })