Skip to content

Commit

Permalink
Scala 2.12 build (#99)
Browse files Browse the repository at this point in the history
Scala 2.11 and 2.12 cross build
  • Loading branch information
mproch authored and arkadius committed Nov 18, 2019
1 parent 16e8174 commit e303e78
Show file tree
Hide file tree
Showing 38 changed files with 236 additions and 169 deletions.
25 changes: 11 additions & 14 deletions .travis.yml
@@ -1,8 +1,5 @@
language: scala

scala:
- 2.11.12

jdk:
- oraclejdk8

Expand Down Expand Up @@ -51,7 +48,7 @@ cache:
- $HOME/.npm

stages:
- tests
- test
- name: releases
# We don't allow do releases from fork and corn type because of security: https://docs.travis-ci.com/user/pull-requests/#pull-requests-and-security-restrictions
# We also don't allow for pull request builds because they are built from merge of source and upstream branches.
Expand All @@ -60,23 +57,23 @@ stages:

jobs:
include:
- stage: tests
- stage: test
name: "Backend tests"
script: ./ciBackendTest.sh $TRAVIS_SCALA_VERSION
after_success: if [ "$COVERAGE" = true ]; then sbt ++$TRAVIS_SCALA_VERSION coveralls; else echo "Skipping coveralls"; fi
- name: "Frontend tests"
before_script: if [[ `npm -v | sed 's/\..*//'` < 6 ]]; then npm i -g npm; fi
script: ./ciFrontendTest.sh
script: ./ciBackendTest.sh
after_success: if [ "$COVERAGE" = true ]; then sbt + coveralls; else echo "Skipping coveralls"; fi
- name: "UI slow tests"
script: sbt ++$TRAVIS_SCALA_VERSION clean ui/slow:test
script: sbt +clean +ui/slow:test
- name: "Management tests"
script: sbt ++$TRAVIS_SCALA_VERSION clean management/it:test
script: sbt +clean +management/it:test
- name: "EngineStandalone tests"
script: sbt ++$TRAVIS_SCALA_VERSION clean engineStandalone/it:test
script: sbt +clean +engineStandalone/it:test
- name: "Frontend tests"
before_script: if [[ `npm -v | sed 's/\..*//'` < 6 ]]; then npm i -g npm; fi
script: ./ciFrontendTest.sh
- stage: releases
name: "Release sonatype packages"
before_script: if [[ `npm -v | sed 's/\..*//'` < 6 ]]; then npm i -g npm; fi
script: ./ciBuild.sh "${BUILD_VERSION}-SNAPSHOT" ${NEXUS_PASSWORD} ${NEXUS_URL} ${NEXUS_USER} true
script: ./ciPublish.sh "${BUILD_VERSION}-SNAPSHOT" ${NEXUS_PASSWORD} ${NEXUS_URL} ${NEXUS_USER}
- name: "Push Docker Image"
before_script: if [[ `npm -v | sed 's/\..*//'` < 6 ]]; then npm i -g npm; fi
script:
Expand Down
9 changes: 5 additions & 4 deletions README.md
Expand Up @@ -17,9 +17,10 @@ Talk to us on our [mailing list](https://groups.google.com/forum/#!forum/nusskna

**Nussknacker** is published under [Apache License 2.0](http://www.apache.org/licenses/LICENSE-2.0).


## Scala compatibility

Currently we only support scala 2.11.x. The main reason we don't support scala < 2.11 and scala 2.12 is
[this jira](https://issues.apache.org/jira/browse/FLINK-5005) - we rely heavily on Flink, and making it support scala 2.12
is suprisingly hard (due to changes in implementation of lambdas).
Currently we do support Scala 2.11 and 2.12, we cross publish versions.

## Flink compatibility

We currently support only one Flink version (please see flinkV in build.sbt)
39 changes: 26 additions & 13 deletions build.sbt
Expand Up @@ -5,7 +5,9 @@ import sbt._
import sbtassembly.AssemblyPlugin.autoImport.assembly
import sbtassembly.MergeStrategy

val scalaV = "2.11.12"
val scala211 = "2.11.12"
val scala212 = "2.12.10"
lazy val supportedScalaVersions = List(scala212, scala211)

//by default we include flink and scala, we want to be able to disable this behaviour for performance reasons
val includeFlinkAndScala = Option(System.getProperty("includeFlinkAndScala", "true")).exists(_.toBoolean)
Expand All @@ -26,7 +28,8 @@ val dockerUpLatest = System.getProperty("dockerUpLatest", "true").toBoolean
// unfortunately it does not work, so we resort to hack by publishing root module to Resolver.defaultLocal
//publishArtifact := false
publishTo := Some(Resolver.defaultLocal)

crossScalaVersions := Nil

val publishSettings = Seq(
publishMavenStyle := true,
releasePublishArtifactsAction := PgpKeys.publishSigned.value,
Expand Down Expand Up @@ -81,7 +84,8 @@ val commonSettings =
Seq(
test in assembly := {},
licenses += ("Apache-2.0", url("https://www.apache.org/licenses/LICENSE-2.0.html")),
scalaVersion := scalaV,
crossScalaVersions := supportedScalaVersions,
scalaVersion := scala212,
resolvers ++= Seq(
"confluent" at "https://packages.confluent.io/maven"
),
Expand All @@ -102,11 +106,15 @@ val commonSettings =
),
javacOptions := Seq(
"-Xlint:deprecation",
"-Xlint:unchecked"
"-Xlint:unchecked",
//we use it e.g. to provide consistent behaviour wrt extracting parameter names from scala and java
"-parameters"
),
assemblyMergeStrategy in assembly := nussknackerMergeStrategy,
coverageMinimum := 60,
coverageFailOnMinimum := false
coverageFailOnMinimum := false,
//problem with scaladoc of api: https://github.com/scala/bug/issues/10134
scalacOptions in (Compile, doc) -= "-Xfatal-warnings"
)

val akkaV = "2.4.20" //same version as in Flink
Expand Down Expand Up @@ -156,7 +164,7 @@ lazy val dockerSettings = {
dockerLabels := Map(
"tag" -> dockerTagName.getOrElse(version.value),
"version" -> version.value,
"scala" -> scalaV,
"scala" -> scalaVersion.value,
"flink" -> flinkV
),
version in Docker := dockerTagName.getOrElse(version.value)
Expand All @@ -172,12 +180,13 @@ lazy val dist = (project in file("nussknacker-dist"))
(assembly in Compile) in generic,
(assembly in Compile) in example
).value,

mappings in Universal += {
val genericModel = generic.base / "target" / "scala-2.11" / "genericModel.jar"
val genericModel = (crossTarget in generic).value / "genericModel.jar"
genericModel -> "model/genericModel.jar"
},
mappings in Universal += {
val exampleModel = example.base / "target" / "scala-2.11" / s"nussknacker-example-assembly-${version.value}.jar"
val exampleModel = (crossTarget in example).value / s"nussknacker-example-assembly-${version.value}.jar"
exampleModel -> "model/exampleModel.jar"
},
publishArtifact := false,
Expand Down Expand Up @@ -355,6 +364,7 @@ lazy val generic = (project in engine("flink/generic")).
)
},
test in assembly := {},

assemblyJarName in assembly := "genericModel.jar",
artifact in (Compile, assembly) := {
val art = (artifact in (Compile, assembly)).value
Expand Down Expand Up @@ -484,7 +494,7 @@ lazy val util = (project in engine("util")).
Seq(
"com.iheart" %% "ficus" % ficusV,
"org.scalatest" %% "scalatest" % scalaTestV % "test",
"io.circe" %% "circe-java8" % "0.11.1"
"io.circe" %% "circe-java8" % circeV
)
}
).dependsOn(api)
Expand Down Expand Up @@ -618,6 +628,9 @@ lazy val httpUtils = (project in engine("httpUtils")).
libraryDependencies ++= {
val sttpV = "2.0.0-M6"
Seq(
//we force circe version here, because sttp has 0.12.1 for scala 2.12, we don't want it ATM
"io.circe" %% "circe-core" % circeV force(),
"io.circe" %% "circe-parser" % circeV force(),
"org.dispatchhttp" %% "dispatch-core" % dispatchV,
"org.asynchttpclient" % "async-http-client" % "2.10.4",
"org.scala-lang.modules" %% "scala-parser-combinators" % scalaParsersV, // scalaxb deps
Expand Down Expand Up @@ -649,11 +662,11 @@ lazy val queryableState = (project in engine("queryableState")).

lazy val buildUi = taskKey[Unit]("builds ui")

def runNpm(command: String, errorMessage: String): Unit = {
def runNpm(command: String, errorMessage: String, outputPath: File): Unit = {
import sys.process.Process
val path = Path.apply("ui/client").asFile
println("Using path: " + path.getAbsolutePath)
val result = Process(s"npm $command", path)!;
val result = Process(s"npm $command", path, "OUTPUT_PATH" -> outputPath.absolutePath)!;
if (result != 0) throw new RuntimeException(errorMessage)
}

Expand All @@ -674,8 +687,8 @@ lazy val ui = (project in file("ui/server"))
.settings(commonSettings)
.settings(
name := "nussknacker-ui",
buildUi := {
runNpm("run build", "Client build failed")
buildUi := {
runNpm("run build", "Client build failed", (crossTarget in compile).value)
},
parallelExecution in ThisBuild := false,
assemblyOption in assembly := (assemblyOption in assembly).value.copy(includeScala = includeFlinkAndScala, level = Level.Info),
Expand Down
14 changes: 10 additions & 4 deletions ciBackendTest.sh
@@ -1,12 +1,18 @@
#!/usr/bin/env bash

scala_version=$1
scala_version=""
scala_cross_build="+"

if ! [[ -z "$1" ]]; then
scala_version="++$1"
scala_cross_build=""
fi

set -e

if [ "$COVERAGE" = true ]; then
sbt ++$scala_version clean coverage test coverageReport
sbt ++$scala_version coverageAggregate
sbt $scala_version ${scala_cross_build}clean ${scala_cross_build}coverage ${scala_cross_build}test ${scala_cross_build}coverageReport
sbt $scala_version ${scala_cross_build}coverageAggregate
else
sbt ++$scala_version clean test
sbt $scala_version ${scala_cross_build}clean ${scala_cross_build}test
fi
27 changes: 0 additions & 27 deletions ciBuild.sh

This file was deleted.

21 changes: 21 additions & 0 deletions ciPublish.sh
@@ -0,0 +1,21 @@
#!/usr/bin/env bash

set -e

espEngineToukVersion=$1
nexusPassword=$2

if [[ -n "$3" ]]; then
nexusUrlProperty="-DnexusUrl=$3"
else
nexusUrlProperty=""
fi

if [[ -n "$4" ]]; then
nexusUserProperty="-DnexusUser=$4"
else
nexusUserProperty=""
fi

cd ui/client && npm ci && cd -
./sbtwrapper -DnexusPassword=$2 ${nexusUrlProperty} ${nexusUserProperty} "set version in ThisBuild := \"$espEngineToukVersion\"" +publish
Expand Up @@ -293,7 +293,7 @@ case object CustomFilter extends CustomStreamTransformer {
= FlinkCustomStreamTransformation((start: DataStream[Context], ctx: FlinkCustomNodeContext) =>
start
.filter(ctx.lazyParameterHelper.lazyFilterFunction(expression))
.map(ValueWithContext(null, _)))
.map(ValueWithContext[Any](null, _)))

}

Expand All @@ -309,7 +309,7 @@ object AdditionalVariableTransformer extends CustomStreamTransformer {
@MethodToInvoke(returnType = classOf[Void])
def execute(@AdditionalVariables(Array(new AdditionalVariable(name = "additional", clazz = classOf[String]))) @ParamName("expression") expression: LazyParameter[Boolean])
= FlinkCustomStreamTransformation((start: DataStream[Context]) =>
start.map(ValueWithContext("", _)))
start.map(ValueWithContext[Any]("", _)))

}

Expand Down
7 changes: 0 additions & 7 deletions engine/flink/management/src/it/resources/application.conf
Expand Up @@ -3,9 +3,6 @@ flinkConfig {
restUrl: "localhost:8081"
queryableStateProxyUrl: "localhost:9069"
jobManagerTimeout: 1m
classpath: [
"./engine/flink/management/sample/target/scala-2.11/managementSample.jar"
]
}

processConfig {
Expand All @@ -15,10 +12,6 @@ processConfig {
bufferSize: 200
parallelismMultiplier: 2
}
#this will be overwritten for docker tests
kafka = {
kafkaAddress = "kafka:9092"
}

signals {
topic: "esp.signals"
Expand Down
3 changes: 1 addition & 2 deletions engine/flink/management/src/it/resources/docker/Dockerfile
@@ -1,4 +1,4 @@
FROM flink:1.7.2-scala_2.11
FROM flink:1.7.2-scala_${scala.major.version}

COPY entrypointWithIP.sh /
COPY docker-entrypoint.sh /
Expand All @@ -16,7 +16,6 @@ RUN chmod +x /docker-entrypoint.sh

USER flink
RUN mkdir -p /tmp/storage
RUN mkdir -p /tmp/storage/1.4/blob/cache

USER root
ENTRYPOINT ["/entrypointWithIP.sh"]
@@ -1,31 +1,34 @@
package pl.touk.nussknacker.engine.management

import java.io.File
import java.nio.file.Files
import java.nio.file.{Files, Path}
import java.nio.file.attribute.{PosixFilePermission, PosixFilePermissions}
import java.util.Collections

import com.spotify.docker.client.{DefaultDockerClient, DockerClient}
import com.typesafe.config.ConfigValueFactory.fromAnyRef
import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory}
import com.typesafe.scalalogging.LazyLogging
import com.whisk.docker.impl.spotify.SpotifyDockerFactory
import com.whisk.docker.scalatest.DockerTestKit
import com.whisk.docker.{ContainerLink, DockerContainer, DockerFactory, DockerReadyChecker, LogLineReceiver, VolumeMapping}
import org.apache.commons.io.FileUtils
import org.apache.commons.io.{FileUtils, IOUtils}
import org.scalatest.Suite
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.time.{Millis, Seconds, Span}
import pl.touk.nussknacker.engine.kafka.KafkaClient
import pl.touk.nussknacker.engine.util.config.ScalaMajorVersionConfig

import scala.concurrent.duration._

trait DockerTest extends DockerTestKit with ScalaFutures with LazyLogging {
self: Suite =>

private val flinkEsp = "flinkesp:1.7.2"
private val flinkEsp = s"flinkesp:1.7.2-scala_${ScalaMajorVersionConfig.scalaMajorVersion}"

private val client: DockerClient = DefaultDockerClient.fromEnv().build()

protected var kafkaClient: KafkaClient = _

override implicit val patienceConfig: PatienceConfig = PatienceConfig(
timeout = Span(90, Seconds),
interval = Span(1, Millis)
Expand All @@ -39,7 +42,9 @@ trait DockerTest extends DockerTestKit with ScalaFutures with LazyLogging {
val dirFile = dir.toFile

List("Dockerfile", "entrypointWithIP.sh", "conf.yml", "docker-entrypoint.sh").foreach { file =>
FileUtils.copyInputStreamToFile(getClass.getResourceAsStream(s"/docker/$file"), new File(dirFile, file))
val resource = IOUtils.toString(getClass.getResourceAsStream(s"/docker/$file"))
val withVersionReplaced = resource.replace("${scala.major.version}", ScalaMajorVersionConfig.scalaMajorVersion)
FileUtils.writeStringToFile(new File(dirFile, file), withVersionReplaced)
}

client.build(dir, flinkEsp)
Expand Down Expand Up @@ -92,4 +97,15 @@ trait DockerTest extends DockerTestKit with ScalaFutures with LazyLogging {
Files.createTempDirectory("dockerTest",
PosixFilePermissions.asFileAttribute(PosixFilePermission.values().toSet[PosixFilePermission].asJava))
}

def config: Config = ConfigFactory.load()
.withValue("flinkConfig.restUrl", fromAnyRef(s"http://${jobManagerContainer.getIpAddresses().futureValue.head}:$FlinkJobManagerRestPort"))
.withValue("flinkConfig.classpath", ConfigValueFactory.fromIterable(Collections.singletonList(classPath)))
.withFallback(additionalConfig)


protected def classPath: String

protected def additionalConfig: Config = ConfigFactory.empty()

}

0 comments on commit e303e78

Please sign in to comment.