diff --git a/README.md b/README.md index a32f604..8970feb 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ ## Maven Dependency -`Flinkrunner 3` is [available on maven central](https://mvnrepository.com/artifact/io.epiphanous/flinkrunner_2.12), +`Flinkrunner 4` is [available on maven central](https://mvnrepository.com/artifact/io.epiphanous/flinkrunner_2.12), built against Flink 1.13 with Scala 2.12 and JDK 11. ```sbtshell diff --git a/build.sbt b/build.sbt index 7f9488f..c565166 100644 --- a/build.sbt +++ b/build.sbt @@ -24,10 +24,11 @@ inThisBuild( Test / parallelExecution := false Test / fork := true resolvers += "Local Maven Repository" at "file://" + Path.userHome.absolutePath + "/.m2/repository" +resolvers += "Confluent Repository" at "https://packages.confluent.io/maven/" val V = new { - val flink = "1.13.2" - val logback = "1.2.6" + val flink = "1.14.0" + val logback = "1.2.7" val scalaLogging = "3.9.4" val scalaTest = "3.2.10" val scalaCheck = "1.15.4" @@ -35,53 +36,58 @@ val V = new { val http4s = "0.21.29" val enumeratum = "1.7.0" val typesafeConfig = "1.4.1" - val guava = "29.0-jre" + val guava = "31.0.1-jre" //"29.0-jre" val squants = "1.8.3" - val avro = "1.10.2" + val avro = "1.11.0" val avro4s = "4.0.11" + val schemaRegistry = "7.0.0" } -val flinkDeps = - Seq("scala", "streaming-scala", "cep-scala").map(a => - "org.apache.flink" %% s"flink-$a" % V.flink % Provided - ) ++ - Seq( - "connector-kafka", - "connector-kinesis", - "connector-cassandra", - "connector-elasticsearch7", - "statebackend-rocksdb" - ).map(a => "org.apache.flink" %% s"flink-$a" % V.flink) ++ - Seq("org.apache.flink" %% "flink-test-utils" % V.flink % Test) +val flinkDeps = + Seq( + "org.apache.flink" %% s"flink-scala" % V.flink % Provided, + "org.apache.flink" %% s"flink-streaming-scala" % V.flink % Provided, + "org.apache.flink" %% s"flink-cep-scala" % V.flink % Provided, + "org.apache.flink" %% s"flink-connector-kafka" % V.flink, + "org.apache.flink" %% s"flink-connector-kinesis" % V.flink, + "org.apache.flink" %% s"flink-connector-cassandra" % V.flink, + "org.apache.flink" %% s"flink-connector-elasticsearch7" % V.flink, + "org.apache.flink" %% s"flink-statebackend-rocksdb" % V.flink, + "org.apache.flink" % s"flink-avro-confluent-registry" % V.flink, + "org.apache.flink" %% s"flink-test-utils" % V.flink % Test + ) val loggingDeps = Seq( "ch.qos.logback" % "logback-classic" % V.logback % Provided, "com.typesafe.scala-logging" %% "scala-logging" % V.scalaLogging ) -val http4sDeps = - Seq("http4s-dsl", "http4s-client", "http4s-blaze-client", "http4s-circe") - .map("org.http4s" %% _ % V.http4s) +val http4sDeps = Seq( + "dsl", + "client", + "blaze-client", + "circe" +).map(d => "org.http4s" %% s"http4s-$d" % V.http4s) val circeDeps = Seq( - "circe-core", - "circe-generic", - "circe-generic-extras", - "circe-parser" -).map( - "io.circe" %% _ % V.circe -) + "core", + "generic", + "generic-extras", + "parser" +).map(d => "io.circe" %% s"circe-$d" % V.circe) -val otherDeps = Seq( - "com.beachape" %% "enumeratum" % V.enumeratum, - "org.apache.avro" % "avro" % V.avro, - "com.typesafe" % "config" % V.typesafeConfig, - "com.google.guava" % "guava" % V.guava, - "org.typelevel" %% "squants" % V.squants, - "com.sksamuel.avro4s" %% "avro4s-core" % V.avro4s, - "org.scalactic" %% "scalactic" % V.scalaTest % Test, - "org.scalatest" %% "scalatest" % V.scalaTest % Test, - "org.scalacheck" %% "scalacheck" % V.scalaCheck % Test +val otherDeps = Seq( +// "io.confluent" % "kafka-schema-registry-client" % V.schemaRegistry, + "io.confluent" % "kafka-streams-avro-serde" % "7.0.0", + "com.beachape" %% "enumeratum" % V.enumeratum, +// "org.apache.avro" % "avro" % V.avro, + "com.typesafe" % "config" % V.typesafeConfig, + "com.google.guava" % "guava" % V.guava, + "org.typelevel" %% "squants" % V.squants, + "com.sksamuel.avro4s" %% "avro4s-core" % V.avro4s, + "org.scalactic" %% "scalactic" % V.scalaTest % Test, + "org.scalatest" %% "scalatest" % V.scalaTest % Test, + "org.scalacheck" %% "scalacheck" % V.scalaCheck % Test ) /** diff --git a/project/build.properties b/project/build.properties index dbae93b..10fd9ee 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=1.4.9 +sbt.version=1.5.5 diff --git a/project/plugins.sbt b/project/plugins.sbt index 8d353e1..42b6ed2 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,3 +1,3 @@ -addSbtPlugin("com.geirsson" % "sbt-ci-release" % "1.5.5") -addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.6.1") -addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.2.7") +addSbtPlugin("com.github.sbt" % "sbt-ci-release" % "1.5.10") +addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.9.2") +addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.3.1") diff --git a/src/main/resources/flink-runner.conf b/src/main/resources/flink-runner.conf index f4441cf..aeb382a 100644 --- a/src/main/resources/flink-runner.conf +++ b/src/main/resources/flink-runner.conf @@ -28,4 +28,5 @@ state { backend = rocksdb } max.lateness = 5m -jobs = {} \ No newline at end of file +watermark.strategy = "bounded lateness" +jobs = {} diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index 010d804..1192bdb 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -1,7 +1,53 @@ package io.epiphanous.flinkrunner import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} +import io.epiphanous.flinkrunner.model._ +import io.epiphanous.flinkrunner.operator.AddToJdbcBatchFunction +import io.epiphanous.flinkrunner.util.{ + BoundedLatenessWatermarkStrategy, + JdbcSink +} +import org.apache.flink.api.common.eventtime.WatermarkStrategy +import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.serialization.{ + DeserializationSchema, + Encoder, + SerializationSchema +} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.connector.kafka.sink.KafkaSink +import org.apache.flink.connector.kafka.source.KafkaSource +import org.apache.flink.core.fs.Path +import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.{ + BasePathBucketAssigner, + DateTimeBucketAssigner +} +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.{ + DefaultRollingPolicy, + OnCheckpointRollingPolicy +} +import org.apache.flink.streaming.api.functions.sink.filesystem.{ + BucketAssigner, + StreamingFileSink +} +import org.apache.flink.streaming.api.scala.{DataStream, _} +import org.apache.flink.streaming.connectors.cassandra.CassandraSink +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema +import org.apache.flink.streaming.connectors.kinesis.{ + FlinkKinesisConsumer, + FlinkKinesisProducer +} +import org.apache.http.HttpHost +import org.elasticsearch.client.Requests + +import java.io.{File, FileNotFoundException} +import java.net.URL +import java.nio.charset.StandardCharsets +import scala.collection.JavaConverters._ +import scala.util.matching.Regex /** * Flink Job Invoker @@ -13,29 +59,14 @@ class FlinkRunner[ADT <: FlinkEvent]( optConfig: Option[String] = None) extends LazyLogging { - implicit val config: FlinkConfig = - new FlinkConfig(args, factory, sources, optConfig) - implicit val env: SEE = config.configureStreamExecutionEnvironment + val config: FlinkConfig[ADT] = + factory.getFlinkConfig(args, sources, optConfig) + val env: SEE = config.configureStreamExecutionEnvironment /** - * An intermediate method to process main args, with optional callback to - * capture output of flink job. - * - * @param callback - * a function from an iterator to unit - */ - def process( - callback: PartialFunction[Stream[ADT], Unit] = { case _ => - () - } - ): Unit = - if (config.jobName == "help") showHelp() - else process1(callback) - - /** - * Actually invoke the job based on the job name and arguments passed in. - * If the job run returns an iterator of results, pass those results to - * the callback. Otherwise, just return. The callback is for testing the + * Invoke a job based on the job name and arguments passed in. If the job + * run returns an iterator of results, pass those results to the + * callback. Otherwise, just return. The callback is for testing the * stream of results from a flink job. It will only be invoked if * --mock.edges option is on. * @@ -43,18 +74,19 @@ class FlinkRunner[ADT <: FlinkEvent]( * a function from a stream to unit that receives results from running * flink job */ - def process1( - callback: PartialFunction[Stream[ADT], Unit] = { case _ => + def process( + callback: PartialFunction[List[_], Unit] = { case _ => () } ): Unit = { - if ( + if (config.jobName == "help") showHelp() + else if ( config.jobArgs.headOption .exists(s => List("help", "--help", "-help", "-h").contains(s)) ) showJobHelp() else { - factory.getJobInstance(config.jobName, config).run match { - case Left(results) => callback(results.toStream) + factory.getJobInstance(config.jobName, this).run() match { + case Left(results) => callback(results) case Right(_) => () } } @@ -103,4 +135,592 @@ class FlinkRunner[ADT <: FlinkEvent]( println(usage) } + val RESOURCE_PATTERN: Regex = "resource://(.*)".r + + /** + * Generates a timestamp and watermark assigner for a stream with a given + * type of element that limits how late an element is allowed to arrive + * in event time. + * + * @tparam E + * the type of stream element + * @return + * BoundedLatenessGenerator[E] + */ + def boundedLatenessWatermarks[E <: ADT: TypeInformation]( + streamID: String + ) = + new BoundedLatenessWatermarkStrategy[E]( + config.maxLateness, + streamID + ) + + /** + * Create a bounded of order watermark strategy with idleness checking + * + * @tparam E + * the type of stream element + * @return + * BoundedLatenessGenerator[E] + */ + def boundedOutOfOrderWatermarks[E <: ADT: TypeInformation]() + : WatermarkStrategy[E] = + WatermarkStrategy + .forBoundedOutOfOrderness(config.maxLateness) + .withIdleness(config.maxIdleness) + + /** + * Creates an ascending timestamp watermark strategy. + * @tparam E + * type of stream element + * @return + * AscendingTimestampExtractor[E] + */ + def ascendingTimestampsWatermarks[E <: ADT: TypeInformation]() + : WatermarkStrategy[E] = WatermarkStrategy.forMonotonousTimestamps() + + /** + * Assign timestamps/watermarks if we're using event time + * @param in + * the input stream to watermark + * @tparam E + * event type + * @return + * the possibly watermarked input stream + */ + def maybeAssignTimestampsAndWatermarks[E <: ADT: TypeInformation]( + in: DataStream[E], + srcConfig: SourceConfig + ): DataStream[E] = + in.assignTimestampsAndWatermarks(srcConfig.watermarkStrategy match { + case "bounded out of order" => + boundedOutOfOrderWatermarks() + case "ascending timestamps" => ascendingTimestampsWatermarks() + case _ => boundedLatenessWatermarks(in.name) + }).name(s"wm:${in.name}") + .uid(s"wm:${in.name}") + + /** + * Configure stream source from configuration. + * + * @param sourceName + * the name of the source to get its configuration + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromSource[E <: ADT: TypeInformation]( + sourceName: String = "" + ): DataStream[E] = { + val name = + if (sourceName.isEmpty) config.getSourceNames.head else sourceName + val src = config.getSourceConfig(name) + val uid = src.label + val stream = (src match { + case src: KafkaSourceConfig => fromKafka(src) + case src: KinesisSourceConfig => fromKinesis(src) + case src: FileSourceConfig => fromFile(src) + case src: SocketSourceConfig => fromSocket(src) + case src: CollectionSourceConfig => fromCollection(src) + }).name(uid).uid(uid) + maybeAssignTimestampsAndWatermarks(stream, src) + } + + /** + * Configure stream from kafka source. + * + * @param srcConfig + * a source config + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromKafka[E <: ADT: TypeInformation]( + srcConfig: KafkaSourceConfig + ): DataStream[E] = + env + .fromSource( + KafkaSource + .builder[E]() + .setProperties(srcConfig.properties) + .setDeserializer( + config + .getKafkaRecordDeserializationSchema[E]( + srcConfig.name + ) + ) + .build(), + srcConfig.watermarkStrategy match { + case "bounded out of order" => + boundedOutOfOrderWatermarks[E]() + case "ascending timestamps" => ascendingTimestampsWatermarks[E]() + case _ => boundedLatenessWatermarks[E](srcConfig.name) + }, + srcConfig.label + ) + + /** + * Configure stream from kinesis. + * + * @param srcConfig + * a source config + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromKinesis[E <: ADT: TypeInformation]( + srcConfig: KinesisSourceConfig + ): DataStream[E] = { + val consumer = + new FlinkKinesisConsumer[E]( + srcConfig.stream, + config + .getKinesisDeserializationSchema[E](srcConfig.name), + srcConfig.properties + ) + env + .addSource(consumer) + } + + /** + * Configure stream from file source. + * + * @param srcConfig + * a source config + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromFile[E <: ADT: TypeInformation]( + srcConfig: FileSourceConfig + ): DataStream[E] = { + val path = srcConfig.path match { + case RESOURCE_PATTERN(p) => getSourceFilePath(p) + case other => other + } + val ds = config + .getDeserializationSchema[E](srcConfig.name) + env + .readTextFile(path) + .name(s"raw:${srcConfig.label}") + .uid(s"raw:${srcConfig.label}") + .map(line => ds.deserialize(line.getBytes(StandardCharsets.UTF_8))) + } + + /** + * Configure stream from socket source. + * + * @param srcConfig + * a source config + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromSocket[E <: ADT: TypeInformation]( + srcConfig: SocketSourceConfig + ): DataStream[E] = + env + .socketTextStream(srcConfig.host, srcConfig.port) + .name(s"raw:${srcConfig.label}") + .uid(s"raw:${srcConfig.label}") + .map(line => + config + .getDeserializationSchema(srcConfig.name) + .asInstanceOf[DeserializationSchema[E]] + .deserialize(line.getBytes(StandardCharsets.UTF_8)) + ) + + /** + * Configure stream from collection source. + * + * @param srcConfig + * a source config + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def fromCollection[E <: ADT: TypeInformation]( + srcConfig: CollectionSourceConfig + ): DataStream[E] = + env + .fromCollection[Array[Byte]]( + config.getCollectionSource(srcConfig.topic) + ) + .name(s"raw:${srcConfig.label}") + .uid(s"raw:${srcConfig.label}") + .map(bytes => + config + .getDeserializationSchema(srcConfig.name) + .asInstanceOf[DeserializationSchema[E]] + .deserialize(bytes) + ) + + /** + * Returns the actual path to a resource file named filename or + * filename.gz. + * + * @param filename + * the name of file + * @return + * String + */ + @throws[FileNotFoundException] + def getSourceFilePath(filename: String): String = { + val loader = getClass + val resource = Option(loader.getResource(filename)) match { + case Some(value) => value.toURI + case None => + Option(loader.getResource(s"$filename.gz")) match { + case Some(value) => value.toURI + case None => + throw new FileNotFoundException( + s"can't load resource $filename" + ) + } + } + val file = new File(resource) + file.getAbsolutePath + } + + val runner: FlinkRunner[ADT] = this + + implicit class EventStreamOps[E <: ADT: TypeInformation]( + stream: DataStream[E]) { + + def as[T <: ADT: TypeInformation]: DataStream[T] = { + val name = stream.name + stream + .filter((e: E) => e.isInstanceOf[T @unchecked]) + .name(s"filter types $name") + .uid(s"filter types $name") + .map((e: E) => e.asInstanceOf[T @unchecked]) + .name(s"cast types $name") + .uid(s"cast types $name") + } + + def toSink(sinkName: String = ""): Object = + runner.toSink[E](stream, sinkName) + + } + + /** + * Configure stream sink from configuration. + * + * @param stream + * the data stream to send to sink + * @param sinkName + * a sink name to obtain configuration + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def toSink[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkName: String = "" + ): Object = { + val name = if (sinkName.isEmpty) config.getSinkNames.head else sinkName + val sinkConfig = config.getSinkConfig(name) + val label = sinkConfig.label + sinkConfig match { + case s: KafkaSinkConfig => + toKafka[E](stream, s).uid(label).name(label) + case s: KinesisSinkConfig => + toKinesis[E](stream, s).uid(label).name(label) + case s: FileSinkConfig => toFile[E](stream, s).uid(label).name(label) + case s: SocketSinkConfig => + toSocket[E](stream, s).uid(label).name(label) + case s: JdbcSinkConfig => toJdbc[E](stream, s).uid(label).name(label) + case s: CassandraSinkConfig => + toCassandraSink[E](stream, s).uid(label).name(label) + case s: ElasticsearchSinkConfig => + toElasticsearchSink[E](stream, s).uid(label).name(label) + case s => + throw new IllegalArgumentException( + s"unsupported source connector: ${s.connector}" + ) + } + } + + /** + * Send stream to a kafka sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toKafka[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: KafkaSinkConfig + ): DataStreamSink[E] = + stream + .sinkTo( + KafkaSink + .builder() + .setKafkaProducerConfig(sinkConfig.properties) + .setRecordSerializer( + config + .getKafkaRecordSerializationSchema[E]( + sinkConfig.name + ) + ) + .build() + ) + + /** + * Send stream to a kinesis sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toKinesis[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: KinesisSinkConfig + ): DataStreamSink[E] = + stream + .addSink { + val sink = + new FlinkKinesisProducer[E]( + config + .getKinesisSerializationSchema(sinkConfig.name) + .asInstanceOf[KinesisSerializationSchema[E]], + sinkConfig.properties + ) + sink.setDefaultStream(sinkConfig.stream) + sink.setFailOnError(true) + sink.setDefaultPartition("0") + sink + } + + /** + * Send stream to a socket sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toJdbc[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: JdbcSinkConfig + ): DataStreamSink[E] = + stream + .addSink( + new JdbcSink[E]( + sinkConfig, + config + .getAddToJdbcBatchFunction(sinkConfig.name) + .asInstanceOf[AddToJdbcBatchFunction[E]] + ) + ) + + /** + * Send stream to a rolling file sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toFile[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: FileSinkConfig + ): DataStreamSink[E] = { + val path = sinkConfig.path + val p = sinkConfig.properties + val bucketCheckInterval = + p.getProperty("bucket.check.interval", s"${60000}").toLong + val bucketAssigner = + p.getProperty("bucket.assigner.type", "datetime") match { + case "none" => new BasePathBucketAssigner[E]() + case "datetime" => + new DateTimeBucketAssigner[E]( + p.getProperty( + "bucket.assigner.datetime.format", + "YYYY/MM/DD/HH" + ) + ) + case "custom" => + config + .getBucketAssigner(sinkConfig.name) + .asInstanceOf[BucketAssigner[E, String]] + case other => + throw new IllegalArgumentException( + s"Unknown bucket assigner type '$other'." + ) + } + val encoderFormat = p.getProperty("encoder.format", "row") + val sink = encoderFormat match { + case "row" => + val builder = + StreamingFileSink.forRowFormat( + new Path(path), + config.getEncoder(sinkConfig.name).asInstanceOf[Encoder[E]] + ) + val rollingPolicy = + p.getProperty("bucket.rolling.policy", "default") match { + case "default" => + DefaultRollingPolicy + .builder() + .withInactivityInterval( + p.getProperty( + "bucket.rolling.policy.inactivity.interval", + s"${60000}" + ).toLong + ) + .withMaxPartSize( + p.getProperty( + "bucket.rolling.policy.max.part.size", + s"${128 * 1024 * 1024}" + ).toLong + ) + .withRolloverInterval( + p.getProperty( + "bucket.rolling.policy.rollover.interval", + s"${Long.MaxValue}" + ).toLong + ) + .build[E, String]() + case "checkpoint" => + OnCheckpointRollingPolicy.build[E, String]() + case policy => + throw new IllegalArgumentException( + s"Unknown bucket rolling policy type: '$policy'" + ) + } + builder + .withBucketAssigner(bucketAssigner) + .withRollingPolicy(rollingPolicy) + .withBucketCheckInterval(bucketCheckInterval) + .build() + case "bulk" => + throw new NotImplementedError("Bulk file sink not implemented yet") + + case _ => + throw new IllegalArgumentException( + s"Unknown file sink encoder format: '$encoderFormat'" + ) + } + stream.addSink(sink) + } + + /** + * Send stream to a socket sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toSocket[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: SocketSinkConfig + ): DataStreamSink[E] = + stream + .writeToSocket( + sinkConfig.host, + sinkConfig.port, + config + .getSerializationSchema(sinkConfig.name) + .asInstanceOf[SerializationSchema[E]] + ) + + /** + * Send stream to a cassandra sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toCassandraSink[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: CassandraSinkConfig): CassandraSink[E] = + CassandraSink + .addSink(stream) + .setHost(sinkConfig.host) + .setQuery(sinkConfig.query) + .build() + + /** + * Send stream to an elasticsearch sink. + * + * @param stream + * the data stream + * @param sinkConfig + * a sink configuration + * @tparam E + * stream element type + * @return + * DataStreamSink[E] + */ + def toElasticsearchSink[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: ElasticsearchSinkConfig + ): DataStreamSink[E] = { + val hosts = sinkConfig.transports.map { s => + val url = new URL(if (s.startsWith("http")) s else s"http://$s") + val hostname = url.getHost + val port = if (url.getPort < 0) 9200 else url.getPort + new HttpHost(hostname, port, url.getProtocol) + }.asJava + val esSink = new ElasticsearchSink.Builder[E]( + hosts, + (element: E, _: RuntimeContext, indexer: RequestIndexer) => { + val data = element.getClass.getDeclaredFields + .filterNot(f => + Seq("$id", "$key", "$timestamp", "$action").contains( + f.getName + ) + ) + .foldLeft(Map.empty[String, Any]) { case (a, f) => + f.setAccessible(true) + val name = f.getName + f.get(element) match { + case Some(v: Any) => a + (name -> v) + case None => a + case v: Any => a + (name -> v) + } + } + .asJava + val req = Requests.indexRequest(sinkConfig.index).source(data) + indexer.add(req) + } + ).build() + stream.addSink(esSink) + } + } diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala index 42abd7c..1623da8 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala @@ -9,6 +9,8 @@ import org.apache.flink.api.common.serialization.{ Encoder, SerializationSchema } +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner import org.apache.flink.streaming.connectors.kafka.{ KafkaDeserializationSchema, @@ -21,45 +23,62 @@ import org.apache.flink.streaming.connectors.kinesis.serialization.{ trait FlinkRunnerFactory[ADT <: FlinkEvent] { - def getJobInstance( + def getFlinkConfig( + args: Array[String], + sources: Map[String, Seq[Array[Byte]]] = Map.empty, + optConfig: Option[String] = None) = + new FlinkConfig[ADT](args, this, sources, optConfig) + + def getJobInstance[DS, OUT <: ADT]( name: String, - config: FlinkConfig): BaseFlinkJob[_, _ <: ADT] + runner: FlinkRunner[ADT]): BaseFlinkJob[DS, OUT, ADT] - def getDeserializationSchema( + def getDeserializationSchema[E <: ADT]( name: String, - config: FlinkConfig): DeserializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): DeserializationSchema[E] = ??? - def getKafkaDeserializationSchema( + def getKafkaDeserializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KafkaDeserializationSchema[ADT] = + config: FlinkConfig[ADT]): KafkaDeserializationSchema[E] = ??? - def getKinesisDeserializationSchema( + def getKafkaRecordSerializationSchema[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): KafkaRecordSerializationSchema[E] = ??? + + def getKafkaRecordDeserializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KinesisDeserializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KafkaRecordDeserializationSchema[E] = ??? - def getSerializationSchema( + def getKinesisDeserializationSchema[E <: ADT]( name: String, - config: FlinkConfig): SerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KinesisDeserializationSchema[E] = ??? - def getKafkaSerializationSchema( + def getSerializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KafkaSerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): SerializationSchema[E] = ??? - def getKinesisSerializationSchema( + def getKafkaSerializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KinesisSerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KafkaSerializationSchema[E] = ??? - def getEncoder(name: String, config: FlinkConfig): Encoder[ADT] = ??? + def getKinesisSerializationSchema[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): KinesisSerializationSchema[E] = ??? - def getAddToJdbcBatchFunction( + def getEncoder[E <: ADT]( name: String, - config: FlinkConfig): AddToJdbcBatchFunction[ADT] = ??? + config: FlinkConfig[ADT]): Encoder[E] = ??? - def getBucketAssigner( + def getAddToJdbcBatchFunction[E <: ADT]( name: String, - config: FlinkConfig): BucketAssigner[ADT, String] = + config: FlinkConfig[ADT]): AddToJdbcBatchFunction[E] = ??? + + def getBucketAssigner[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): BucketAssigner[E, String] = ??? - def getAvroCoder(name: String, config: FlinkConfig): AvroCoder[_] = ??? + def getAvroCoder(name: String, config: FlinkConfig[ADT]): AvroCoder[_] = + ??? } diff --git a/src/main/scala/io/epiphanous/flinkrunner/algorithm/cardinality/HyperLogLog.scala b/src/main/scala/io/epiphanous/flinkrunner/algorithm/cardinality/HyperLogLog.scala index 4e0c221..fdbded2 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/algorithm/cardinality/HyperLogLog.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/algorithm/cardinality/HyperLogLog.scala @@ -4,12 +4,12 @@ import com.google.common.hash.Funnel import com.google.common.hash.Hashing.murmur3_128 /** - * Implements hyperloglog cardinality estimate based on paper by - * P. Flajolet, È. Fusy, O. Gandouet, F. Meiunier. - * HyperLogLog: the analysis of a near-optimal - * cardinality estimation algorithm. Proceedings of Discrete Mathematics and Theoretical Computer Science. - * Pages 127-146. 2007. - */ + * Implements hyperloglog cardinality estimate based on paper by P. + * Flajolet, È. Fusy, O. Gandouet, F. Meiunier. HyperLogLog: the analysis + * of a near-optimal cardinality estimation algorithm. Proceedings of + * Discrete Mathematics and Theoretical Computer Science. Pages 127-146. + * 2007. + */ case class HyperLogLog[T](funnel: Funnel[T], b: Int) { require(b >= 4 && b <= 16, "b must be an integer in [4,16]") @@ -47,11 +47,14 @@ case class HyperLogLog[T](funnel: Funnel[T], b: Int) { def nonEmpty = cardinality > 0 /** - * Incorporates an item into the registers, updates the cardinality estimate and returns it. - * - * @param item the item to add - * @return Long - */ + * Incorporates an item into the registers, updates the cardinality + * estimate and returns it. + * + * @param item + * the item to add + * @return + * Long + */ def add(item: T) = { val x = hash(item) val j = 1 + (x & (m - 1)) @@ -61,12 +64,13 @@ case class HyperLogLog[T](funnel: Funnel[T], b: Int) { } /** - * Compute the current distinct cardinality estimate. - * - * @return Long - */ + * Compute the current distinct cardinality estimate. + * + * @return + * Long + */ private def estimateCardinality: Long = { - val E = am2 / M.map(i => 1 / math.pow(2d, i.toDouble)).sum + val E = am2 / M.map(i => 1 / math.pow(2d, i.toDouble)).sum // small range correction val Estar = if (E <= smallRange) { val V = M.count(_ == 0) @@ -83,37 +87,45 @@ case class HyperLogLog[T](funnel: Funnel[T], b: Int) { } /** - * Merge another HyperLogLog[T] instance into this instance. Note the other instance must have the same b - * parameter as this instance. - * - * @param another the other HyperLogLog[T] instance - */ + * Merge another HyperLogLog[T] instance into this instance. Note the + * other instance must have the same b parameter as this instance. + * + * @param another + * the other HyperLogLog[T] instance + */ def merge(another: HyperLogLog[T]) = { if (another.nonEmpty) { require(another.m == m, s"Can only merge HLL with same b=$b") - another.M.zipWithIndex.foreach { case (other, i) => if (M(i) < other) M(i) = other } + another.M.zipWithIndex.foreach { case (other, i) => + if (M(i) < other) M(i) = other + } estimateCardinality } this } /** - * Computes positive integer hash of item - * - * @param item item to hash - * @return Int - */ + * Computes positive integer hash of item + * + * @param item + * item to hash + * @return + * Int + */ private def hash(item: T): Int = { val h = hasher.hashObject(item, funnel).asInt() if (h < 0) ~h else h } /** - * Computes most significant set bit of an integer, where returned bit in [0,32]. - * - * @param i the non-negative Int to examine - * @return Int - */ + * Computes most significant set bit of an integer, where returned bit in + * [0,32]. + * + * @param i + * the non-negative Int to examine + * @return + * Int + */ private def rho(i: Int): Int = { require(i >= 0, "i must be non-negative integer") (32 - HyperLogLog.MASKS.lastIndexWhere(_ <= i)) % 33 @@ -121,8 +133,8 @@ case class HyperLogLog[T](funnel: Funnel[T], b: Int) { } object HyperLogLog { - val MASKS = Range(1, 32).map(i => 1 << (i - 1)) + val MASKS = Range(1, 32).map(i => 1 << (i - 1)) val ALPHA_M = 1 / (2 * math.log(2)) - val TWO32 = math.pow(2, 32) + val TWO32 = math.pow(2, 32) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala index 26db8d1..7a10fde 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala @@ -41,7 +41,7 @@ case class StableBloomFilter[T]( ) /** number of cells per unit storage */ - val storedCells: Int = Math.floor(STORAGE_BITS / d).toInt + val storedCells: Int = STORAGE_BITS / d /** number of bits used per unit storage */ val storedBits: Int = storedCells * d diff --git a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilterBuilder.scala b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilterBuilder.scala index eb789f5..1dcd9a7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilterBuilder.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilterBuilder.scala @@ -3,24 +3,30 @@ package io.epiphanous.flinkrunner.algorithm.membership import com.google.common.hash.Funnel /** - * A builder interface for creating StableBloomFilter instances. - * - * @param funnel a guava funnel - * @param numCells number of cells in the filter - * @param bitsPerCell number of bits per cell in the filter - * @param falsePositiveRate desired maximum false positive rate of the filter - * @tparam T the type of item inserted into the filter - */ + * A builder interface for creating StableBloomFilter instances. + * + * @param funnel + * a guava funnel + * @param numCells + * number of cells in the filter + * @param bitsPerCell + * number of bits per cell in the filter + * @param falsePositiveRate + * desired maximum false positive rate of the filter + * @tparam T + * the type of item inserted into the filter + */ case class StableBloomFilterBuilder[T]( - funnel: Funnel[T], - numCells: Long = 1000000, - bitsPerCell: Int = 3, - falsePositiveRate: Double = 0.01) { + funnel: Funnel[T], + numCells: Long = 1000000, + bitsPerCell: Int = 3, + falsePositiveRate: Double = 0.01) { def withNumCells(m: Long) = copy(numCells = m) def withBitsPerCell(d: Int) = copy(bitsPerCell = d) def withFalsePositiveRate(p: Double) = copy(falsePositiveRate = p) - def build() = StableBloomFilter(funnel, numCells, bitsPerCell, falsePositiveRate) + def build() = + StableBloomFilter(funnel, numCells, bitsPerCell, falsePositiveRate) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala index 9a69fff..8b49cec 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala @@ -14,6 +14,10 @@ import scala.util.Try * @tparam Context * the schema registry context type */ +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) class AvroCoder[Context](registry: AvroSchemaRegistryClient[Context]) extends LazyLogging with Serializable { diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala index 048495c..4a1c997 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala @@ -1,4 +1,10 @@ package io.epiphanous.flinkrunner.avro -class AvroCodingException(message: String = "Failure during Avro coding", cause: Throwable = None.orNull) - extends Exception(message, cause) +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) +class AvroCodingException( + message: String = "Failure during Avro coding", + cause: Throwable = None.orNull) + extends Exception(message, cause) diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroSchemaRegistryClient.scala index 0498901..d2cf803 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroSchemaRegistryClient.scala @@ -5,6 +5,10 @@ import scala.util.Try /** * A trait for an avro schema registry */ +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) trait AvroSchemaRegistryClient[Context] { /** diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala index c2f0f83..deddaf9 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala @@ -4,10 +4,11 @@ import cats.effect.{ContextShift, IO, Resource, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.typesafe.scalalogging.LazyLogging import io.circe.Decoder -import io.epiphanous.flinkrunner.model.FlinkConfig +import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} import io.epiphanous.flinkrunner.util.StringUtils import org.apache.avro.Schema.Parser -import org.apache.flink.runtime.concurrent.Executors.directExecutionContext +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.util.concurrent.Executors import org.http4s.EntityDecoder import org.http4s.circe.jsonOf import org.http4s.client.Client @@ -15,11 +16,15 @@ import org.http4s.client.blaze.BlazeClientBuilder import java.nio.ByteBuffer import java.util.concurrent.TimeUnit -import scala.concurrent.ExecutionContext +import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.util.{Failure, Success, Try} -class ConfluentSchemaRegistryClient()(implicit - config: FlinkConfig, +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) +class ConfluentSchemaRegistryClient[ADT <: FlinkEvent: TypeInformation]( + config: FlinkConfig[ADT])(implicit decoder: Decoder[ConfluentSchemaRegistryResponse]) extends AvroSchemaRegistryClient[ConfluentSchemaRegistryContext] with StringUtils @@ -37,7 +42,8 @@ class ConfluentSchemaRegistryClient()(implicit jsonOf[IO, ConfluentSchemaRegistryResponse] @transient - lazy implicit val ec: ExecutionContext = directExecutionContext() + lazy implicit val ec: ExecutionContextExecutor = + ExecutionContext.fromExecutor(Executors.directExecutor()) @transient lazy implicit val cs: ContextShift[IO] = IO.contextShift(ec) @@ -105,7 +111,7 @@ class ConfluentSchemaRegistryClient()(implicit .concurrencyLevel( config.getInt(s"$configPrefix.cache.concurrency.level") ) - .maximumSize(config.getInt(s"$configPrefix.cache.max.size")) + .maximumSize(config.getLong(s"$configPrefix.cache.max.size")) .expireAfterWrite(expireAfter.toMillis, TimeUnit.MILLISECONDS) // .expireAfterWrite(expireAfter) // for guava 27 if (!config.getBoolean(s"$configPrefix.cache.use.strong.keys")) @@ -209,14 +215,17 @@ class ConfluentSchemaRegistryClient()(implicit */ protected def getSubjectName[E]( event: E, - optContext: Option[ConfluentSchemaRegistryContext] = None): String = - (event.getClass.getCanonicalName.split("\\.") - :+ (if (optContext.getOrElse(ConfluentSchemaRegistryContext()).isKey) - "key" - else "value")) - .map(snakify) - .map(name => clean(name, replacement = "_")) - .mkString("_") + optContext: Option[ConfluentSchemaRegistryContext] = None) + : String = { + val keyOrValue = + if (optContext.getOrElse(ConfluentSchemaRegistryContext()).isKey) + "key" + else "value" + val subjectName = config.getString( + s"schema.registry.${event.getClass.getCanonicalName}" + ) + s"$subjectName-$keyOrValue" + } /** * Retrieve a schema based on its id or subject, and optionally, some diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryContext.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryContext.scala index c797df7..be1c450 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryContext.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryContext.scala @@ -1,5 +1,9 @@ package io.epiphanous.flinkrunner.avro +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) case class ConfluentSchemaRegistryContext( isKey: Boolean = false, version: String = "latest") diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryResponse.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryResponse.scala index cf57ba0..ca3527d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryResponse.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryResponse.scala @@ -5,18 +5,34 @@ import io.circe.generic.semiauto.{deriveDecoder, deriveEncoder} import io.circe.syntax._ import io.circe.{Decoder, Encoder} +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) sealed trait ConfluentSchemaRegistryResponse { def schema: String } +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) case class ConfluentSchemaRegistryResponseById(schema: String) extends ConfluentSchemaRegistryResponse +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) object ConfluentSchemaRegistryResponseById { implicit val encoder = deriveEncoder[ConfluentSchemaRegistryResponseById] implicit val decoder = deriveDecoder[ConfluentSchemaRegistryResponseById] } +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) case class ConfluentSchemaRegistryResponseBySubjectVersion( subject: String, id: Int, diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala index 0f0a335..ad70dfb 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala @@ -8,6 +8,10 @@ import java.time.Instant import java.time.temporal.ChronoUnit import scala.util.Try +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) case class RegisteredAvroSchema( schema: Schema, id: String, @@ -28,8 +32,7 @@ case class RegisteredAvroSchema( */ def decode[E: Decoder](bytes: Array[Byte]): Try[E] = { Try( - AvroInputStream - .binary[E] + AvroInputStream.binary .from(bytes) .build(schema) .iterator @@ -55,7 +58,7 @@ case class RegisteredAvroSchema( magic: Array[Byte] = Array.emptyByteArray): Try[Array[Byte]] = Try { val baos = new ByteArrayOutputStream() - val os = AvroOutputStream.binary[E].to(baos).build() + val os = AvroOutputStream.binary.to(baos).build() os.write(event) os.flush() os.close() diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/TestSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/TestSchemaRegistryClient.scala index ae73af3..37a34a8 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/TestSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/TestSchemaRegistryClient.scala @@ -4,6 +4,10 @@ import java.nio.ByteBuffer import scala.collection.mutable import scala.util.{Failure, Success, Try} +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) class TestSchemaRegistryClient extends AvroSchemaRegistryClient[ConfluentSchemaRegistryContext] { val schemas = diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala index 5bf7771..53fbcb6 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala @@ -1,84 +1,107 @@ package io.epiphanous.flinkrunner.flink import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.SEE import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} -import io.epiphanous.flinkrunner.util.StreamUtils._ +import io.epiphanous.flinkrunner.util.StreamUtils.Pipe +import io.epiphanous.flinkrunner.{FlinkRunner, SEE} import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.datastream.DataStreamUtils import org.apache.flink.streaming.api.scala.DataStream -import scala.collection.JavaConverters._ +import scala.util.Try /** - * An abstract flink job to transform on an input stream into an output stream. - * - * @tparam DS The type of the input stream - * @tparam OUT The type of output stream elements - */ -abstract class BaseFlinkJob[DS: TypeInformation, OUT <: FlinkEvent : TypeInformation] extends LazyLogging { + * An abstract flink job to transform on an input stream into an output + * stream. + * @param runner + * the flink runner associated with this job + * @tparam DS + * The type of the input data stream (not its elements) + * @tparam OUT + * The type of output stream elements + * @tparam ADT + * The flink runner's algebraic data type + */ +abstract class BaseFlinkJob[ + DS: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends LazyLogging { + + val config: FlinkConfig[ADT] = runner.config + val env: SEE = runner.env /** - * A pipeline for transforming a single stream. Passes the output of source() - * through transform() and the result of that into maybeSink(), which may pass it - * into sink() if we're not testing. Ultimately, returns the output data stream to - * facilitate testing. - * - * @param config implicit flink job config - * @return data output stream - */ - def flow()(implicit config: FlinkConfig, env: SEE): DataStream[OUT] = - source |> transform |# maybeSink + * A pipeline for transforming a single stream. Passes the output of + * source() through transform() and the result of that into maybeSink(), + * which may pass it into sink() if we're not testing. Ultimately, + * returns the output data stream to facilitate testing. + * + * @return + * data output stream + */ + def flow(): DataStream[OUT] = source |> transform |# maybeSink - def run()(implicit config: FlinkConfig, env: SEE): Either[Iterator[OUT], JobExecutionResult] = { + def run(limitOpt: Option[Int] = None) + : Either[List[OUT], JobExecutionResult] = { - logger.info(s"\nSTARTING FLINK JOB: ${config.jobName} ${config.jobArgs.mkString(" ")}\n") + logger.info( + s"\nSTARTING FLINK JOB: ${config.jobName} ${config.jobArgs.mkString(" ")}\n" + ) - val stream = flow + val stream = flow() - if (config.showPlan) logger.info(s"PLAN:\n${env.getExecutionPlan}\n") + if (config.showPlan) + logger.info(s"PLAN:\n${env.getExecutionPlan}\n") - if (config.mockEdges) - Left(DataStreamUtils.collect(stream.javaStream).asScala) - else + if (config.mockEdges) { + val limit = limitOpt.getOrElse( + Try(config.getJobConfig(config.jobName).getInt("run.limit")) + .getOrElse(100) + ) + Left(stream.executeAndCollect(config.jobName, limit)) + } else Right(env.execute(config.jobName)) } /** - * Returns source data stream to pass into transform(). This must be overridden by subclasses. - * - * @return input data stream - */ - def source()(implicit config: FlinkConfig, env: SEE): DS + * Returns source data stream to pass into transform(). This must be + * overridden by subclasses. + * + * @return + * input data stream + */ + def source(): DS /** - * Primary method to transform the source data stream into the output data stream. The output of - * this method is passed into sink(). This method must be overridden by subclasses. - * - * @param in input data stream created by source() - * @param config implicit flink job config - * @return output data stream - */ - def transform(in: DS)(implicit config: FlinkConfig, env: SEE): DataStream[OUT] + * Primary method to transform the source data stream into the output + * data stream. The output of this method is passed into sink(). This + * method must be overridden by subclasses. + * + * @param in + * input data stream created by source() + * @return + * output data stream + */ + def transform(in: DS): DataStream[OUT] /** - * Writes the transformed data stream to configured output sinks. - * - * @param out a transformed stream from transform() - * @param config implicit flink job config - */ - def sink(out: DataStream[OUT])(implicit config: FlinkConfig, env: SEE): Unit = - config.getSinkNames.foreach(name => out.toSink(name)) + * Writes the transformed data stream to configured output sinks. + * + * @param out + * a transformed stream from transform() + */ + def sink(out: DataStream[OUT]): Unit = + config.getSinkNames.foreach(name => runner.toSink[OUT](out, name)) /** - * The output stream will only be passed to BaseFlinkJob.sink - * if FlinkConfig.mockEdges is false (ie, you're not testing). - * - * @param out the output data stream to pass into BaseFlinkJob.sink) - * @param config implicit flink job config - */ - def maybeSink(out: DataStream[OUT])(implicit config: FlinkConfig, env: SEE): Unit = + * The output stream will only be passed to BaseFlinkJob.sink if + * FlinkConfig.mockEdges is false (ie, you're not testing). + * + * @param out + * the output data stream to pass into BaseFlinkJob.sink) + */ + def maybeSink(out: DataStream[OUT]): Unit = if (!config.mockEdges) sink(out) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/BroadcastFlinkJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/BroadcastFlinkJob.scala index 50cf18a..53b562a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/BroadcastFlinkJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/BroadcastFlinkJob.scala @@ -1,8 +1,7 @@ package io.epiphanous.flinkrunner.flink -import io.epiphanous.flinkrunner.SEE -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} -import io.epiphanous.flinkrunner.util.StreamUtils._ +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model.FlinkEvent import org.apache.flink.api.common.state.MapStateDescriptor import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ @@ -18,18 +17,25 @@ import org.apache.flink.streaming.api.scala.{ * href="https://flink.apache.org/2019/06/26/broadcast-state.html">broadcast * stream join pattern. * + * @param runner + * the flink runner associated with this job * @tparam IN * Input stream event type * @tparam BC * Broadcast stream event type * @tparam OUT * Output stream event type + * @tparam ADT + * The flink runner's algebraic data type */ abstract class BroadcastFlinkJob[ - IN <: FlinkEvent: TypeInformation, - BC <: FlinkEvent: TypeInformation, - OUT <: FlinkEvent: TypeInformation] - extends BaseFlinkJob[BroadcastConnectedStream[IN, BC], OUT] { + IN <: ADT: TypeInformation, + BC <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends BaseFlinkJob[BroadcastConnectedStream[IN, BC], OUT, ADT]( + runner + ) { import BroadcastFlinkJob._ @@ -38,12 +44,10 @@ abstract class BroadcastFlinkJob[ * output data stream from the connected broadcast + events stream. Must * be overridden by sub-classes. * - * @param config - * implicit flink config * @return * KeyedBroadcastProcessFunction[String, IN, BC, OUT] */ - def getBroadcastProcessFunction()(implicit config: FlinkConfig) + def getBroadcastProcessFunction : KeyedBroadcastProcessFunction[String, IN, BC, OUT] /** @@ -52,14 +56,12 @@ abstract class BroadcastFlinkJob[ * @param nameOpt * the name of the broadcast stream in the source configuration * (default "broadcast") - * @param config - * implicit flink config * @return * MapStateDescriptor[String, BC] */ def getBroadcastStateDescriptor( nameOpt: Option[String] = None - )(implicit config: FlinkConfig): MapStateDescriptor[String, BC] = + ): MapStateDescriptor[String, BC] = new MapStateDescriptor[String, BC]( nameOpt.getOrElse(BROADCAST_STATE_DESCRIPTOR_NAME), createTypeInformation[String], @@ -69,41 +71,26 @@ abstract class BroadcastFlinkJob[ /** * Creates the broadcast source stream. * - * @param config - * implicit flink config - * @param env - * implicit streaming execution environment * @return * broadcast stream */ - def broadcastSource(implicit - config: FlinkConfig, - env: SEE): BroadcastStream[BC] = - fromSource[BC](getBroadcastSourceName).broadcast( - getBroadcastStateDescriptor() - ) - - def getBroadcastSourceName()(implicit config: FlinkConfig) = - BROADCAST_SOURCE_NAME - - def getEventSourceName()(implicit config: FlinkConfig) = - EVENT_SOURCE_NAME + def broadcastSource: BroadcastStream[BC] = + runner + .fromSource[BC](getBroadcastSourceName) + .broadcast( + getBroadcastStateDescriptor() + ) /** * Creates the broadcast stream and the input event stream and connects * them * - * @param config - * implicit flink config - * @param env - * implicit streaming execution environment * @return * connected broadcast + events stream */ - override def source()(implicit - config: FlinkConfig, - env: SEE): BroadcastConnectedStream[IN, BC] = - (fromSource[IN](getEventSourceName)) + override def source(): BroadcastConnectedStream[IN, BC] = + (runner + .fromSource[IN](getEventSourceName)) .keyBy((in: IN) => in.$key) .connect(broadcastSource) @@ -122,12 +109,16 @@ abstract class BroadcastFlinkJob[ */ override def transform( in: BroadcastConnectedStream[IN, BC] - )(implicit config: FlinkConfig, env: SEE): DataStream[OUT] = { + ): DataStream[OUT] = { val name = - s"processed:${getEventSourceName()}+${getBroadcastSourceName()}" - in.process(getBroadcastProcessFunction()).name(name).uid(name) + s"processed:$getEventSourceName+$getBroadcastSourceName" + in.process(getBroadcastProcessFunction).name(name).uid(name) } + def getBroadcastSourceName: String = BROADCAST_SOURCE_NAME + + def getEventSourceName: String = EVENT_SOURCE_NAME + } object BroadcastFlinkJob { diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/FilterByControlJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/FilterByControlJob.scala index e4ccc9e..ac1497e 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/FilterByControlJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/FilterByControlJob.scala @@ -1,12 +1,7 @@ package io.epiphanous.flinkrunner.flink -import io.epiphanous.flinkrunner.SEE -import io.epiphanous.flinkrunner.model.{ - DataOrControl, - FlinkConfig, - FlinkEvent -} -import io.epiphanous.flinkrunner.util.StreamUtils._ +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model.{DataOrControl, FlinkEvent} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.DataStream @@ -24,6 +19,8 @@ import org.apache.flink.streaming.api.scala.DataStream * * would output `d3 d4 d5`. * + * @param runner + * the flink runner associated with this job * @tparam D * the data type * @tparam C @@ -32,32 +29,30 @@ import org.apache.flink.streaming.api.scala.DataStream * the output stream element type */ abstract class FilterByControlJob[ - D <: FlinkEvent: TypeInformation, - C <: FlinkEvent: TypeInformation, - OUT <: FlinkEvent: TypeInformation] - extends FlinkJob[D, OUT] { + D <: ADT: TypeInformation, + C <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends FlinkJob[D, OUT, ADT](runner) { + + import io.epiphanous.flinkrunner.flink.FilterByControlJob._ /** * A source data stream for the data events. * - * @param config - * implicit flink config * @return * a data stream of data events. */ - def data(implicit config: FlinkConfig, env: SEE): DataStream[D] = - fromSource[D]("data") + def data: DataStream[D] = + runner.fromSource[D](getDataStreamName) /** * A source data stream for the control events. * - * @param config - * implicit flink config * @return * a data stream of control events. */ - def control(implicit config: FlinkConfig, env: SEE): DataStream[C] = - fromSource[C]("control") + def control: DataStream[C] = runner.fromSource[C](getControlStreamName) /** * Generate a stream of data records filtered by the control stream. This @@ -67,26 +62,31 @@ abstract class FilterByControlJob[ * when to emit the data records. It remembers the last control time and * state and updates it when the state changes. * * - * @param config - * implicit flink config * @return * data stream of data records */ - override def source()(implicit - config: FlinkConfig, - env: SEE): DataStream[D] = { + override def source(): DataStream[D] = { val controlLockoutDuration = config.getDuration("control.lockout.duration").toMillis + val name = getDataControlStreamName + + implicit val typeInformation + : TypeInformation[DataOrControl[D, C, ADT]] = + TypeInformation.of(classOf[DataOrControl[D, C, ADT]]) + val in = data .connect(control) - .map(DataOrControl.data[D, C], DataOrControl.control[D, C]) - .name("data+control") - .uid("data+control") + .map( + DataOrControl.data[D, C, ADT], + DataOrControl.control[D, C, ADT] + ) + .name(name) + .uid(name) - in.keyBy((e: DataOrControl[D, C]) => e.$key) + in.keyBy((e: DataOrControl[D, C, ADT]) => e.$key) .filterWithState[(Long, Boolean)]((dc, lastControlOpt) => { if (dc.isData) { val emit = lastControlOpt match { @@ -109,9 +109,21 @@ abstract class FilterByControlJob[ }) .name(s"filter:${in.name}") .uid(s"filter:${in.name}") - .map((x: DataOrControl[D, C]) => x.data.get) + .map((x: DataOrControl[D, C, ADT]) => x.data.get) .name("filtered:data") .uid("filtered:data") } + def getDataStreamName: String = DATA_STREAM_NAME + + def getControlStreamName: String = CONTROL_STREAM_NAME + + def getDataControlStreamName: String = + s"$getDataStreamName+$getControlStreamName" + +} + +object FilterByControlJob { + val DATA_STREAM_NAME = "data" + val CONTROL_STREAM_NAME = "control" } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/FlinkJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/FlinkJob.scala index 3f0655b..3f6f549 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/FlinkJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/FlinkJob.scala @@ -1,8 +1,7 @@ package io.epiphanous.flinkrunner.flink -import io.epiphanous.flinkrunner.SEE -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} -import io.epiphanous.flinkrunner.util.StreamUtils._ +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model.FlinkEvent import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.DataStream @@ -10,24 +9,27 @@ import org.apache.flink.streaming.api.scala.DataStream * An abstract flink job to transform on a stream of events from an * algebraic data type (ADT). * + * @param runner + * the flink runner associated with this job * @tparam IN * The type of input stream elements * @tparam OUT * The type of output stream elements + * @tparam ADT + * The flink runner's algebraic data type */ abstract class FlinkJob[ - IN <: FlinkEvent: TypeInformation, - OUT <: FlinkEvent: TypeInformation] - extends BaseFlinkJob[DataStream[IN], OUT] { + IN <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends BaseFlinkJob[DataStream[IN], OUT, ADT](runner) { /** * Return the primary event source name - * @param config - * implicit flink config * @return * primary source name */ - def getEventSourceName(implicit config: FlinkConfig): String = + def getEventSourceName: String = config.getSourceNames.headOption.getOrElse("events") /** @@ -37,7 +39,7 @@ abstract class FlinkJob[ * @return * input data stream */ - def source()(implicit config: FlinkConfig, env: SEE): DataStream[IN] = - fromSource[IN](getEventSourceName) + def source(): DataStream[IN] = + runner.fromSource[IN](getEventSourceName) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/IdentityJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/IdentityJob.scala index 98c892f..de79d6c 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/IdentityJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/IdentityJob.scala @@ -1,26 +1,32 @@ package io.epiphanous.flinkrunner.flink -import io.epiphanous.flinkrunner.SEE -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model.FlinkEvent import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.DataStream -class IdentityJob[E <: FlinkEvent: TypeInformation] - extends FlinkJob[E, E] { +/** + * An identity mapper that passes through an input events to its configured + * sinks. + * @param runner + * the flink runner associated with this job + * @tparam E + * the input and output event type + * @tparam ADT + * The flink runner's algebraic data type + */ +class IdentityJob[ + E <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends FlinkJob[E, E, ADT](runner) { /** * Does the identity transform (passes the stream through unchanged). * * @param in * input data stream created by source() - * @param config - * implicit flink job config - * @param env - * streaming execution environment * @return * output data stream */ - override def transform( - in: DataStream[E])(implicit config: FlinkConfig, env: SEE) = - in + override def transform(in: DataStream[E]): DataStream[E] = in } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/ConfigToProps.scala b/src/main/scala/io/epiphanous/flinkrunner/model/ConfigToProps.scala index f15d55c..31403a7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/ConfigToProps.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/ConfigToProps.scala @@ -5,32 +5,34 @@ import com.typesafe.config.ConfigObject import java.util.{Properties, List => JList, Map => JMap} import scala.collection.JavaConverters._ -trait ConfigToProps { - def config: Option[ConfigObject] +object ConfigToProps { - // this flattens a hierarchical config into a string -> string properties map - val properties: Properties = { - val p = new Properties() + implicit class RichConfigObject(val config: Option[ConfigObject]) { - def flatten(key: String, value: Object): Unit = { - val pkey = if (key.isEmpty) key else s"$key." - value match { - case map: JMap[String, Object] @unchecked => - map.asScala.foreach { case (k, v) => flatten(s"$pkey$k", v) } - case list: JList[Object] @unchecked => - list.asScala.zipWithIndex.foreach { case (v, i) => - flatten(s"$pkey$i", v) - } - case v => - p.put(key, v.toString) - () // force unit return + // this flattens a hierarchical config into a string -> string properties map + def asProperties: Properties = { + val p = new Properties() + + def flatten(key: String, value: Object): Unit = { + val pkey = if (key.isEmpty) key else s"$key." + value match { + case map: JMap[String, Object] @unchecked => + map.asScala.foreach { case (k, v) => flatten(s"$pkey$k", v) } + case list: JList[Object] @unchecked => + list.asScala.zipWithIndex.foreach { case (v, i) => + flatten(s"$pkey$i", v) + } + case v => + p.put(key, v.toString) + () // force unit return + } } - } - config match { - case Some(c) => flatten("", c.unwrapped()) - case None => // noop + config match { + case Some(c) => flatten("", c.unwrapped()) + case None => // noop + } + p } - p } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/DataControlPeriod.scala b/src/main/scala/io/epiphanous/flinkrunner/model/DataControlPeriod.scala index 6bac29d..8f09583 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/DataControlPeriod.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/DataControlPeriod.scala @@ -2,7 +2,7 @@ package io.epiphanous.flinkrunner.model import java.util.UUID -case class DataControlPeriod[D <: FlinkEvent]( +case class DataControlPeriod[D <: ADT, ADT <: FlinkEvent]( id: String = UUID.randomUUID().toString, key: String, start: Long = 0L, diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/DataOrControl.scala b/src/main/scala/io/epiphanous/flinkrunner/model/DataOrControl.scala index b6aeff3..6c3f105 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/DataOrControl.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/DataOrControl.scala @@ -1,6 +1,8 @@ package io.epiphanous.flinkrunner.model -case class DataOrControl[D <: FlinkEvent, C <: FlinkEvent]( +import org.apache.flink.api.common.typeinfo.TypeInformation + +case class DataOrControl[D <: ADT, C <: ADT, ADT <: FlinkEvent]( event: Either[D, C]) extends FlinkEvent { def $id: String = event.fold(_.$id, _.$id) @@ -23,9 +25,11 @@ case class DataOrControl[D <: FlinkEvent, C <: FlinkEvent]( } object DataOrControl { - def data[D <: FlinkEvent, C <: FlinkEvent]( - event: D): DataOrControl[D, C] = DataOrControl[D, C](Left(event)) + def data[D <: ADT, C <: ADT, ADT <: FlinkEvent: TypeInformation]( + data: D): DataOrControl[D, C, ADT] = + DataOrControl[D, C, ADT](Left(data)) - def control[D <: FlinkEvent, C <: FlinkEvent]( - event: C): DataOrControl[D, C] = DataOrControl[D, C](Right(event)) + def control[D <: ADT, C <: ADT, ADT <: FlinkEvent: TypeInformation]( + control: C) = + DataOrControl[D, C, ADT](Right(control)) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala index 505ec33..07cc530 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -2,26 +2,23 @@ package io.epiphanous.flinkrunner.model import com.typesafe.config.{ConfigFactory, ConfigObject} import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.model.ConfigToProps.RichConfigObject import io.epiphanous.flinkrunner.{FlinkRunnerFactory, SEE} import org.apache.flink.api.java.utils.ParameterTool -import org.apache.flink.contrib.streaming.state.{ - PredefinedOptions, - RocksDBStateBackend -} -import org.apache.flink.runtime.state.filesystem.FsStateBackend -import org.apache.flink.streaming.api.TimeCharacteristic +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema +import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import java.io.File import java.time.Duration -import java.util.{Properties, List => JList, Map => JMap} +import java.util.Properties import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} @SerialVersionUID(1544548116L) -class FlinkConfig( +class FlinkConfig[ADT <: FlinkEvent]( args: Array[String], - factory: FlinkRunnerFactory[_], + factory: FlinkRunnerFactory[ADT], sources: Map[String, Seq[Array[Byte]]] = Map.empty, optConfig: Option[String] = None) extends LazyLogging @@ -128,24 +125,7 @@ class FlinkConfig( case (_, p) => _config.getDuration(p) } - def getProperties(path: String): Properties = { - val p = new Properties() - - def flatten(key: String, value: Object): Unit = { - val pkey = if (key.isEmpty) key else s"$key." - value match { - case map: JMap[String, Object] @unchecked => - map.asScala.foreach { case (k, v) => flatten(s"$pkey$k", v) } - case list: JList[Object] @unchecked => - list.asScala.zipWithIndex.foreach { case (v, i) => - flatten(s"$pkey$i", v) - } - case v => - p.put(key, v.toString) - () // force unit return - } - } - + def getProperties(path: String): Properties = (_s(path) match { case ("a", p) => Some( @@ -155,12 +135,7 @@ class FlinkConfig( ) case (_, p) => if (_config.hasPath(p)) Some(_config.getObject(p)) else None - }) match { - case Some(c) => flatten("", c.unwrapped()) - case None => // noop - } - p - } + }).asProperties def _classInstance[T](path: String): T = Class @@ -171,33 +146,46 @@ class FlinkConfig( // def getJobInstance = factory.getJobInstance(jobName, this) - def getDeserializationSchema(name: String) = - factory.getDeserializationSchema(name, this) + def getDeserializationSchema[E <: ADT](name: String) = + factory.getDeserializationSchema[E](name, this) + + def getKafkaDeserializationSchema[E <: ADT](name: String) = + factory.getKafkaDeserializationSchema[E](name, this) - def getKafkaDeserializationSchema(name: String) = - factory.getKafkaDeserializationSchema(name, this) + def getKafkaRecordDeserializationSchema[E <: ADT]( + name: String): KafkaRecordDeserializationSchema[E] = + factory.getKafkaRecordDeserializationSchema[E](name, this) - def getKinesisDeserializationSchema(name: String) = - factory.getKinesisDeserializationSchema(name, this) + def getKinesisDeserializationSchema[E <: ADT](name: String) = + factory.getKinesisDeserializationSchema[E](name, this) - def getSerializationSchema(name: String) = - factory.getSerializationSchema(name, this) + def getSerializationSchema[E <: ADT](name: String) = + factory.getSerializationSchema[E](name, this) - def getKafkaSerializationSchema(name: String) = - factory.getKafkaSerializationSchema(name, this) + def getKafkaSerializationSchema[E <: ADT](name: String) = + factory.getKafkaSerializationSchema[E](name, this) - def getKinesisSerializationSchema(name: String) = - factory.getKinesisSerializationSchema(name, this) + def getKafkaRecordSerializationSchema[E <: ADT](name: String) = + factory.getKafkaRecordSerializationSchema[E](name, this) - def getEncoder(name: String) = factory.getEncoder(name, this) + def getKinesisSerializationSchema[E <: ADT](name: String) = + factory.getKinesisSerializationSchema[E](name, this) - def getAddToJdbcBatchFunction(name: String) = - factory.getAddToJdbcBatchFunction(name, this) + def getEncoder[E <: ADT](name: String) = + factory.getEncoder[E](name, this) - def getBucketAssigner(name: String) = - factory.getBucketAssigner(name, this) + def getAddToJdbcBatchFunction[E <: ADT](name: String) = + factory.getAddToJdbcBatchFunction[E](name, this) - def getAvroCoder(name: String) = factory.getAvroCoder(name, this) + def getBucketAssigner[E <: ADT](name: String) = + factory.getBucketAssigner[E](name, this) + + @deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and ...Deserialization classes instead", + "4.0.0" + ) + def getAvroCoder(name: String) = + factory.getAvroCoder(name, this) def getSourceConfig(name: String): SourceConfig = SourceConfig(name, this) @@ -232,9 +220,6 @@ class FlinkConfig( else StreamExecutionEnvironment.getExecutionEnvironment - // use event time - env.setStreamTimeCharacteristic(timeCharacteristic) - // set parallelism env.setParallelism(globalParallelism) @@ -250,44 +235,16 @@ class FlinkConfig( checkpointMaxConcurrent ) - val backend = if (stateBackend == "rocksdb") { - logger.info(s"Using ROCKS DB state backend at $checkpointUrl") - val rocksBackend = - new RocksDBStateBackend(checkpointUrl, checkpointIncremental) - if (checkpointFlash) - rocksBackend.setPredefinedOptions( - PredefinedOptions.FLASH_SSD_OPTIMIZED - ) - rocksBackend - } else { - logger.info(s"Using FILE SYSTEM state backend at $checkpointUrl") - new FsStateBackend(checkpointUrl) - } - /* this deprecation is annoying; its due to rocksdb's state backend - extending AbstractStateBackend which is deprecated */ - env.setStateBackend(backend) + logger.info(s"Using ROCKS DB state backend at $checkpointUrl") + env.setStateBackend( + new EmbeddedRocksDBStateBackend(checkpointIncremental) + ) + env.getCheckpointConfig.setCheckpointStorage(checkpointUrl) } env } - def getTimeCharacteristic(tc: String): TimeCharacteristic = { - tc.toLowerCase - .replaceFirst("\\s*time$", "") match { - case "event" => TimeCharacteristic.EventTime - case "processing" => TimeCharacteristic.ProcessingTime - case "ingestion" => TimeCharacteristic.IngestionTime - case unknown => - throw new RuntimeException( - s"Unknown time.characteristic setting: '$unknown'" - ) - } - } - - lazy val timeCharacteristic = getTimeCharacteristic( - getString("time.characteristic") - ) - def getWatermarkStrategy(ws: String) = ws.toLowerCase.replaceAll("[^a-z]", "") match { case "boundedlateness" => "bounded lateness" diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala index 58a73b5..68b0b39 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala @@ -1,7 +1,9 @@ package io.epiphanous.flinkrunner.model +import com.google.common.collect.Maps import io.epiphanous.flinkrunner.model.FlinkConnectorName._ +import java.util import java.util.Properties sealed trait SinkConfig { @@ -12,10 +14,15 @@ sealed trait SinkConfig { def label: String = s"$connector/$name" def properties: Properties + + def propertiesMap: util.HashMap[String, String] = + Maps.newHashMap(Maps.fromProperties(properties)) } object SinkConfig { - def apply(name: String, config: FlinkConfig): SinkConfig = { + def apply[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig[ADT]): SinkConfig = { val p = s"sinks.$name" FlinkConnectorName.withNameInsensitiveOption( config.getString(s"$p.connector") diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala index f736f53..6c68c0d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala @@ -1,9 +1,11 @@ package io.epiphanous.flinkrunner.model +import com.google.common.collect.Maps import io.epiphanous.flinkrunner.model.FlinkConnectorName._ -import org.apache.flink.streaming.api.TimeCharacteristic +import java.util import java.util.Properties +import scala.concurrent.duration.DurationInt import scala.util.Try sealed trait SourceConfig { @@ -13,20 +15,24 @@ sealed trait SourceConfig { def label: String = s"$connector/$name" - def timeCharacteristic: TimeCharacteristic - def watermarkStrategy: String + def maxAllowedLateness: Long + def properties: Properties + + def propertiesMap: util.HashMap[String, String] = + Maps.newHashMap(Maps.fromProperties(properties)) } object SourceConfig { - def apply(name: String, config: FlinkConfig): SourceConfig = { + def apply[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig[ADT]): SourceConfig = { val p = s"sources.$name" - val timeCharacteristic = - Try(config.getString(s"$p.time.characteristic")) - .map(config.getTimeCharacteristic) - .getOrElse(config.timeCharacteristic) + val maxAllowedLateness = Try( + config.getDuration(s"$p.max.allowed.lateness") + ).map(_.toMillis).getOrElse(5.minutes.toMillis) val watermarkStrategy = Try(config.getString(s"$p.watermark.strategy")) .map(config.getWatermarkStrategy) .getOrElse(config.watermarkStrategy) @@ -42,8 +48,8 @@ object SourceConfig { name, config.getString(s"$p.topic"), config.getBoolean(s"$p.isKeyed"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case Kinesis => @@ -51,8 +57,8 @@ object SourceConfig { connector, name, config.getString(s"$p.stream"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case File => @@ -60,8 +66,8 @@ object SourceConfig { connector, name, config.getString(s"$p.path"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case Socket => @@ -70,8 +76,8 @@ object SourceConfig { name, config.getString(s"$p.host"), config.getInt(s"$p.port"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case Collection => @@ -79,8 +85,8 @@ object SourceConfig { connector, name, name, - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case other => @@ -101,8 +107,8 @@ final case class KafkaSourceConfig( name: String, topic: String, isKeyed: Boolean, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -110,8 +116,8 @@ final case class KinesisSourceConfig( connector: FlinkConnectorName = Kinesis, name: String, stream: String, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -119,8 +125,8 @@ final case class FileSourceConfig( connector: FlinkConnectorName = File, name: String, path: String, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -129,8 +135,8 @@ final case class SocketSourceConfig( name: String, host: String, port: Int, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -138,7 +144,7 @@ final case class CollectionSourceConfig( connector: FlinkConnectorName = Collection, name: String, topic: String, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/UnitMapper.scala b/src/main/scala/io/epiphanous/flinkrunner/model/UnitMapper.scala index f210406..4dadd2c 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/UnitMapper.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/UnitMapper.scala @@ -145,6 +145,7 @@ trait UnitMapper extends LazyLogging { ) } + //noinspection ScalaUnusedSymbol def getSymbolFromString(dimension: String, unit: String): String = unit } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Aggregate.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Aggregate.scala index 37e923d..4f43ac0 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Aggregate.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Aggregate.scala @@ -32,11 +32,12 @@ trait Aggregate extends Product with Serializable with LazyLogging { // a copy constructor private def _copy( - newValue: Double, - aggregatedLastUpdated: Instant, - dependentAggregations: Map[String, Aggregate] - ): Aggregate = - Aggregate(name, + newValue: Double, + aggregatedLastUpdated: Instant, + dependentAggregations: Map[String, Aggregate] + ): Aggregate = + Aggregate( + name, dimension, outUnit, newValue, @@ -44,64 +45,96 @@ trait Aggregate extends Product with Serializable with LazyLogging { aggregatedLastUpdated, Instant.now(), dependentAggregations, - params) + params + ) /** - * Used by some subclasses to update the underlying aggregate value as a Quantity. - * When this is called, any dependent aggregations will be updated and passed into - * the depAggs parameter. You can find the previous dependent aggregations in - * `this.dependentAggregations` if you need them. - * - * @param current Quantity value of the aggregate - * @param quantity Quantity the new quantity to incorporate into the aggregate - * @param depAggs dependent aggregations already updated with the new quantity - * @tparam A the dimension of the quantity - * @return A - */ - def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]): A = ??? + * Used by some subclasses to update the underlying aggregate value as a + * Quantity. When this is called, any dependent aggregations will be + * updated and passed into the depAggs parameter. You can find the + * previous dependent aggregations in `this.dependentAggregations` if you + * need them. + * + * @param current + * Quantity value of the aggregate + * @param quantity + * Quantity the new quantity to incorporate into the aggregate + * @param depAggs + * dependent aggregations already updated with the new quantity + * @tparam A + * the dimension of the quantity + * @return + * A + */ + def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]): A = ??? /** - * Update dependent aggregations. - * - * @param q the quantity being added to the aggregations - * @param aggLU the instant associated with the new quantity - * @param unitMapper a unit mapper - * @tparam A the type of the quantity - * @return - */ - def updateDependents[A <: Quantity[A]](q: A, aggLU: Instant, unitMapper: UnitMapper): Map[String, Aggregate] = + * Update dependent aggregations. + * + * @param q + * the quantity being added to the aggregations + * @param aggLU + * the instant associated with the new quantity + * @param unitMapper + * a unit mapper + * @tparam A + * the type of the quantity + * @return + */ + def updateDependents[A <: Quantity[A]]( + q: A, + aggLU: Instant, + unitMapper: UnitMapper): Map[String, Aggregate] = getDependents .map(kv => kv._1 -> kv._2.update(q, aggLU, unitMapper)) .filter(_._2.nonEmpty) .map(kv => kv._1 -> kv._2.get) - def getDependents: Map[String, Aggregate] = this.dependentAggregations + def getDependents: Map[String, Aggregate] = this.dependentAggregations /** - * Update the aggregate with a Quantity. - * - * @param q Quantity[A] - * @param aggLU event timestamp of quantity - * @tparam A dimension of Quantity - * @return Aggregate - */ - def update[A <: Quantity[A]](q: A, aggLU: Instant, unitMapper: UnitMapper): Option[Aggregate] = { + * Update the aggregate with a Quantity. + * + * @param q + * Quantity[A] + * @param aggLU + * event timestamp of quantity + * @tparam A + * dimension of Quantity + * @return + * Aggregate + */ + def update[A <: Quantity[A]]( + q: A, + aggLU: Instant, + unitMapper: UnitMapper): Option[Aggregate] = { if (q.dimension.name != dimension) { - logger.error(s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)") + logger.error( + s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)" + ) None } else { val depAggs = updateDependents(q, aggLU, unitMapper) if (depAggs.size < this.dependentAggregations.size) { - logger.error(s"$name[$dimension,$unit] dependents can not be updated with (Quantity[${q.dimension.name}]=$q)") + logger.error( + s"$name[$dimension,$unit] dependents can not be updated with (Quantity[${q.dimension.name}]=$q)" + ) None } else { unitMapper .createQuantity(q.dimension, value, unit) - .map(current => updateQuantity(current, q, depAggs) in current.unit) match { + .map(current => + updateQuantity(current, q, depAggs) in current.unit + ) match { case Some(updated) => Some(_copy(updated.value, aggLU, depAggs)) - case None => - logger.error(s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)") + case None => + logger.error( + s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)" + ) None } } @@ -109,20 +142,24 @@ trait Aggregate extends Product with Serializable with LazyLogging { } /** - * Most common entry point for updating aggregates. - * - * @param value Double value of quantity to update aggregate with - * @param unit String unit of quantity to update aggregate with - * @param aggLU event timestamp of value - * @param unitMapper allows caller to customize unit system mappings - * @return - */ + * Most common entry point for updating aggregates. + * + * @param value + * Double value of quantity to update aggregate with + * @param unit + * String unit of quantity to update aggregate with + * @param aggLU + * event timestamp of value + * @param unitMapper + * allows caller to customize unit system mappings + * @return + */ def update( - value: Double, - unit: String, - aggLU: Instant, - unitMapper: UnitMapper = UnitMapper.defaultUnitMapper - ): Option[Aggregate] = + value: Double, + unit: String, + aggLU: Instant, + unitMapper: UnitMapper = UnitMapper.defaultUnitMapper + ): Option[Aggregate] = unitMapper.updateAggregateWith(this, value, unit, aggLU) def isEmpty: Boolean = count == BigInt(0) @@ -139,91 +176,189 @@ trait Aggregate extends Product with Serializable with LazyLogging { object Aggregate extends LazyLogging { implicit class caseOps(s: String) { - def normalize: String = "[^A-Za-z\\d]".r.replaceAllIn(s, "").toLowerCase() + def normalize: String = + "[^A-Za-z\\d]".r.replaceAllIn(s, "").toLowerCase() } def apply( - name: String, - dimension: String, - unit: String, - value: Double = 0, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String] - ): Aggregate = { + name: String, + dimension: String, + unit: String, + value: Double = 0, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String] + ): Aggregate = { val normalizedName = name.normalize - val initValue = if (normalizedName == "min" && count == 0 && value == 0) Double.MaxValue else value + val initValue = + if (normalizedName == "min" && count == 0 && value == 0) + Double.MaxValue + else value normalizedName match { - case "mean" => - Mean(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated) - case "count" => - Count(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated) + case "mean" => + Mean( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated + ) + case "count" => + Count( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated + ) case "exponentialmovingaverage" => - ExponentialMovingAverage(dimension, + ExponentialMovingAverage( + dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations, - maybeUpdateParams(params, "alpha", ExponentialMovingAverage.defaultAlpha)) + maybeUpdateParams( + params, + "alpha", + ExponentialMovingAverage.defaultAlpha + ) + ) case "exponentialmovingstandarddeviation" => - ExponentialMovingStandardDeviation(dimension, + ExponentialMovingStandardDeviation( + dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations, - maybeUpdateParams(params, + maybeUpdateParams( + params, "alpha", - ExponentialMovingStandardDeviation.defaultAlpha)) + ExponentialMovingStandardDeviation.defaultAlpha + ) + ) case "exponentialmovingvariance" => - ExponentialMovingVariance(dimension, + ExponentialMovingVariance( + dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations, - maybeUpdateParams(params, "alpha", ExponentialMovingVariance.defaultAlpha)) + maybeUpdateParams( + params, + "alpha", + ExponentialMovingVariance.defaultAlpha + ) + ) case "histogram" => - Histogram(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations) + Histogram( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated, + dependentAggregations + ) case "max" => - Max(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated) + Max( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated + ) case "min" => - Min(dimension, unit, initValue, count, aggregatedLastUpdated, lastUpdated) + Min( + dimension, + unit, + initValue, + count, + aggregatedLastUpdated, + lastUpdated + ) case "range" => - Range(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations) + Range( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated, + dependentAggregations + ) case "sum" => - Sum(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated) + Sum( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated + ) case "variance" => - Variance(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations) + Variance( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated, + dependentAggregations + ) case "standarddeviation" => - StandardDeviation(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations) + StandardDeviation( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated, + dependentAggregations + ) case "sumofsquareddeviations" => - SumOfSquaredDeviations(dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations) - case "percentage" => - Percentage(dimension, + SumOfSquaredDeviations( + dimension, + unit, + value, + count, + aggregatedLastUpdated, + lastUpdated, + dependentAggregations + ) + case "percentage" => + Percentage( + dimension, unit, value, count, aggregatedLastUpdated, lastUpdated, dependentAggregations, - maybeUpdateParams(params, "base", Percentage.defaultBase)) + maybeUpdateParams(params, "base", Percentage.defaultBase) + ) case _ => val message = s"Unknown aggregation type '$name'" @@ -232,6 +367,9 @@ object Aggregate extends LazyLogging { } } - def maybeUpdateParams(map: Map[String, String], key: String, defaultValue: String): Map[String, String] = + def maybeUpdateParams( + map: Map[String, String], + key: String, + defaultValue: String): Map[String, String] = if (map.contains(key)) map else map.updated(key, defaultValue) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Count.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Count.scala index 51ab50d..f67acf3 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Count.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Count.scala @@ -5,21 +5,25 @@ import squants.{Dimensionless, Each, Quantity} import java.time.Instant final case class Count( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { override def isDimensionless = true override def outUnit: String = Each.symbol - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = current + current.unit(1) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingAverage.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingAverage.scala index 7e7bacc..d0f6c7a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingAverage.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingAverage.scala @@ -5,19 +5,27 @@ import squants.Quantity import java.time.Instant final case class ExponentialMovingAverage( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map("alpha" -> ExponentialMovingAverage.defaultAlpha)) - extends Aggregate { - - def alpha: Double = params.getOrElse("alpha", ExponentialMovingAverage.defaultAlpha).toDouble - - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map( + "alpha" -> ExponentialMovingAverage.defaultAlpha + )) + extends Aggregate { + + def alpha: Double = params + .getOrElse("alpha", ExponentialMovingAverage.defaultAlpha) + .toDouble + + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = if (count == 0) quantity else current * (1 - alpha) + quantity * alpha } @@ -27,6 +35,13 @@ object ExponentialMovingAverage { def defaultAlpha = DEFAULT_ALPHA.toString - def apply(dimension: String, unit: String, alpha: Double): ExponentialMovingAverage = - ExponentialMovingAverage(dimension, unit, params = Map("alpha" -> alpha.toString)) + def apply( + dimension: String, + unit: String, + alpha: Double): ExponentialMovingAverage = + ExponentialMovingAverage( + dimension, + unit, + params = Map("alpha" -> alpha.toString) + ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingStandardDeviation.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingStandardDeviation.scala index eab05c0..6f7b43f 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingStandardDeviation.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingStandardDeviation.scala @@ -5,24 +5,37 @@ import squants.Quantity import java.time.Instant final case class ExponentialMovingStandardDeviation( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map("alpha" -> ExponentialMovingStandardDeviation.defaultAlpha)) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map( + "alpha" -> ExponentialMovingStandardDeviation.defaultAlpha + )) + extends Aggregate { override def getDependents = { if (this.dependentAggregations.isEmpty) - Map("ExponentialMovingVariance" -> ExponentialMovingVariance(dimension, unit, params = params)) + Map( + "ExponentialMovingVariance" -> ExponentialMovingVariance( + dimension, + unit, + params = params + ) + ) else this.dependentAggregations } - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { - if (count == 0) current.unit(0d) else { + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { + if (count == 0) current.unit(0d) + else { val updatedEmv = depAggs("ExponentialMovingVariance") current.unit(Math.sqrt(updatedEmv.value)) } @@ -35,13 +48,20 @@ object ExponentialMovingStandardDeviation { def defaultAlpha = DEFAULT_ALPHA.toString - def apply(dimension: String, unit: String, alpha: Double): ExponentialMovingStandardDeviation = - ExponentialMovingStandardDeviation(dimension, + def apply( + dimension: String, + unit: String, + alpha: Double): ExponentialMovingStandardDeviation = + ExponentialMovingStandardDeviation( + dimension, unit, dependentAggregations = Map( - "ExponentialMovingVariance" -> ExponentialMovingVariance(dimension, + "ExponentialMovingVariance" -> ExponentialMovingVariance( + dimension, unit, - alpha) + alpha + ) ), - params = Map("alpha" -> alpha.toString)) + params = Map("alpha" -> alpha.toString) + ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingVariance.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingVariance.scala index 3cd6a94..e665e0f 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingVariance.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/ExponentialMovingVariance.scala @@ -5,29 +5,44 @@ import squants.Quantity import java.time.Instant final case class ExponentialMovingVariance( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map("alpha" -> ExponentialMovingVariance.defaultAlpha)) - extends Aggregate { - - def alpha = params.getOrElse("alpha", ExponentialMovingVariance.defaultAlpha).toDouble + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map( + "alpha" -> ExponentialMovingVariance.defaultAlpha + )) + extends Aggregate { + + def alpha = params + .getOrElse("alpha", ExponentialMovingVariance.defaultAlpha) + .toDouble override def getDependents = { if (this.dependentAggregations.isEmpty) - Map("ExponentialMovingAverage" -> ExponentialMovingAverage(dimension, unit, params = params)) + Map( + "ExponentialMovingAverage" -> ExponentialMovingAverage( + dimension, + unit, + params = params + ) + ) else this.dependentAggregations } - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { - if (count == 0) quantity.unit(0d) else { + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { + if (count == 0) quantity.unit(0d) + else { val currentEma = getDependents("ExponentialMovingAverage") - val q = quantity in current.unit - val delta = q - current.unit(currentEma.value) + val q = quantity in current.unit + val delta = q - current.unit(currentEma.value) (1 - alpha) * (current + delta * delta.value * alpha) } } @@ -39,7 +54,14 @@ object ExponentialMovingVariance { def defaultAlpha = DEFAULT_ALPHA.toString - def apply(dimension: String, unit: String, alpha: Double): ExponentialMovingVariance = - ExponentialMovingVariance(dimension, unit, params = Map("alpha" -> alpha.toString)) + def apply( + dimension: String, + unit: String, + alpha: Double): ExponentialMovingVariance = + ExponentialMovingVariance( + dimension, + unit, + params = Map("alpha" -> alpha.toString) + ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Histogram.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Histogram.scala index 6b9ae3d..18552d3 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Histogram.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Histogram.scala @@ -6,68 +6,82 @@ import squants.Quantity import java.time.Instant final case class Histogram( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { import Histogram._ - def bin(key: String): Aggregate = this.dependentAggregations.getOrElse(key, Count(dimension, unit)) + def bin(key: String): Aggregate = + this.dependentAggregations.getOrElse(key, Count(dimension, unit)) - /** Compute a dynamic bin for the requested quantity. This picks a bin - * based on the order of magnitude of the quantity in the aggregate's preferred unit. - * If the order of magnitude is 3 (say the value is 2345) - * For instance if the quantity value is 0.00157, its order of magnitude is -3. We - * reduce that in absolute value by 1 (= -2) to compute the min and max of the bin as - * [floor(0.0157 * 10**2)/10**2 (= 0.01) and - * ceil(0.0157 * 10**2)/10**2 (= 0.02). - * - * @param q the quantity to compute a bin of - * @return - */ + /** + * Compute a dynamic bin for the requested quantity. This picks a bin + * based on the order of magnitude of the quantity in the aggregate's + * preferred unit. If the order of magnitude is 3 (say the value is 2345) + * For instance if the quantity value is 0.00157, its order of magnitude + * is -3. We reduce that in absolute value by 1 (= -2) to compute the min + * and max of the bin as [floor(0.0157 * 10**2)/10**2 (= 0.01) and + * ceil(0.0157 * 10**2)/10**2 (= 0.02). + * + * @param q + * the quantity to compute a bin of + * @return + */ def binOf[A <: Quantity[A]](q: A, unitMapper: UnitMapper) = { unitMapper .createQuantity(q.dimension, value, unit) .map(_.unit) .map(u => (q in u).value) - .map(d => { - val absd = math.abs(d) - val magnitude = + .map { d => + val absd = math.abs(d) + val magnitude = math.floor(math.log10(if (absd < TOL) TOL else absd)).toInt - val sign = math.signum(magnitude) - val abs = math.abs(magnitude) - val mag = sign * (abs - 1) - val pow = math.pow(10, mag.toDouble) - val min = math.floor(d / pow) * pow - val max = math.ceil(d / pow) * pow + val sign = math.signum(magnitude) + val abs = math.abs(magnitude) + val mag = sign * (abs - 1) + val pow = math.pow(10, mag.toDouble) + val min = math.floor(d / pow) * pow + val max = math.ceil(d / pow) * pow val formatString = if (abs < 8) { - val fs = s"%${if (sign < 0) "." else ""}$abs${if (sign > 0) ".0" else ""}" + val fs = + s"%${if (sign < 0) "." else ""}$abs${if (sign > 0) ".0" else ""}" s"${fs}f,${fs}f" } else { "%e,%e" } formatString.format(min, max) - }) + } } - override def update[A <: Quantity[A]](q: A, aggLU: Instant, unitMapper: UnitMapper) = + override def update[A <: Quantity[A]]( + q: A, + aggLU: Instant, + unitMapper: UnitMapper) = binOf(q, unitMapper) match { case Some(binKey) => bin(binKey) .update(q.value, q.unit.symbol, aggLU, unitMapper) match { - case Some(updatedBin) => Some(copy(dependentAggregations = dependentAggregations.updated(binKey, updatedBin))) - case None => { - logger.error(s"$name[$dimension,$unit] Quantity[$q] can't be binned") + case Some(updatedBin) => + Some( + copy(dependentAggregations = + dependentAggregations.updated(binKey, updatedBin) + ) + ) + case None => + logger.error( + s"$name[$dimension,$unit] Quantity[$q] can't be binned" + ) None - } } - case None => + case None => logger.error(s"$name[$dimension,$unit] can't be updated with $q") None } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Max.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Max.scala index 5c84f28..6848a7d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Max.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Max.scala @@ -5,17 +5,21 @@ import squants.Quantity import java.time.Instant final case class Max( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = if (count == 0) quantity else current.max(quantity) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Mean.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Mean.scala index 4fd281c..1293d60 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Mean.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Mean.scala @@ -5,17 +5,21 @@ import squants.Quantity import java.time.Instant final case class Mean( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { val n = count.doubleValue() (current * n + quantity) / (n + 1) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Min.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Min.scala index a38b7e8..04ad344 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Min.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Min.scala @@ -5,17 +5,21 @@ import squants.Quantity import java.time.Instant final case class Min( - dimension: String, - unit: String, - value: Double = Double.MaxValue, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = Double.MaxValue, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = if (count == 0) quantity else current.min(quantity) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Percentage.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Percentage.scala index 9fe0d5f..bea5aad 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Percentage.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Percentage.scala @@ -6,33 +6,46 @@ import squants.{Percent, Quantity} import java.time.Instant final case class Percentage( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map("base" -> Percentage.defaultBase)) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map("base" -> Percentage.defaultBase)) + extends Aggregate { override def isDimensionless = true override def outUnit = Percent.symbol - val baseParam: Double = params.getOrElse("base", Percentage.defaultBase).toDouble + val baseParam: Double = + params.getOrElse("base", Percentage.defaultBase).toDouble def baseQuantity[A <: Quantity[A]](q: A, unitMapper: UnitMapper) = unitMapper.createQuantity(q.dimension, baseParam, unit) - override def update[A <: Quantity[A]](q: A, aggLU: Instant, unitMapper: UnitMapper) = { + override def update[A <: Quantity[A]]( + q: A, + aggLU: Instant, + unitMapper: UnitMapper) = { val updateValue = baseQuantity(q, unitMapper).map(b => q / b) match { case Some(addValue) => addValue * 100.0 - case None => - logger.error(s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)") + case None => + logger.error( + s"$name[$dimension,$unit] can not be updated with (Quantity[${q.dimension.name}]=$q)" + ) 0d } - Some(copy(value = this.value + updateValue, count = count + 1, aggregatedLastUpdated = aggLU)) + Some( + copy( + value = this.value + updateValue, + count = count + 1, + aggregatedLastUpdated = aggLU + ) + ) } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Range.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Range.scala index 0fc2a49..2c234c7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Range.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Range.scala @@ -5,15 +5,16 @@ import squants.Quantity import java.time.Instant final case class Range( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { override def getDependents = { if (this.dependentAggregations.isEmpty) @@ -21,6 +22,10 @@ final case class Range( else this.dependentAggregations } - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = - if (count == 0) current else current.unit(depAggs("Max").value - depAggs("Min").value) + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = + if (count == 0) current + else current.unit(depAggs("Max").value - depAggs("Min").value) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/StandardDeviation.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/StandardDeviation.scala index de104cb..17ca4ff 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/StandardDeviation.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/StandardDeviation.scala @@ -5,18 +5,23 @@ import squants.Quantity import java.time.Instant final case class StandardDeviation( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { - if (count == 0) current else { + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { + if (count == 0) current + else { val updatedVariance = depAggs("Variance") current.unit(Math.sqrt(updatedVariance.value)) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Sum.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Sum.scala index fbe1199..0d8db5a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Sum.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Sum.scala @@ -5,17 +5,21 @@ import squants.Quantity import java.time.Instant final case class Sum( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = current + quantity } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/SumOfSquaredDeviations.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/SumOfSquaredDeviations.scala index 42a2231..b9fbff7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/SumOfSquaredDeviations.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/SumOfSquaredDeviations.scala @@ -5,15 +5,16 @@ import squants.Quantity import java.time.Instant final case class SumOfSquaredDeviations( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { override def getDependents = { if (this.dependentAggregations.isEmpty) @@ -22,8 +23,11 @@ final case class SumOfSquaredDeviations( } // see https://www.johndcook.com/blog/standard_deviation/ - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { - val q = quantity in current.unit + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { + val q = quantity in current.unit val currentMean = q.unit(getDependents("Mean").value) val updatedMean = q.unit(depAggs("Mean").value) current + (q - currentMean) * (q - updatedMean).value diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Variance.scala b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Variance.scala index 2f9663e..2186815 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Variance.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/aggregate/Variance.scala @@ -5,24 +5,31 @@ import squants.Quantity import java.time.Instant final case class Variance( - dimension: String, - unit: String, - value: Double = 0d, - count: BigInt = BigInt(0), - aggregatedLastUpdated: Instant = Instant.EPOCH, - lastUpdated: Instant = Instant.now(), - dependentAggregations: Map[String, Aggregate] = Map.empty[String, Aggregate], - params: Map[String, String] = Map.empty[String, String]) - extends Aggregate { + dimension: String, + unit: String, + value: Double = 0d, + count: BigInt = BigInt(0), + aggregatedLastUpdated: Instant = Instant.EPOCH, + lastUpdated: Instant = Instant.now(), + dependentAggregations: Map[String, Aggregate] = + Map.empty[String, Aggregate], + params: Map[String, String] = Map.empty[String, String]) + extends Aggregate { override def getDependents = { if (this.dependentAggregations.isEmpty) - Map("SumOfSquaredDeviations" -> SumOfSquaredDeviations(dimension, unit)) + Map( + "SumOfSquaredDeviations" -> SumOfSquaredDeviations(dimension, unit) + ) else this.dependentAggregations } - override def updateQuantity[A <: Quantity[A]](current: A, quantity: A, depAggs: Map[String, Aggregate]) = { - if (count < 2) current else { + override def updateQuantity[A <: Quantity[A]]( + current: A, + quantity: A, + depAggs: Map[String, Aggregate]) = { + if (count < 2) current + else { val k = count.doubleValue() val s = current.unit(depAggs("SumOfSquaredDeviations").value) s / k diff --git a/src/main/scala/io/epiphanous/flinkrunner/operator/AddToJdbcBatchFunction.scala b/src/main/scala/io/epiphanous/flinkrunner/operator/AddToJdbcBatchFunction.scala index 85758bf..b546699 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/operator/AddToJdbcBatchFunction.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/operator/AddToJdbcBatchFunction.scala @@ -1,10 +1,8 @@ package io.epiphanous.flinkrunner.operator -import io.epiphanous.flinkrunner.model.FlinkEvent - import java.sql.PreparedStatement -abstract class AddToJdbcBatchFunction[E <: FlinkEvent] { +abstract class AddToJdbcBatchFunction[E] { def addToJdbcStatement(row: E, ps: PreparedStatement): Unit diff --git a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala index a3a85b8..8c54587 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala @@ -4,8 +4,8 @@ import cats.effect.{ContextShift, IO, Timer} import com.google.common.cache.{CacheBuilder, CacheLoader} import com.typesafe.scalalogging.LazyLogging import io.circe.Decoder -import io.epiphanous.flinkrunner.model.FlinkConfig -import org.apache.flink.runtime.concurrent.Executors.directExecutionContext +import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} +import org.apache.flink.util.concurrent.Executors import org.apache.flink.streaming.api.scala.async.{ AsyncFunction, ResultFuture @@ -58,11 +58,16 @@ import scala.util.{Failure, Success, Try} * @tparam CV * the cache value type */ -abstract class EnrichmentAsyncFunction[IN, OUT, CV <: AnyRef]( +abstract class EnrichmentAsyncFunction[ + IN, + OUT, + CV <: AnyRef, + ADT <: FlinkEvent]( configPrefix: String, cacheLoaderOpt: Option[CacheLoader[String, Option[CV]]] = None, - preloaded: Map[String, CV] = Map.empty[String, CV] -)(implicit config: FlinkConfig, decoder: Decoder[CV]) + preloaded: Map[String, CV] = Map.empty[String, CV], + config: FlinkConfig[ADT] +)(implicit decoder: Decoder[CV]) extends AsyncFunction[IN, OUT] with LazyLogging { @@ -70,7 +75,8 @@ abstract class EnrichmentAsyncFunction[IN, OUT, CV <: AnyRef]( lazy implicit val entityDecoder: EntityDecoder[IO, CV] = jsonOf[IO, CV] @transient - lazy implicit val ec: ExecutionContext = directExecutionContext() + lazy implicit val ec: ExecutionContext = + ExecutionContext.fromExecutor(Executors.directExecutor()) @transient lazy implicit val cs: ContextShift[IO] = IO.contextShift(ec) diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala new file mode 100644 index 0000000..a7f0d59 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala @@ -0,0 +1,72 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.circe.Decoder +import io.circe.parser._ +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + SourceConfig +} +import org.apache.flink.api.common.serialization.DeserializationSchema +import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} + +import java.nio.charset.StandardCharsets + +/** + * @param sourceName + * the name of the source we are deserializing from + * @param config + * flink runner configuration + * @tparam ADT + * the algebraic data type of our events + */ +class CirceJsonDeserializationSchema[E <: ADT, ADT <: FlinkEvent]( + sourceName: String, + config: FlinkConfig[ADT])(implicit + circeDecoder: Decoder[E], + ev: Null <:< E) + extends DeserializationSchema[E] + with LazyLogging { + + val sourceConfig: SourceConfig = config.getSourceConfig(sourceName) + + /** + * Deserialize a json byte array into an ADT event instance or return + * null if the byte array can't be successfully deserialized + * @param bytes + * a json-encoded byte array + * @return + * an instance of an ADT event type + */ + override def deserialize(bytes: Array[Byte]): E = { + val payload = new String(bytes, StandardCharsets.UTF_8) + decode[E](payload).toOption match { + case Some(event) => event + case other => + logger.error( + s"Failed to deserialize JSON payload from source $sourceName: $payload" + ) + other.orNull + } + } + + /** + * Determine if the next event is the end of the stream or not. We always + * return false since we assume the stream never ends. + * @param nextEvent + * the next event + * @return + * false + */ + override def isEndOfStream(nextEvent: E): Boolean = false + + /** + * Compute the produced type when deserializing a byte array + * @return + * TypeInformation[E] + */ + override def getProducedType: TypeInformation[E] = + TypeInformation.of(new TypeHint[E] {}) + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala new file mode 100644 index 0000000..48b8796 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala @@ -0,0 +1,74 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.circe.Encoder +import io.circe.syntax._ +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + SinkConfig +} +import org.apache.flink.api.common.serialization.SerializationSchema + +import java.nio.charset.StandardCharsets + +/** + * A JSON serialization schema that uses the circe json library. + * + * @param sinkName + * name of the sink we're serializing to + * @param config + * a flink runner config + * @param circeEncoder + * an implicit circe encoder + * @tparam E + * the ADT member type we're serializing + * @tparam ADT + * the flink runner ADT + */ +class CirceJsonSerializationSchema[E <: ADT, ADT <: FlinkEvent]( + sinkName: String, + config: FlinkConfig[ADT])(implicit circeEncoder: Encoder[E]) + extends SerializationSchema[E] + with LazyLogging { + + val sourceConfig: SinkConfig = config.getSinkConfig(sinkName) + val configPretty: Boolean = + sourceConfig.properties.getProperty("pretty", "false").toBoolean + val configSort: Boolean = + sourceConfig.properties.getProperty("sort", "false").toBoolean + + /** + * Serialize an ADT event into json byte array + * @param event + * an instance of an ADT event type + * @return + * a json encoded byte array + */ + override def serialize(event: E): Array[Byte] = + toJson(event).getBytes(StandardCharsets.UTF_8) + + /** + * Utility method to convert an event into a JSON string with options for + * pretty-printing and sorting keys + * @param event + * the ADT event instance to encode + * @param pretty + * true to encode with lines and 2 space indentation + * @param sortKeys + * true to sort the json keys + * @return + * a json-encoded string + */ + def toJson( + event: E, + pretty: Boolean = configPretty, + sortKeys: Boolean = configSort): String = { + val j = event.asJson + if (pretty) { + if (sortKeys) j.spaces2SortKeys else j.spaces2 + } else { + if (sortKeys) j.noSpacesSortKeys else j.noSpaces + } + } +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala new file mode 100644 index 0000000..e9b8cf6 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -0,0 +1,101 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient +import io.confluent.kafka.serializers.KafkaAvroDeserializer +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkConnectorName, + FlinkEvent, + KafkaSourceConfig +} +import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema +import org.apache.flink.util.Collector +import org.apache.kafka.clients.consumer.ConsumerRecord + +import java.util + +/** + * A schema to deserialize bytes from kafka into an ADT event using a + * confluent avro schema registry. + * + * @param sourceName + * name of the source stream + * @param config + * flink runner config + * @tparam E + * the event type we are producing here, which is a member of the ADT + * @tparam ADT + * the flink runner ADT + */ +abstract class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ + E <: ADT, + ADT <: FlinkEvent +]( + sourceName: String, + config: FlinkConfig[ADT] +) extends KafkaRecordDeserializationSchema[E] + with LazyLogging { + + val sourceConfig: KafkaSourceConfig = { + val sc = config.getSourceConfig(sourceName) + if (sc.connector != FlinkConnectorName.Kafka) + throw new RuntimeException( + s"Requested source $sourceName is not a kafka source" + ) + sc.asInstanceOf[KafkaSourceConfig] + } + + val schemaRegistryProps: util.HashMap[String, String] = + sourceConfig.propertiesMap + + val topic: String = sourceConfig.topic + + /** + * Implementing subclasses must provide an instance of a schema registry + * client to use, for instance a CachedSchemaRegistryClient + * or a MockSchemaRegistryClient for testing. + */ + def schemaRegistryClient: SchemaRegistryClient + + val valueDeserializer = new KafkaAvroDeserializer( + schemaRegistryClient, + schemaRegistryProps + ) + + val keyDeserializer: Option[KafkaAvroDeserializer] = + if (sourceConfig.isKeyed) { + val ks = new KafkaAvroDeserializer(schemaRegistryClient) + ks.configure(schemaRegistryProps, true) + Some(ks) + } else None + + /** + * Convert a deserialized key/value pair of objects into an instance of + * the flink runner ADT. This method must be implemented by subclasses. + * + * The key and value are passed as AnyRefs, so implementing subclasses + * will need to pattern match. + * + * @param key + * an optional deserialized key object + * @param value + * a deserialized value object + * @return + * an instance of the flink runner ADT + */ + def fromKeyValue(key: Option[AnyRef], value: AnyRef): E + + override def deserialize( + record: ConsumerRecord[Array[Byte], Array[Byte]], + out: Collector[E]): Unit = { + val key = + keyDeserializer.map(ds => ds.deserialize(topic, record.key())) + val value = valueDeserializer.deserialize(topic, record.value()) + if (Option(value).nonEmpty) out.collect(fromKeyValue(key, value)) + } + + override def getProducedType: TypeInformation[E] = + TypeInformation.of(new TypeHint[E] {}) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala new file mode 100644 index 0000000..f39007a --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -0,0 +1,125 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient +import io.confluent.kafka.serializers.KafkaAvroSerializer +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkConnectorName, + FlinkEvent, + KafkaSinkConfig +} +import org.apache.avro.specific.SpecificRecord +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema +import org.apache.kafka.clients.producer.ProducerRecord + +import java.{lang, util} + +/** + * A schema to serialize an ADT event using a confluent avro schema + * registry. An implementing class must provide a Flink + * [[ConfluentRegistryAvroSerializationSchema]] to interface with the + * schema registry. That registry is specific to a type that implements + * Avro's [[SpecificRecord]] interface type. + * @param sinkName + * name of the sink stream + * @param config + * flink runner config + * @tparam E + * the event type we are serializing from, which is a member of the ADT + * @tparam ADT + * the flink runner ADT + */ +abstract class ConfluentAvroRegistryKafkaRecordSerializationSchema[ + E <: ADT, + ADT <: FlinkEvent]( + sinkName: String, + config: FlinkConfig[ADT] +) extends KafkaRecordSerializationSchema[E] + with LazyLogging { + + val sinkConfig: KafkaSinkConfig = { + val sc = config.getSinkConfig(sinkName) + if (sc.connector != FlinkConnectorName.Kafka) + throw new RuntimeException( + s"Requested sink $sinkName is not a kafka sink" + ) + sc.asInstanceOf[KafkaSinkConfig] + } + + val schemaRegistryProps: util.HashMap[String, String] = + sinkConfig.propertiesMap + + /** map to store the value, and optionally, key serializers */ + val valueSerializer = new KafkaAvroSerializer( + schemaRegistryClient, + schemaRegistryProps + ) + + /** add the key serializer if needed */ + val keySerializer: Option[KafkaAvroSerializer] = + if (sinkConfig.isKeyed) { + val ks = new KafkaAvroSerializer(schemaRegistryClient) + ks.configure(schemaRegistryProps, true) + Some(ks) + } else None + + val topic: String = sinkConfig.topic + + /** + * A helper method to serialize an arbitary key/value pair. This should + * be used by subclasses that implement the [[toKeyValue()]] method. + * + * @param key + * the key + * @param value + * the value + * @tparam K + * the type of key + * @tparam V + * the type of value + * @return + * a tuple of byte arrays (with the key optional) + */ +// def kvSerialize[K, V](key: K, value: V): (Array[Byte], Array[Byte]) = { +// ( +// keySerializer.map(s => s.serialize(topic, key)).orNull, +// valueSerializer.serialize(topic, value) +// ) +// } + + /** + * Implementing subclasses must provide an instance of a schema registry + * client to use, for instance a CachedSchemaRegistryClient + * or a MockSchemaRegistryClient for testing. + */ + def schemaRegistryClient: SchemaRegistryClient + + /** + * Convert a flink runner ADT instance into a key/value pair of objects + * to serialize into a kafka message. This must be defined by + * implementing subclasses. + * + * The purpose of this method is to decouple the structure of the flink + * runner ADT from the avro schemas of the underlying kafka messages. + * + * @param element + * an instance of the flinkrunner ADT + * @return + * (Option[AnyRef], AnyRef) + */ + def toKeyValue(element: E): (Option[AnyRef], AnyRef) + + override def serialize( + element: E, + context: KafkaRecordSerializationSchema.KafkaSinkContext, + timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = { + val (k, v) = toKeyValue(element) + val key = + keySerializer.flatMap(ks => k.map(kk => ks.serialize(topic, kk))) + val value = valueSerializer.serialize(topic, v) + new ProducerRecord(topic, null, element.$timestamp, key.orNull, value) + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/BoundedLatenessWatermarkStrategy.scala b/src/main/scala/io/epiphanous/flinkrunner/util/BoundedLatenessWatermarkStrategy.scala index 951c3d8..ee18fd8 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/BoundedLatenessWatermarkStrategy.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/BoundedLatenessWatermarkStrategy.scala @@ -2,16 +2,17 @@ package io.epiphanous.flinkrunner.util import io.epiphanous.flinkrunner.model.FlinkEvent import org.apache.flink.api.common.eventtime.{ - WatermarkGenerator, WatermarkGeneratorSupplier, WatermarkStrategy } +import java.time.Duration + class BoundedLatenessWatermarkStrategy[E <: FlinkEvent]( - val maxAllowedLateness: Long, + val maxAllowedLateness: Duration, val streamID: String) extends WatermarkStrategy[E] { override def createWatermarkGenerator( - context: WatermarkGeneratorSupplier.Context): WatermarkGenerator[E] = - new BoundedLatenessGenerator[E](maxAllowedLateness, streamID) + context: WatermarkGeneratorSupplier.Context) = + new BoundedLatenessGenerator(maxAllowedLateness.toMillis, streamID) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala b/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala index 8075d88..bfa4657 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala @@ -1,7 +1,7 @@ package io.epiphanous.flinkrunner.util import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.FlinkEvent +import io.epiphanous.flinkrunner.model.JdbcSinkConfig import io.epiphanous.flinkrunner.operator.AddToJdbcBatchFunction import org.apache.flink.api.common.state.{ListState, ListStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation @@ -16,7 +16,6 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction.Context import org.apache.flink.streaming.api.scala._ import java.sql.{Connection, DriverManager, PreparedStatement} -import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import scala.util.{Failure, Success, Try} @@ -37,13 +36,15 @@ import scala.util.{Failure, Success, Try} * @tparam E * the class of sink elements. */ -class JdbcSink[E <: FlinkEvent: TypeInformation]( - batchFunction: AddToJdbcBatchFunction[E], - props: Properties) - extends RichSinkFunction[E] +class JdbcSink[E: TypeInformation]( + sinkConfig: JdbcSinkConfig, + batchFunction: AddToJdbcBatchFunction[E] +) extends RichSinkFunction[E] with CheckpointedFunction with LazyLogging { + val props = sinkConfig.properties + val bufferSize = props.getProperty("buffer.size").toInt private val pendingRows = ListBuffer.empty[E] private var connection: Connection = _ diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala index 6ae444e..3a4a810 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala @@ -1,64 +1,9 @@ package io.epiphanous.flinkrunner.util import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.SEE -import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.operator.AddToJdbcBatchFunction -import org.apache.flink.api.common.eventtime.WatermarkStrategy -import org.apache.flink.api.common.functions.RuntimeContext -import org.apache.flink.api.common.serialization.{ - DeserializationSchema, - Encoder, - SerializationSchema -} -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.core.fs.Path -import org.apache.flink.streaming.api.TimeCharacteristic -import org.apache.flink.streaming.api.datastream.DataStreamSink -import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.{ - BasePathBucketAssigner, - DateTimeBucketAssigner -} -import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.{ - DefaultRollingPolicy, - OnCheckpointRollingPolicy -} -import org.apache.flink.streaming.api.functions.sink.filesystem.{ - BucketAssigner, - StreamingFileSink -} -import org.apache.flink.streaming.api.scala._ -import org.apache.flink.streaming.connectors.cassandra.CassandraSink -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic -import org.apache.flink.streaming.connectors.kafka.{ - FlinkKafkaConsumer, - FlinkKafkaProducer, - KafkaDeserializationSchema, - KafkaSerializationSchema -} -import org.apache.flink.streaming.connectors.kinesis.serialization.{ - KinesisDeserializationSchema, - KinesisSerializationSchema -} -import org.apache.flink.streaming.connectors.kinesis.{ - FlinkKinesisConsumer, - FlinkKinesisProducer -} -import org.apache.http.HttpHost -import org.elasticsearch.client.Requests - -import java.io.{File, FileNotFoundException} -import java.net.URL -import java.nio.charset.StandardCharsets -import scala.collection.JavaConverters._ -import scala.util.matching.Regex object StreamUtils extends LazyLogging { - val RESOURCE_PATTERN: Regex = "resource://(.*)".r - /** * A little syntactic sugar for writing stream program. This is the pipe * operator, ala F#. @@ -91,619 +36,4 @@ object StreamUtils extends LazyLogging { } } - /** - * Generates a timestamp and watermark assigner for a stream with a given - * type of element that limits how late an element is allowed to arrive - * in event time. - * - * @param config - * implicitly provided job config - * @tparam E - * the type of stream element - * @return - * BoundedLatenessGenerator[E] - */ - def boundedLatenessWatermarks[E <: FlinkEvent: TypeInformation]( - streamID: String - )(implicit config: FlinkConfig) = - new BoundedLatenessWatermarkStrategy[E]( - config.maxLateness.toMillis, - streamID - ) - - /** - * Create a bounded of order watermark strategy with idleness checking - * - * @param config - * implicitly provided job config - * @tparam E - * the type of stream element - * @return - * BoundedLatenessGenerator[E] - */ - def boundedOutofOrdernessWatermarks[E <: FlinkEvent: TypeInformation]()( - implicit config: FlinkConfig): WatermarkStrategy[E] = - WatermarkStrategy - .forBoundedOutOfOrderness(config.maxLateness) - .withIdleness(config.maxIdleness) - - /** - * Creates an ascending timestamp watermark strategy. - * @tparam E - * type of stream element - * @return - * AscendingTimestampExtractor[E] - */ - def ascendingTimestampsWatermarks[E <: FlinkEvent: TypeInformation]() - : WatermarkStrategy[E] = WatermarkStrategy.forMonotonousTimestamps() - - /** - * Assign timestamps/watermarks if we're using event time - * @param in - * the input stream to watermark - * @param config - * implicit flink configuration - * @param env - * implicit stream execution environment - * @tparam E - * event type - * @return - * the possibly watermarked input stream - */ - def maybeAssignTimestampsAndWatermarks[E <: FlinkEvent: TypeInformation]( - in: DataStream[E], - srcConfig: SourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = - if (srcConfig.timeCharacteristic == TimeCharacteristic.EventTime) { - in.assignTimestampsAndWatermarks(srcConfig.watermarkStrategy match { - case "bounded out of orderness" => - boundedOutofOrdernessWatermarks() - case "ascending timestamps" => ascendingTimestampsWatermarks() - case _ => boundedLatenessWatermarks(in.name) - }).name(s"wm:${in.name}") - .uid(s"wm:${in.name}") - } else in - - /** - * Configure stream source from configuration. - * - * @param sourceName - * the name of the source to get its configuration - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromSource[E <: FlinkEvent: TypeInformation]( - sourceName: String = "" - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = { - val name = - if (sourceName.isEmpty) config.getSourceNames.head else sourceName - val src = config.getSourceConfig(name) - val uid = src.label - val stream = (src match { - case src: KafkaSourceConfig => fromKafka(src) - case src: KinesisSourceConfig => fromKinesis(src) - case src: FileSourceConfig => fromFile(src) - case src: SocketSourceConfig => fromSocket(src) - case src: CollectionSourceConfig => fromCollection(src) - case src => - throw new IllegalArgumentException( - s"unsupported source connector: ${src.connector}" - ) - }).name(uid).uid(uid) - maybeAssignTimestampsAndWatermarks(stream, src) - } - - /** - * Configure stream from kafka source. - * - * @param srcConfig - * a source config - * @param config - * implicitly provided job config - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromKafka[E <: FlinkEvent: TypeInformation]( - srcConfig: KafkaSourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = { - val consumer = - new FlinkKafkaConsumer[E]( - srcConfig.topic, - config - .getKafkaDeserializationSchema(srcConfig.name) - .asInstanceOf[KafkaDeserializationSchema[E]], - srcConfig.properties - ) - env - .addSource(consumer) - } - - /** - * Configure stream from kinesis. - * - * @param srcConfig - * a source config - * @param config - * implicitly provided job config - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromKinesis[E <: FlinkEvent: TypeInformation]( - srcConfig: KinesisSourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = { - val consumer = - new FlinkKinesisConsumer[E]( - srcConfig.stream, - config - .getKinesisDeserializationSchema(srcConfig.name) - .asInstanceOf[KinesisDeserializationSchema[E]], - srcConfig.properties - ) - env - .addSource(consumer) - .name(srcConfig.label) - } - - /** - * Configure stream from file source. - * - * @param srcConfig - * a source config - * @param config - * implicitly provided job config - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromFile[E <: FlinkEvent: TypeInformation]( - srcConfig: FileSourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = { - val path = srcConfig.path match { - case RESOURCE_PATTERN(p) => getSourceFilePath(p) - case other => other - } - val ds = config - .getDeserializationSchema(srcConfig.name) - .asInstanceOf[DeserializationSchema[E]] - env - .readTextFile(path) - .name(s"raw:${srcConfig.label}") - .uid(s"raw:${srcConfig.label}") - .map(line => ds.deserialize(line.getBytes(StandardCharsets.UTF_8))) - } - - /** - * Configure stream from socket source. - * - * @param srcConfig - * a source config - * @param config - * implicitly provided job config - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromSocket[E <: FlinkEvent: TypeInformation]( - srcConfig: SocketSourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = - env - .socketTextStream(srcConfig.host, srcConfig.port) - .name(s"raw:${srcConfig.label}") - .uid(s"raw:${srcConfig.label}") - .map(line => - config - .getDeserializationSchema(srcConfig.name) - .asInstanceOf[DeserializationSchema[E]] - .deserialize(line.getBytes(StandardCharsets.UTF_8)) - ) - - /** - * Configure stream from collection source. - * - * @param srcConfig - * a source config - * @param config - * implicitly provided job config - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def fromCollection[E <: FlinkEvent: TypeInformation]( - srcConfig: CollectionSourceConfig - )(implicit config: FlinkConfig, env: SEE): DataStream[E] = - env - .fromCollection[Array[Byte]]( - config.getCollectionSource(srcConfig.topic) - ) - .name(s"raw:${srcConfig.label}") - .uid(s"raw:${srcConfig.label}") - .map(bytes => - config - .getDeserializationSchema(srcConfig.name) - .asInstanceOf[DeserializationSchema[E]] - .deserialize(bytes) - ) - - /** - * Returns the actual path to a resource file named filename or - * filename.gz. - * - * @param filename - * the name of file - * @return - * String - */ - @throws[FileNotFoundException] - def getSourceFilePath(filename: String): String = { - val loader = getClass - val resource = Option(loader.getResource(filename)) match { - case Some(value) => value.toURI - case None => - Option(loader.getResource(s"$filename.gz")) match { - case Some(value) => value.toURI - case None => - throw new FileNotFoundException( - s"can't load resource $filename" - ) - } - } - val file = new File(resource) - file.getAbsolutePath - } - - implicit class EventStreamOps[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E]) { - - def as[T <: FlinkEvent: TypeInformation]: DataStream[T] = { - val name = stream.name - stream - .filter((e: E) => e.isInstanceOf[T @unchecked]) - .name(s"filter types $name") - .uid(s"filter types $name") - .map((e: E) => e.asInstanceOf[T @unchecked]) - .name(s"cast types $name") - .uid(s"cast types $name") - } - - def toSink(sinkName: String = "")(implicit config: FlinkConfig) = - StreamUtils.toSink[E](stream, sinkName) - - } - - /** - * Configure stream sink from configuration. - * - * @param stream - * the data stream to send to sink - * @param sinkName - * a sink name to obtain configuration - * @param config - * implicit flink job args - * @tparam E - * stream element type - * @return - * DataStream[E] - */ - def toSink[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkName: String = "" - )(implicit config: FlinkConfig) = { - val name = if (sinkName.isEmpty) config.getSinkNames.head else sinkName - config.getSinkConfig(name) match { - case s: KafkaSinkConfig => toKafka[E](stream, s) - case s: KinesisSinkConfig => toKinesis[E](stream, s) - case s: FileSinkConfig => toFile[E](stream, s) - case s: SocketSinkConfig => toSocket[E](stream, s) - case s: JdbcSinkConfig => toJdbc[E](stream, s) - case s: CassandraSinkConfig => toCassandraSink[E](stream, s) - case s: ElasticsearchSinkConfig => toElasticsearchSink[E](stream, s) - case s => - throw new IllegalArgumentException( - s"unsupported source connector: ${s.connector}" - ) - } - } - - /** - * Send stream to a kafka sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @param config - * implicit job args - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toKafka[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: KafkaSinkConfig - )(implicit config: FlinkConfig): DataStreamSink[E] = - stream - .addSink( - new FlinkKafkaProducer[E]( - sinkConfig.topic, - config - .getKafkaSerializationSchema(sinkConfig.name) - .asInstanceOf[KafkaSerializationSchema[E]], - sinkConfig.properties, - Semantic.AT_LEAST_ONCE - ) - ) - .uid(sinkConfig.label) - .name(sinkConfig.label) - - /** - * Send stream to a kinesis sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @param config - * implicit job args - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toKinesis[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: KinesisSinkConfig - )(implicit config: FlinkConfig): DataStreamSink[E] = - stream - .addSink { - val sink = - new FlinkKinesisProducer[E]( - config - .getKinesisSerializationSchema(sinkConfig.name) - .asInstanceOf[KinesisSerializationSchema[E]], - sinkConfig.properties - ) - sink.setDefaultStream(sinkConfig.stream) - sink.setFailOnError(true) - sink.setDefaultPartition("0") - sink - } - .uid(sinkConfig.label) - .name(sinkConfig.label) - - /** - * Send stream to a socket sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @param config - * implicit job args - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toJdbc[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: JdbcSinkConfig - )(implicit config: FlinkConfig): DataStreamSink[E] = - stream - .addSink( - new JdbcSink( - config - .getAddToJdbcBatchFunction(sinkConfig.name) - .asInstanceOf[AddToJdbcBatchFunction[E]], - sinkConfig.properties - ) - ) - .uid(sinkConfig.label) - .name(sinkConfig.label) - - /** - * Send stream to a rolling file sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @param config - * implicit job args - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toFile[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: FileSinkConfig - )(implicit config: FlinkConfig): DataStreamSink[E] = { - val path = sinkConfig.path - val p = sinkConfig.properties - val bucketCheckInterval = - p.getProperty("bucket.check.interval", s"${60000}").toLong - val bucketAssigner = - p.getProperty("bucket.assigner.type", "datetime") match { - case "none" => new BasePathBucketAssigner[E]() - case "datetime" => - new DateTimeBucketAssigner[E]( - p.getProperty( - "bucket.assigner.datetime.format", - "YYYY/MM/DD/HH" - ) - ) - case "custom" => - config - .getBucketAssigner(sinkConfig.name) - .asInstanceOf[BucketAssigner[E, String]] - case other => - throw new IllegalArgumentException( - s"Unknown bucket assigner type '$other'." - ) - } - val encoderFormat = p.getProperty("encoder.format", "row") - val sink = encoderFormat match { - case "row" => - val builder = - StreamingFileSink.forRowFormat( - new Path(path), - config.getEncoder(sinkConfig.name).asInstanceOf[Encoder[E]] - ) - val rollingPolicy = - p.getProperty("bucket.rolling.policy", "default") match { - case "default" => - DefaultRollingPolicy - .builder() - .withInactivityInterval( - p.getProperty( - "bucket.rolling.policy.inactivity.interval", - s"${60000}" - ).toLong - ) - .withMaxPartSize( - p.getProperty( - "bucket.rolling.policy.max.part.size", - s"${128 * 1024 * 1024}" - ).toLong - ) - .withRolloverInterval( - p.getProperty( - "bucket.rolling.policy.rollover.interval", - s"${Long.MaxValue}" - ).toLong - ) - .build[E, String]() - case "checkpoint" => - OnCheckpointRollingPolicy.build[E, String]() - case policy => - throw new IllegalArgumentException( - s"Unknown bucket rolling policy type: '$policy'" - ) - } - builder - .withBucketAssigner(bucketAssigner) - .withRollingPolicy(rollingPolicy) - .withBucketCheckInterval(bucketCheckInterval) - .build() - case "bulk" => - throw new NotImplementedError("Bulk file sink not implemented yet") - - case _ => - throw new IllegalArgumentException( - s"Unknown file sink encoder format: '$encoderFormat'" - ) - } - stream.addSink(sink).uid(sinkConfig.label).name(sinkConfig.label) - } - - /** - * Send stream to a socket sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @param config - * implicit job args - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toSocket[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: SocketSinkConfig - )(implicit config: FlinkConfig): DataStreamSink[E] = - stream - .writeToSocket( - sinkConfig.host, - sinkConfig.port, - config - .getSerializationSchema(sinkConfig.name) - .asInstanceOf[SerializationSchema[E]] - ) - .uid(sinkConfig.label) - .name(sinkConfig.label) - - /** - * Send stream to a cassandra sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toCassandraSink[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: CassandraSinkConfig) = - CassandraSink - .addSink(stream) - .setHost(sinkConfig.host) - .setQuery(sinkConfig.query) - .build() - .uid(sinkConfig.label) - .name(sinkConfig.label) - - /** - * Send stream to an elasticsearch sink. - * - * @param stream - * the data stream - * @param sinkConfig - * a sink configuration - * @tparam E - * stream element type - * @return - * DataStreamSink[E] - */ - def toElasticsearchSink[E <: FlinkEvent: TypeInformation]( - stream: DataStream[E], - sinkConfig: ElasticsearchSinkConfig - ): DataStreamSink[E] = { - val hosts = sinkConfig.transports.map { s => - val url = new URL(if (s.startsWith("http")) s else s"http://$s") - val hostname = url.getHost - val port = if (url.getPort < 0) 9200 else url.getPort - new HttpHost(hostname, port, url.getProtocol) - }.asJava - val esSink = new ElasticsearchSink.Builder[E]( - hosts, - (element: E, _: RuntimeContext, indexer: RequestIndexer) => { - val data = element.getClass.getDeclaredFields - .filterNot(f => - Seq("$id", "$key", "$timestamp", "$action").contains( - f.getName - ) - ) - .foldLeft(Map.empty[String, Any]) { case (a, f) => - f.setAccessible(true) - val name = f.getName - f.get(element) match { - case Some(v: Any) => a + (name -> v) - case None => a - case v: Any => a + (name -> v) - } - } - .asJava - val req = Requests.indexRequest(sinkConfig.index).source(data) - indexer.add(req) - } - ).build() - stream.addSink(esSink).uid(sinkConfig.label).name(sinkConfig.label) - } - } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala new file mode 100644 index 0000000..99dce01 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala @@ -0,0 +1,120 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.avro.specific.SpecificRecord +import org.apache.avro.{AvroRuntimeException, Schema} + +import java.time.Instant + +sealed trait MyAvroADT extends FlinkEvent + +case class AWrapper(value: ARecord) extends MyAvroADT { + override val $id: String = value.a0 + override val $key: String = $id + override val $timestamp: Long = value.a3.toEpochMilli +} + +case class BWrapper(value: BRecord) extends MyAvroADT { + override val $id: String = value.b0 + override val $key: String = $id + override val $timestamp: Long = value.b3.toEpochMilli +} + +case class ARecord( + var a0: String, + var a1: Int, + var a2: Double, + var a3: Instant) + extends SpecificRecord { + override def put(i: Int, v: Any): Unit = { + (i, v) match { + case (0, x: String) => this.a0 = x + case (1, x: Int) => this.a1 = x + case (2, x: Double) => this.a2 = x + case (3, x: Long) => this.a3 = Instant.ofEpochMilli(x) + case _ => + if (i < 0 || i > 3) new AvroRuntimeException("Bad index") + else new AvroRuntimeException("Bad value") + } + } + + override def get(i: Int): AnyRef = i match { + case 0 => a0.asInstanceOf[AnyRef] + case 1 => a1.asInstanceOf[AnyRef] + case 2 => a2.asInstanceOf[AnyRef] + case 3 => a3.toEpochMilli.asInstanceOf[AnyRef] + case _ => new AvroRuntimeException("Bad index") + } + + override def getSchema: Schema = ARecord.SCHEMA$ +} +object ARecord { + val schemaString: String = + """ + |{ + | "type": "record", + | "name": "ARecord", + | "namespace": "io.epiphanous.flinkrunner.model", + | "fields": [ + | { "name": "a0", "type": "string" }, + | { "name": "a1", "type": "int" }, + | { "name": "a2", "type": "double" }, + | { "name": "a3", "type": "long", "logicalType": "time-millis" } + | ] + |}""".stripMargin + val SCHEMA$ : Schema = new Schema.Parser().parse(schemaString) +} + +case class BRecord( + var b0: String, + var b1: Option[Int], + var b2: Option[Double], + var b3: Instant) + extends SpecificRecord { + override def put(i: Int, v: Any): Unit = { + (i, v) match { + case (0, x: String) => this.b0 = x + case (1, x: Int) => this.b1 = Some(x) + case (1, _) => this.b1 = null + case (2, x: Double) => this.b2 = Some(x) + case (2, _) => this.b2 = null + case (3, x: Long) => this.b3 = Instant.ofEpochMilli(x) + case _ => + if (i < 0 || i > 3) new AvroRuntimeException("Bad index") + else new AvroRuntimeException("Bad value") + } + } + + override def get(i: Int): AnyRef = i match { + case 0 => b0.asInstanceOf[AnyRef] + case 1 => + (b1 match { + case Some(x) => x + case None => null + }).asInstanceOf[AnyRef] + case 2 => + (b2 match { + case Some(x) => x + case None => null + }).asInstanceOf[AnyRef] + case 3 => b3.toEpochMilli.asInstanceOf[AnyRef] + case _ => new AvroRuntimeException("Bad index") + } + + override def getSchema: Schema = BRecord.SCHEMA$ +} +object BRecord { + val schemaString: String = + """ + |{ + | "type": "record", + | "name": "BRecord", + | "namespace": "io.epiphanous.flinkrunner.model", + | "fields": [ + | { "name": "b0", "type": "string" }, + | { "name": "b1", "type": ["null", "int"] }, + | { "name": "b2", "type": ["null", "double"] }, + | { "name": "b3", "type": "long", "logicalType": "time-millis" } + | ] + |}""".stripMargin + val SCHEMA$ : Schema = new Schema.Parser().parse(schemaString) +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/NoJobFactory.scala b/src/test/scala/io/epiphanous/flinkrunner/model/NoJobFactory.scala new file mode 100644 index 0000000..e9fbac1 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/NoJobFactory.scala @@ -0,0 +1,10 @@ +package io.epiphanous.flinkrunner.model + +import io.epiphanous.flinkrunner.flink.BaseFlinkJob +import io.epiphanous.flinkrunner.{FlinkRunner, FlinkRunnerFactory} + +class NoJobFactory[ADT <: FlinkEvent] extends FlinkRunnerFactory[ADT] { + override def getJobInstance[DS, OUT <: ADT]( + name: String, + runner: FlinkRunner[ADT]): BaseFlinkJob[DS, OUT, ADT] = ??? +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala new file mode 100644 index 0000000..f597218 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala @@ -0,0 +1,109 @@ +package io.epiphanous.flinkrunner.serde + +import io.confluent.kafka.schemaregistry.client.SchemaMetadata +import io.epiphanous.flinkrunner.UnitSpec +import io.epiphanous.flinkrunner.model._ +import org.apache.avro.Schema +import org.apache.avro.generic.GenericDatumWriter +import org.apache.avro.io.EncoderFactory + +import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.time.Instant + +class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest + extends UnitSpec { + val factory = new NoJobFactory[MyAvroADT] + val optConfig: String = + s""" + |sinks { + | test { + | connector = kafka + | topic = test + | isKeyed = true + | config { + | schema.registry.url = "mock://test" + | avro.use.logical.type.converters = true + | avro.remove.java.properties = true + | value.subject.name.strategy = io.confluent.kafka.serializers.subject.RecordNameStrategy + | } + | } + |} + |""".stripMargin + val config = new FlinkConfig[MyAvroADT]( + Array.empty[String], + factory, + Map.empty, + Some(optConfig) + ) + val serde = + new MyADTConfluentAvroRegistryKafkaRecordSerializationSchema[ + MyAvroADT]( + "test", // sink name must match this + config + ) + + // helper to return the class name of the object passed in (without a $ at the end) + def className[T](obj: T): String = { + obj.getClass.getName match { + case s if s.endsWith("$") => s.substring(0, s.length - 1) + case s => s + } + } + + // printout a byte array with a prefix + def showBytes(prefix: String, bytes: Array[Byte]): Unit = + println(s"$prefix: ${bytes.mkString("Array(", ", ", ")")}") + + // mimic the binary encoding used for schema registry encoded objects + def binaryEncode[T](obj: T, schemaInfo: SchemaMetadata): Array[Byte] = { + val schema = new Schema.Parser().parse(schemaInfo.getSchema) + val schemaId = schemaInfo.getId + val baos = new ByteArrayOutputStream() + baos.write(0) + val dos = new DataOutputStream(baos) + dos.writeInt(schemaId) + dos.flush() + val encoder = EncoderFactory.get().binaryEncoder(baos, null) + val datumWriter = new GenericDatumWriter[T](schema) + datumWriter.write(obj, encoder) + encoder.flush() + val bytes = baos.toByteArray + baos.close() + bytes + } + + // some test fixtures + val aRecord: ARecord = ARecord("a123", 17, 32.2, Instant.now()) + val aWrapper: AWrapper = AWrapper(aRecord) + val aName: String = className(aRecord) + val keySchemaInfo: SchemaMetadata = + serde.schemaRegistryClient.getLatestSchemaMetadata("test-key") + val aSchemaInfo: SchemaMetadata = + serde.schemaRegistryClient.getLatestSchemaMetadata( + aName + ) + + behavior of "ConfluentAvroSerializationSchema" + + it should "find the right schema for a key" in { + keySchemaInfo.getSchema shouldEqual "\"string\"" + } + + it should "find the right schema for a class" in { + aSchemaInfo.getSchema shouldEqual ARecord.SCHEMA$.toString + } + + it should "serialize to a producer record" in { + val (aKey, aValue) = serde.toKeyValue(aWrapper) + val aWrapperKeyExpectedBytes: Option[Array[Byte]] = + aKey.map(k => binaryEncode(k, keySchemaInfo)) + val aWrapperValueExpectedBytes: Array[Byte] = + binaryEncode(aValue, aSchemaInfo) + val result = serde.serialize(aWrapper, null, aWrapper.$timestamp) + result.key() shouldEqual aWrapperKeyExpectedBytes.value + result.value() shouldEqual aWrapperValueExpectedBytes + result.timestamp() shouldEqual aWrapper.$timestamp + result.topic() shouldEqual serde.topic + } + +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala new file mode 100644 index 0000000..9b2acc4 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -0,0 +1,69 @@ +package io.epiphanous.flinkrunner.serde + +import io.confluent.kafka.schemaregistry.avro.AvroSchema +import io.confluent.kafka.schemaregistry.client.{ + MockSchemaRegistryClient, + SchemaRegistryClient +} +import io.epiphanous.flinkrunner.model.{ + ARecord, + AWrapper, + BRecord, + BWrapper, + FlinkConfig, + MyAvroADT +} +import org.apache.avro.specific.SpecificRecord + +class MyADTConfluentAvroRegistryKafkaRecordSerializationSchema[ + E <: MyAvroADT](name: String, config: FlinkConfig[MyAvroADT]) + extends ConfluentAvroRegistryKafkaRecordSerializationSchema[ + E, + MyAvroADT]( + name, + config + ) { + + /** + * Implementing subclasses must provide an instance of a schema registry + * client to use, for instance a CachedSchemaRegistryClient + * or a MockSchemaRegistryClient for testing. + */ + override val schemaRegistryClient: SchemaRegistryClient = + new MockSchemaRegistryClient() + + // for testing purposes + val stringSchema: AvroSchema = new AvroSchema("""{"type":"string"}""") + val aRecordName: String = + ARecord.getClass.getCanonicalName.replaceAll("\\$$", "") + val bRecordName: String = + BRecord.getClass.getCanonicalName.replaceAll("\\$$", "") + schemaRegistryClient.register( + s"test-key", + stringSchema + ) + schemaRegistryClient.register( + aRecordName, + new AvroSchema(ARecord.SCHEMA$) + ) + schemaRegistryClient.register( + bRecordName, + new AvroSchema(BRecord.SCHEMA$) + ) + + /** + * Map a flinkrunner ADT instance into a key/value pair to serialize into + * kafka + * @param element + * an instance of the flinkrunner ADT + * @return + * (Option[AnyRef], AnyRef) + */ + override def toKeyValue(element: E): (Option[String], SpecificRecord) = + element match { + case a: AWrapper => + (Some(a.$id), a.value) + case b: BWrapper => + (Some(b.$id), b.value) + } +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala index 0448d76..27c459a 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala @@ -2,25 +2,30 @@ package io.epiphanous.flinkrunner.util import io.epiphanous.flinkrunner.UnitSpec import org.apache.flink.api.common.eventtime.{Watermark, WatermarkOutput} +import org.scalatest.Assertion import java.util.UUID.randomUUID import scala.util.Random class BoundedLatenessGeneratorTest extends UnitSpec { - val now = System.currentTimeMillis() - val random = new Random() + val now: Long = System.currentTimeMillis() + val random = new Random() def getBlg(maxAllowedLateness: Long = 10L, streamID: String = "Test") = - new BoundedLatenessGenerator[TestEvent](maxAllowedLateness, streamID) + new BoundedLatenessGenerator[TestEvent]( + maxAllowedLateness, + streamID + ) - def uuid = randomUUID().toString + def uuid: String = randomUUID().toString - def nextEvent(ts: Long) = TestEvent(uuid, ts) + def nextEvent(ts: Long): TestEvent = TestEvent(uuid, ts) - def ascending(space: Long = 5) = (prev: Long) => prev + space + def ascending(space: Long = 5): Long => Long = (prev: Long) => + prev + space - def randomWalk(minSpace: Long = -20, maxSpace: Long = 40) = + def randomWalk(minSpace: Long = -20, maxSpace: Long = 40): Long => Long = (prev: Long) => prev + Math.ceil(minSpace + random.nextDouble() * maxSpace).toLong @@ -29,7 +34,7 @@ class BoundedLatenessGeneratorTest extends UnitSpec { progress: Long => Long = randomWalk(), probSpike: Double = 0, spikeSize: Long = 50000 - ) = + ): Stream[TestEvent] = Stream .iterate((TestEvent(uuid, start), start)) { case (_, timeline) => val spike = if (random.nextDouble() < probSpike) spikeSize else 0L @@ -42,7 +47,7 @@ class BoundedLatenessGeneratorTest extends UnitSpec { start: Long = now - 1000L, space: Long = 5, probSpike: Double = 0, - spikeSize: Long = 50000) = + spikeSize: Long = 50000): Stream[TestEvent] = events(start, ascending(space), probSpike, spikeSize) def randomEvents( @@ -51,7 +56,7 @@ class BoundedLatenessGeneratorTest extends UnitSpec { maxSpace: Long = 40, probSpike: Double = 0, spikeSize: Long = 50000 - ) = + ): Stream[TestEvent] = events(start, randomWalk(minSpace, maxSpace), probSpike, spikeSize) def randomEventsWithSpike( @@ -60,7 +65,7 @@ class BoundedLatenessGeneratorTest extends UnitSpec { maxSpace: Long = 40, probSpike: Double = .20, spikeSize: Long = 50000 - ) = + ): Stream[TestEvent] = randomEvents(start, minSpace, maxSpace, probSpike, spikeSize) behavior of "BoundedLatenessGenerator" @@ -84,13 +89,14 @@ class BoundedLatenessGeneratorTest extends UnitSpec { assert(seenSpaces.head === space) } - def watermarkTest(testEvents: Stream[TestEvent]) = { + def watermarkTest(testEvents: Stream[TestEvent]): Assertion = { var prevTs = -1L val maxLateness = 10L val blg = getBlg(maxLateness) val wmo = new WatermarkOutput { override def emitWatermark(watermark: Watermark): Unit = {} override def markIdle(): Unit = {} + override def markActive(): Unit = {} } var maxTs = 0L val result = testEvents