Skip to content

Commit

Permalink
[NU-1735] Use installation example for a local development purposes (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
mk-software-pl authored Sep 17, 2024
1 parent 825d320 commit 95e6043
Show file tree
Hide file tree
Showing 23 changed files with 8,648 additions and 14 deletions.
6 changes: 3 additions & 3 deletions .run/NussknackerApp-postgres.run.xml
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@
<env name="USAGE_REPORTS_FINGERPRINT" value="development" />
<env name="USAGE_REPORTS_SOURCE" value="sources" />
<env name="INPUT_CONFIG_RESOLVE_ENV_VARIABLES" value="false" />
<env name="DB_URL" value="jdbc:postgresql://localhost:5432/world-db" />
<env name="DB_URL" value="jdbc:postgresql://localhost:5432/nu-db" />
<env name="DB_DRIVER" value="org.postgresql.Driver" />
<env name="DB_USER" value="world" />
<env name="DB_PASSWORD" value="world123" />
<env name="DB_USER" value="nu" />
<env name="DB_PASSWORD" value="nupassword" />
<env name="TABLES_DEFINITION_FILE" value="../../../nussknacker-dist/src/universal/conf/dev-tables-definition.sql" />
</envs>
<option name="INCLUDE_PROVIDED_SCOPE" value="true" />
Expand Down
15 changes: 15 additions & 0 deletions .run/RunEnvForLocalDesigner.run.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
<component name="ProjectRunConfigurationManager">
<configuration default="false" name="RunEnvForLocalDesigner" type="Application" factoryName="Application">
<option name="MAIN_CLASS_NAME" value="pl.touk.nussknacker.dev.RunEnvForLocalDesigner" />
<module name="nussknacker-designer" />
<extension name="coverage">
<pattern>
<option name="PATTERN" value="pl.touk.nussknacker.dev.*" />
<option name="ENABLED" value="true" />
</pattern>
</extension>
<method v="2">
<option name="Make" enabled="true" />
</method>
</configuration>
</component>
14 changes: 12 additions & 2 deletions CONTRIBUTING.md
Original file line number Diff line number Diff line change
Expand Up @@ -130,8 +130,18 @@ Changing the version of the Scala is done by setting `NUSSKNACKER_SCALA_VERSION`
#### Running using integration environment
- Clone [nussknacker-quickstart](https://github.com/TouK/nussknacker-quickstart/tree/old-quickstart)
- Run `docker-compose -f docker-compose-env.yml -f docker-compose-custom.yml up -d` inside it
Use one of the following method:
1. run using SBT: `sbt designer/test:"runMain pl.touk.nussknacker.dev.RunEnvForLocalDesigner"`
2. run using Intellij configuration: `RunEnvForLocalDesigner`
3. run Docker Compose: `docker compose -f examples/dev/local-testing.docker-compose.yml -f examples/dev/nu-scala213.override.yml up -d`
You can also customize the setup by adding your changes in separate yaml file:
* like this: `sbt designer/test:"runMain pl.touk.nussknacker.dev.RunEnvForLocalDesigner --customizeYaml=/tmp/my.override.yml"`
* or this: `docker compose -f examples/dev/local-testing.docker-compose.yml -f examples/dev/nu-scala213.override.yml -f /tmp/my.override.yml up -d`
By default, an environment for Scala 2.13 is prepared. To run one for Scala 2.12:
* run: `sbt designer/test:"runMain pl.touk.nussknacker.dev.RunEnvForLocalDesigner --scalaV scala212"`
* or run: `docker compose -f examples/dev/local-testing.docker-compose.yml up -d`
#### Running Designer with model classes on the same classes as designer
Expand Down
1 change: 1 addition & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -2006,6 +2006,7 @@ lazy val designer = (project in file("designer/server"))
"org.apache.flink" % "flink-metrics-dropwizard" % flinkV % Test,
"com.github.tomakehurst" % "wiremock-jre8" % wireMockV % Test,
"io.circe" %% "circe-yaml" % circeYamlV % Test,
"com.github.scopt" %% "scopt" % "4.1.0" % Test,
"org.questdb" % "questdb" % "7.4.2",
) ++ forScalaVersion(scalaVersion.value) {
case (2, 13) =>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
package pl.touk.nussknacker.dev

import cats.effect.{ExitCode, IO, IOApp}
import com.dimafeng.testcontainers.{DockerComposeContainer, WaitingForService}
import com.typesafe.scalalogging.LazyLogging
import org.testcontainers.containers.wait.strategy.ShellStrategy
import pl.touk.nussknacker.dev.RunEnvForLocalDesigner.Config.ScalaV
import scopt.{OParser, Read}

import java.io.{File => JFile}

// You can use it for a development purposes. It runs docker compose defined in `examples/dev` folder.
// After running this class you can run Nu Designer locally that can connect to the exposed services.
object RunEnvForLocalDesigner extends IOApp with LazyLogging {

override def run(args: List[String]): IO[ExitCode] = for {
config <- readConfig(args)
_ <- log(s"Starting docker compose-based stack (for ${config.scalaV}) to be used with locally run Nu Designer...")
_ <- createDockerEnv(config)
_ <- log("You can run designer now...")
_ <- IO.never[Unit]
} yield ExitCode.Success

private def readConfig(args: List[String]) = IO.delay {
OParser
.parse(Config.parser, args, Config())
.getOrElse(throw new Exception("Invalid arguments"))
}

private def createDockerEnv(config: Config) = IO.delay {
val scalaVOverrideYmlFile = config.scalaV match {
case ScalaV.Scala212 => None
case ScalaV.Scala213 => Some(new JFile("examples/dev/nu-scala213.override.yml"))
}
val env = new LocalTestingEnvDockerCompose(
dockerComposeTweakFiles = scalaVOverrideYmlFile.toList ::: config.customizeYaml.toList
)
env.start()
env
}

private def log(message: => String) = IO.delay(logger.info(message))

final case class Config(scalaV: ScalaV = ScalaV.Scala213, customizeYaml: Option[JFile] = None)

object Config {

sealed trait ScalaV

object ScalaV {
case object Scala212 extends ScalaV
case object Scala213 extends ScalaV

implicit val scalaVRead: Read[ScalaV] =
scopt.Read.reads(_.toLowerCase).map {
case "scala212" => ScalaV.Scala212
case "scala213" => ScalaV.Scala213
}

}

private val builder = OParser.builder[Config]

import builder._

lazy val parser: OParser[Unit, Config] = OParser.sequence(
head("Env for local development of Nu Designer"),
programName("sbt designer/test:runMain pl.touk.nussknacker.dev.RunEnvForLocalDesigner"),
opt[ScalaV]('s', "scalaV")
.optional()
.action((scalaV, c) => c.copy(scalaV = scalaV))
.text("Scala version. Available options: scala212, scala213"),
opt[JFile]('c', "customizeYaml")
.optional()
.valueName("<absolute file path>")
.validate { file =>
if (!file.exists()) Left(s"'$file' does NOT exist")
else if (!file.isFile) Left(s"'$file' is NOT a file")
else if (!file.canRead) Left(s"CANNOT read the file '$file'")
else Right(())
}
.action((customizeYaml, c) => c.copy(customizeYaml = Some(customizeYaml)))
.text("Yaml file for docker compose override"),
)

}

class LocalTestingEnvDockerCompose(dockerComposeTweakFiles: Iterable[JFile])
extends DockerComposeContainer(
composeFiles = new JFile("examples/dev/local-testing.docker-compose.yml") ::
dockerComposeTweakFiles.toList,
waitingFor = Some(
WaitingForService("wait-for-all", new ShellStrategy().withCommand("pwd")),
),
// Change to 'true' to enable logging
tailChildContainers = false
)

}
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,14 @@ import java.io.{File => JFile}
// Before running tests in this module, a fresh docker image should be built from sources and placed in the local
// registry. If you run tests based on this trait in Intellij Idea and the images is not built, you can do it manually:
// `bash -c "export NUSSKNACKER_SCALA_VERSION=2.12 && sbt dist/Docker/publishLocal"`
trait BaseE2eSpec extends BeforeAndAfterAll with BeforeAndAfterEach with LazyLogging {
trait BaseE2ESpec extends BeforeAndAfterAll with BeforeAndAfterEach with LazyLogging {
this: Suite =>

val client: DockerBasedInstallationExampleClient =
BaseE2eSpec.dockerBasedInstallationExampleNuEnvironmentSingleton.client
BaseE2ESpec.dockerBasedInstallationExampleNuEnvironmentSingleton.client
}

object BaseE2eSpec extends LazyLogging {
object BaseE2ESpec extends LazyLogging {

val dockerBasedInstallationExampleNuEnvironmentSingleton =
new DockerBasedInstallationExampleNuEnvironment(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ import pl.touk.nussknacker.ui.process.marshall.CanonicalProcessConverter.toScena

class BatchDataGenerationSpec
extends AnyFreeSpecLike
with BaseE2eSpec
with BaseE2ESpec
with Matchers
with VeryPatientScalaFutures
with NuRestAssureExtensions
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import org.scalatest.freespec.AnyFreeSpecLike
import org.scalatest.matchers.should.Matchers
import pl.touk.nussknacker.test.VeryPatientScalaFutures

class DetectLargeTransactionSpec extends AnyFreeSpecLike with BaseE2eSpec with Matchers with VeryPatientScalaFutures {
class DetectLargeTransactionSpec extends AnyFreeSpecLike with BaseE2ESpec with Matchers with VeryPatientScalaFutures {

"Large transactions should be properly detected" in {
val smallAmountTransactions = List(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ import org.scalatest.freespec.AnyFreeSpecLike
import org.scalatest.matchers.should.Matchers
import pl.touk.nussknacker.test.VeryPatientScalaFutures

class DetermineOfferedPlanSpec extends AnyFreeSpecLike with BaseE2eSpec with Matchers with VeryPatientScalaFutures {
class DetermineOfferedPlanSpec extends AnyFreeSpecLike with BaseE2ESpec with Matchers with VeryPatientScalaFutures {

"Properly determine offers for customers" in {
val customers = List(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ import pl.touk.nussknacker.engine.api.generics.ExpressionParseError.TabularDataD
import pl.touk.nussknacker.engine.api.parameter.ParameterName
import pl.touk.nussknacker.engine.api.process.ComponentUseCase
import pl.touk.nussknacker.engine.api.test.InvocationCollectors
import pl.touk.nussknacker.engine.api.typed.typing.{Typed, TypedObjectTypingResult, TypingResult, Unknown}
import pl.touk.nussknacker.engine.api.typed.typing.{Typed, TypingResult, Unknown}
import pl.touk.nussknacker.engine.graph.expression.TabularTypedData
import pl.touk.nussknacker.engine.graph.expression.TabularTypedData.Column

Expand Down
17 changes: 17 additions & 0 deletions examples/dev/flink/Dockerfile
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
ARG FLINK_VERSION

FROM flink:${FLINK_VERSION}

USER root
RUN echo '#!/bin/sh' > /ex-docker-entrypoint.sh && \
echo 'export FLINK_PROPERTIES=$(cat /opt/flink/conf/flink-properties.yml) && /docker-entrypoint.sh "$@"' >> /ex-docker-entrypoint.sh && \
chmod +x /ex-docker-entrypoint.sh

USER flink
COPY flink-properties.yml /opt/flink/conf/
RUN mkdir -p /opt/flink/data && \
chmod -R 777 /opt/flink/data

VOLUME /opt/flink/data

ENTRYPOINT [ "/ex-docker-entrypoint.sh" ]
28 changes: 28 additions & 0 deletions examples/dev/flink/flink-properties.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
taskmanager.numberOfTaskSlots: 8

state.backend: filesystem
state.checkpoints.dir: file:///opt/flink/data/checkpoints
state.savepoints.dir: file:///opt/flink/data/savepoints

#Below are base settings for rocksdb metrics, that can be used for grafana dashboards
state.backend.rocksdb.metrics.estimate-num-keys: true
state.backend.rocksdb.metrics.estimate-live-data-size: true
state.backend.rocksdb.metrics.cur-size-all-mem-tables: true
state.backend.rocksdb.metrics.size-all-mem-tables: true
# We can have many jobs per cluster, in such setting managed memory is not easy to tune
state.backend.rocksdb.memory.managed: false
# For frequent writes increase the value as needed. Currently RocksDB settings can only be changed per Flink cluster
state.backend.rocksdb.writebuffer.size: 256m

metrics.reporters: influxdb_reporter
metrics.reporter.influxdb_reporter.factory.class: org.apache.flink.metrics.influxdb.InfluxdbReporterFactory
metrics.reporter.influxdb_reporter.host: telegraf
metrics.reporter.influxdb_reporter.port: 8087
metrics.reporter.influxdb_reporter.db: nussknacker_metrics
metrics.reporter.influxdb_reporter.scope.variables.excludes: tm_id;job_id;task_id;task_attempt_id;operator_id;task_attempt_num;task_name
metrics.scope.jm: local.<host>.jobmanagerGlobal
metrics.scope.jm.job: local.<host>.jobmanagerJob.<job_name>
metrics.scope.tm: local.<host>.taskmanagerGlobal.<tm_id>
metrics.scope.tm.job: local.<host>.taskmanagerJob.<tm_id>.<job_name>
metrics.scope.task: local.<host>.taskmanagerTask.<tm_id>.<job_name>.<task_name>.<subtask_index>
metrics.scope.operator: local.<host>.taskmanagerTask.<tm_id>.<job_name>.<operator_name>.<subtask_index>
13 changes: 13 additions & 0 deletions examples/dev/grafana/dashboards/dashboards.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
apiVersion: 1
providers:
- name: default
orgId: 1
folder: ''
folderUid: ''
type: file
disableDeletion: true
editable: true
updateIntervalSeconds: 10
allowUiUpdates: true
options:
path: /var/lib/grafana/dashboards
Loading

0 comments on commit 95e6043

Please sign in to comment.