Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

NU-1848: Flink base types registration mechanism #7097

Merged
merged 4 commits into from
Oct 29, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -1643,7 +1643,8 @@ lazy val flinkComponentsApi = (project in flink("components-api"))
name := "nussknacker-flink-components-api",
libraryDependencies ++= {
Seq(
"org.apache.flink" % "flink-streaming-java" % flinkV % Provided,
"org.apache.flink" % "flink-streaming-java" % flinkV % Provided,
"org.scalatest" %% "scalatest" % scalaTestV % Test
)
}
)
Expand Down
4 changes: 3 additions & 1 deletion docs/Changelog.md
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,9 @@
* [#6988](https://github.com/TouK/nussknacker/pull/6988) Remove unused API classes: `MultiMap`, `TimestampedEvictableStateFunction`
* [#7000](https://github.com/TouK/nussknacker/pull/7000) Show all possible options for dictionary editor on open.
* [#6979](https://github.com/TouK/nussknacker/pull/6979) Introduces an activities panel that provides information about all system activities.
* [#7058](https://github.com/TouK/nussknacker/pull/7058) Performance optimization: Add missing Flink TypeInformation for better serialization
* Performance optimization:
* [#7058](https://github.com/TouK/nussknacker/pull/7058) Add missing Flink TypeInformation for better serialization
* [#7097](https://github.com/TouK/nussknacker/pull/7097) Flink base types registration mechanism

## 1.17

Expand Down
5 changes: 4 additions & 1 deletion docs/MigrationGuide.md
Original file line number Diff line number Diff line change
Expand Up @@ -75,10 +75,13 @@ To see the biggest differences please consult the [changelog](Changelog.md).
want to keep using Flink pre-1.19 with current Nussknacker, please refer to compatibility providing plugins in
https://github.com/TouK/nussknacker-flink-compatibility.

* [#7058](https://github.com/TouK/nussknacker/pull/7058) Performance optimization: Add missing Flink TypeInformation for better serialization
* Performance optimization:
* [#7058](https://github.com/TouK/nussknacker/pull/7058) Add missing Flink TypeInformation for better serialization
* In case of using base (bounded and unbounded) Flink components state will be probably not compatible
* `FlinkCustomNodeContext.typeInformationDetection` has been removed, please use `TypeInformationDetection.instance` instead
* `FlinkCustomNodeContext.forCustomContext` has been removed, please use `TypeInformationDetection.instance.forValueWithContext` instead
* [#7097](https://github.com/TouK/nussknacker/pull/7097) Flink base types registration mechanism
* In case of using types: java.time.LocalDate, java.time.LocalTime, java.time.LocalDateTime with CaseClassTypeInfo mechanism, state probably will be lost

### Configuration changes
* [#6979](https://github.com/TouK/nussknacker/pull/6979) Add `type: "activities-panel"` to the `processToolbarConfig` which replaces removed `{ type: "versions-panel" }` `{ type: "comments-panel" }` and `{ type: "attachments-panel" }`
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
package pl.touk.nussknacker.engine.flink.api.typeinformation

import org.apache.flink.api.common.typeinfo.{TypeInfoFactory, TypeInformation, Types}
import org.apache.flink.api.java.typeutils.TypeExtractor

import java.lang.reflect.Type
import java.time.{LocalDate, LocalDateTime, LocalTime}
import java.util

object FlinkTypeInfoRegistrar {

private case class RegistrationEntry[T, K <: TypeInfoFactory[T]](klass: Class[T], factoryClass: Class[K])

private val typesToRegister = List(
RegistrationEntry(classOf[LocalDate], classOf[LocalDateTypeInfoFactory]),
RegistrationEntry(classOf[LocalTime], classOf[LocalTimeTypeInfoFactory]),
RegistrationEntry(classOf[LocalDateTime], classOf[LocalDateTimeTypeInfoFactory]),
)

def ensureBaseTypesAreRegistered(): Unit =
typesToRegister.foreach { base =>
register(base)
}

private def register(entry: RegistrationEntry[_, _ <: TypeInfoFactory[_]]): Unit = {
val opt = Option(TypeExtractor.getTypeInfoFactory(entry.klass))
if (opt.isEmpty) {
TypeExtractor.registerFactory(entry.klass, entry.factoryClass)
}
}

class LocalDateTypeInfoFactory extends TypeInfoFactory[LocalDate] {

override def createTypeInfo(
t: Type,
genericParameters: util.Map[String, TypeInformation[_]]
): TypeInformation[LocalDate] =
Types.LOCAL_DATE

}

class LocalTimeTypeInfoFactory extends TypeInfoFactory[LocalTime] {

override def createTypeInfo(
t: Type,
genericParameters: util.Map[String, TypeInformation[_]]
): TypeInformation[LocalTime] =
Types.LOCAL_TIME

}

class LocalDateTimeTypeInfoFactory extends TypeInfoFactory[LocalDateTime] {

override def createTypeInfo(
t: Type,
genericParameters: util.Map[String, TypeInformation[_]]
): TypeInformation[LocalDateTime] =
Types.LOCAL_DATE_TIME

}

}
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,8 @@ object TypeInformationDetection {
// We use SPI to provide implementation of TypeInformationDetection because we don't want to make
// implementation classes available in flink-components-api module.
val instance: TypeInformationDetection = {
FlinkTypeInfoRegistrar.ensureBaseTypesAreRegistered()

val classloader = Thread.currentThread().getContextClassLoader
ServiceLoader
.load(classOf[TypeInformationDetection], classloader)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
package pl.touk.nussknacker.engine.flink.api.typeinformation

import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
import org.apache.flink.api.java.typeutils.GenericTypeInfo
import org.scalatest.funsuite.AnyFunSuite
import org.scalatest.matchers.should.Matchers

import java.time.{Instant, LocalDate, LocalDateTime, LocalTime}
import java.sql.{Date, Time, Timestamp}

class FlinkTypeInfoRegistrarTest extends AnyFunSuite with Matchers {

private val nuTypesMapping: Map[Class[_], TypeInformation[_]] = Map(
classOf[LocalDate] -> Types.LOCAL_DATE,
classOf[LocalTime] -> Types.LOCAL_TIME,
classOf[LocalDateTime] -> Types.LOCAL_DATE_TIME,
)

private val flinkTypesMapping: Map[Class[_], TypeInformation[_]] = Map(
classOf[String] -> Types.STRING,
classOf[Boolean] -> Types.BOOLEAN,
classOf[Byte] -> Types.BYTE,
classOf[Short] -> Types.SHORT,
classOf[Integer] -> Types.INT,
classOf[Long] -> Types.LONG,
classOf[Float] -> Types.FLOAT,
classOf[Double] -> Types.DOUBLE,
classOf[Character] -> Types.CHAR,
classOf[java.math.BigDecimal] -> Types.BIG_DEC,
classOf[java.math.BigInteger] -> Types.BIG_INT,
classOf[Instant] -> Types.INSTANT,
classOf[Date] -> Types.SQL_DATE,
classOf[Time] -> Types.SQL_TIME,
classOf[Timestamp] -> Types.SQL_TIMESTAMP,
)

test("Looking for TypeInformation for a NU types should return a GenericTypeInfo") {
nuTypesMapping.foreach { case (klass, _) =>
val typeInfo = TypeInformation.of(klass)
typeInfo shouldBe new GenericTypeInfo(klass)
}
}

test("Looking for TypeInformation for a NU types with registrar should return a specific TypeInformation") {
FlinkTypeInfoRegistrar.ensureBaseTypesAreRegistered()

nuTypesMapping.foreach { case (klass, expected) =>
val typeInfo = TypeInformation.of(klass)
typeInfo shouldBe expected
}
}

test("Make sure that the other types have specific TypeInformation") {
flinkTypesMapping.foreach { case (klass, expected) =>
val typeInfo = TypeInformation.of(klass)
typeInfo shouldBe expected
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import pl.touk.nussknacker.engine.ModelData
import pl.touk.nussknacker.engine.api.namespaces.NamingStrategy
import pl.touk.nussknacker.engine.api.{JobData, ProcessVersion}
import pl.touk.nussknacker.engine.deployment.DeploymentData
import pl.touk.nussknacker.engine.flink.api.typeinformation.FlinkTypeInfoRegistrar
import pl.touk.nussknacker.engine.flink.api.{NamespaceMetricsTags, NkGlobalParameters}
import pl.touk.nussknacker.engine.process.util.Serializers

Expand Down Expand Up @@ -100,6 +101,7 @@ object ExecutionConfigPreparer extends LazyLogging {
override def prepareExecutionConfig(
config: ExecutionConfig
)(jobData: JobData, deploymentData: DeploymentData): Unit = {
FlinkTypeInfoRegistrar.ensureBaseTypesAreRegistered()
Serializers.registerSerializers(modelData, config)
if (enableObjectReuse) {
config.enableObjectReuse()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,27 +29,6 @@ import pl.touk.nussknacker.engine.util.Implicits._
*/
class TypingResultAwareTypeInformationDetection extends TypeInformationDetection {

private val registeredTypeInfos: Map[TypedClass, TypeInformation[_]] = Map(
Typed.typedClass[String] -> Types.STRING,
Typed.typedClass[Boolean] -> Types.BOOLEAN,
Typed.typedClass[Byte] -> Types.BYTE,
Typed.typedClass[Short] -> Types.SHORT,
Typed.typedClass[Integer] -> Types.INT,
Typed.typedClass[Long] -> Types.LONG,
Typed.typedClass[Float] -> Types.FLOAT,
Typed.typedClass[Double] -> Types.DOUBLE,
Typed.typedClass[Character] -> Types.CHAR,
Typed.typedClass[java.math.BigDecimal] -> Types.BIG_DEC,
Typed.typedClass[java.math.BigInteger] -> Types.BIG_INT,
Typed.typedClass[java.time.LocalDate] -> Types.LOCAL_DATE,
Typed.typedClass[java.time.LocalTime] -> Types.LOCAL_TIME,
Typed.typedClass[java.time.LocalDateTime] -> Types.LOCAL_DATE_TIME,
Typed.typedClass[java.time.Instant] -> Types.INSTANT,
Typed.typedClass[java.sql.Date] -> Types.SQL_DATE,
Typed.typedClass[java.sql.Time] -> Types.SQL_TIME,
Typed.typedClass[java.sql.Timestamp] -> Types.SQL_TIMESTAMP,
)

def forContext(validationContext: ValidationContext): TypeInformation[Context] = {
val variables = forType(
Typed.record(validationContext.localVariables, Typed.typedClass[Map[String, AnyRef]])
Expand Down Expand Up @@ -82,8 +61,6 @@ class TypingResultAwareTypeInformationDetection extends TypeInformationDetection
// We generally don't use scala Maps in our runtime, but it is useful for some internal type infos: TODO move it somewhere else
case a: TypedObjectTypingResult if a.runtimeObjType.klass == classOf[Map[String, _]] =>
createScalaMapTypeInformation(a)
case a: SingleTypingResult if registeredTypeInfos.contains(a.runtimeObjType) =>
registeredTypeInfos(a.runtimeObjType)
// TODO: scala case classes are not handled nicely here... CaseClassTypeInfo is created only via macro, here Kryo is used
case a: SingleTypingResult if a.runtimeObjType.params.isEmpty =>
TypeInformation.of(a.runtimeObjType.klass)
Expand Down
Loading