From 6d12b3a0b8d8200d0f71166bcd050a9088d1f286 Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Fri, 1 Oct 2021 07:50:22 -0400 Subject: [PATCH 1/8] refactor to use less implicits --- README.md | 2 +- project/build.properties | 2 +- .../epiphanous/flinkrunner/FlinkRunner.scala | 667 ++++++++++++++++- .../flinkrunner/FlinkRunnerFactory.scala | 47 +- .../algorithm/cardinality/HyperLogLog.scala | 80 ++- .../membership/StableBloomFilter.scala | 229 +++--- .../membership/StableBloomFilterBuilder.scala | 32 +- .../avro/AvroCodingException.scala | 6 +- .../avro/ConfluentSchemaRegistryClient.scala | 9 +- .../flinkrunner/flink/BaseFlinkJob.scala | 131 ++-- .../flinkrunner/flink/BroadcastFlinkJob.scala | 71 +- .../flink/FilterByControlJob.scala | 70 +- .../flinkrunner/flink/FlinkJob.scala | 24 +- .../flinkrunner/flink/IdentityJob.scala | 28 +- .../flinkrunner/model/ConfigToProps.scala | 46 +- .../flinkrunner/model/DataControlPeriod.scala | 2 +- .../flinkrunner/model/DataOrControl.scala | 14 +- .../flinkrunner/model/FlinkConfig.scala | 119 +--- .../flinkrunner/model/SinkConfig.scala | 4 +- .../flinkrunner/model/SourceConfig.scala | 37 +- .../flinkrunner/model/UnitMapper.scala | 1 + .../model/aggregate/Aggregate.scala | 316 ++++++--- .../flinkrunner/model/aggregate/Count.scala | 24 +- .../aggregate/ExponentialMovingAverage.scala | 45 +- .../ExponentialMovingStandardDeviation.scala | 54 +- .../aggregate/ExponentialMovingVariance.scala | 58 +- .../model/aggregate/Histogram.scala | 90 ++- .../flinkrunner/model/aggregate/Max.scala | 24 +- .../flinkrunner/model/aggregate/Mean.scala | 24 +- .../flinkrunner/model/aggregate/Min.scala | 24 +- .../model/aggregate/Percentage.scala | 41 +- .../flinkrunner/model/aggregate/Range.scala | 27 +- .../model/aggregate/StandardDeviation.scala | 27 +- .../flinkrunner/model/aggregate/Sum.scala | 24 +- .../aggregate/SumOfSquaredDeviations.scala | 26 +- .../model/aggregate/Variance.scala | 31 +- .../operator/AddToJdbcBatchFunction.scala | 4 +- .../operator/EnrichmentAsyncFunction.scala | 13 +- .../BoundedLatenessWatermarkStrategy.scala | 9 +- .../flinkrunner/util/JdbcSink.scala | 13 +- .../flinkrunner/util/StreamUtils.scala | 670 ------------------ .../util/BoundedLatenessGeneratorTest.scala | 6 +- 42 files changed, 1742 insertions(+), 1429 deletions(-) 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/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/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index 010d804..7d384a3 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -1,7 +1,61 @@ 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.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.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 /** * Flink Job Invoker @@ -13,9 +67,9 @@ 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 @@ -25,7 +79,7 @@ class FlinkRunner[ADT <: FlinkEvent]( * a function from an iterator to unit */ def process( - callback: PartialFunction[Stream[ADT], Unit] = { case _ => + callback: PartialFunction[List[_], Unit] = { case _ => () } ): Unit = @@ -44,7 +98,7 @@ class FlinkRunner[ADT <: FlinkEvent]( * flink job */ def process1( - callback: PartialFunction[Stream[ADT], Unit] = { case _ => + callback: PartialFunction[List[_], Unit] = { case _ => () } ): Unit = { @@ -53,8 +107,8 @@ class FlinkRunner[ADT <: FlinkEvent]( .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, config).run() match { + case Left(results) => callback(results) case Right(_) => () } } @@ -103,4 +157,601 @@ 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 boundedOutofOrdernessWatermarks[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 + * @param env + * implicit stream execution environment + * @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 orderness" => + boundedOutofOrdernessWatermarks() + 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] = { + val consumer = + new FlinkKafkaConsumer[E]( + srcConfig.topic, + config + .getKafkaDeserializationSchema[E](srcConfig.name) + .asInstanceOf[KafkaDeserializationSchema[E]], + srcConfig.properties + ) + env + .addSource(consumer) + } + + /** + * 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(srcConfig.name) + .asInstanceOf[KinesisDeserializationSchema[E]], + srcConfig.properties + ) + env + .addSource(consumer) + .name(srcConfig.label) + } + + /** + * 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(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 + * @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 = 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 = "") = + 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 + * @param config + * implicit flink job args + * @tparam E + * stream element type + * @return + * DataStream[E] + */ + def toSink[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkName: String = "" + ) = { + 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 <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: KafkaSinkConfig + ): 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 <: 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 + } + .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 <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: JdbcSinkConfig + ): DataStreamSink[E] = + stream + .addSink( + new JdbcSink[E]( + sinkConfig, + config + .getAddToJdbcBatchFunction(sinkConfig.name) + .asInstanceOf[AddToJdbcBatchFunction[E]] + ) + ) + .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 <: 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).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 <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: SocketSinkConfig + ): 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 <: ADT: 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 <: 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).uid(sinkConfig.label).name(sinkConfig.label) + } + } diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala index 42abd7c..3814d1d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala @@ -21,45 +21,54 @@ import org.apache.flink.streaming.connectors.kinesis.serialization.{ trait FlinkRunnerFactory[ADT <: FlinkEvent] { + 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( name: String, - config: FlinkConfig): BaseFlinkJob[_, _ <: ADT] + config: FlinkConfig[ADT]): BaseFlinkJob[_, _, 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 getKinesisDeserializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KinesisDeserializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KinesisDeserializationSchema[E] = ??? - def getSerializationSchema( + def getSerializationSchema[E <: ADT]( name: String, - config: FlinkConfig): SerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): SerializationSchema[E] = ??? - def getKafkaSerializationSchema( + def getKafkaSerializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KafkaSerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KafkaSerializationSchema[E] = ??? - def getKinesisSerializationSchema( + def getKinesisSerializationSchema[E <: ADT]( name: String, - config: FlinkConfig): KinesisSerializationSchema[ADT] = ??? + config: FlinkConfig[ADT]): KinesisSerializationSchema[E] = ??? - def getEncoder(name: String, config: FlinkConfig): Encoder[ADT] = ??? + def getEncoder[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): Encoder[E] = ??? - def getAddToJdbcBatchFunction( + def getAddToJdbcBatchFunction[E <: ADT]( name: String, - config: FlinkConfig): AddToJdbcBatchFunction[ADT] = ??? + config: FlinkConfig[ADT]): AddToJdbcBatchFunction[E] = ??? - def getBucketAssigner( + def getBucketAssigner[E <: ADT]( name: String, - config: FlinkConfig): BucketAssigner[ADT, 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 bcc2f39..976db49 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala @@ -7,26 +7,38 @@ import java.nio.ByteBuffer import scala.util.Random /** - * Implements the stable bloom filter from the paper by - * F. Deng and D. Rafiei. Approximately detecting - * duplicates for streaming data using stable bloom - * filters. In SIGMOD, pages 25–36, 2006. - * - * We use heap storage (an array of Longs). - * This implies M=m*d can be set as high as about 125 giga-bits. - * - * @param funnel a Guava funnel for taking input - * @param m number of cells (see the paper, m is a Long but m/floor(63/d) - * must fit in a 32-bit Int) - * @param d bits per cell (see the paper, should lie in [1,63] but often set to 1, 2 or 3) - * @param FPR expected false positive rate (should lie in (0,1)) - * @tparam T the type of funnel used - */ -case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) { + * Implements the stable bloom filter from the paper by F. Deng and D. + * Rafiei. Approximately detecting duplicates for streaming data + * using stable bloom filters. In SIGMOD, pages 25–36, 2006. + * + * We use heap storage (an array of Longs). This implies M=m*d + * can be set as high as about 125 giga-bits. + * + * @param funnel + * a Guava funnel for taking input + * @param m + * number of cells (see the paper, m is a Long + * but m/floor(63/d) must fit in a 32-bit Int) + * @param d + * bits per cell (see the paper, should lie in [1,63] but often set to 1, + * 2 or 3) + * @param FPR + * expected false positive rate (should lie in (0,1)) + * @tparam T + * the type of funnel used + */ +case class StableBloomFilter[T]( + funnel: Funnel[T], + m: Long, + d: Int, + FPR: Double) { import StableBloomFilter._ - require(d > 0 && d <= STORAGE_BITS, s"d must be an integer in [1,$STORAGE_BITS]") + require( + d > 0 && d <= STORAGE_BITS, + s"d must be an integer in [1,$STORAGE_BITS]" + ) /** number of bits used per unit storage */ val storedBits: Long = STORAGE_BITS.toLong / (d * d) @@ -34,7 +46,10 @@ case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) /** total memory required */ val M = m * d - require(M / storedBits < Int.MaxValue, s"M/$storedBits must be <= ${Int.MaxValue}") + require( + M / storedBits < Int.MaxValue, + s"M/$storedBits must be <= ${Int.MaxValue}" + ) require(FPR > 0 && FPR < 1, "FPR must be a double in (0,1)") /** cell value to set upon insertion */ @@ -59,13 +74,14 @@ case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) val storage = Array.fill[Long](w)(0) /** - * Insert a stream element into the filter. - * - * @param item the item to insert - * @return - */ + * Insert a stream element into the filter. + * + * @param item + * the item to insert + * @return + */ def add(item: T): Boolean = { - val cells = hash(item) + val cells = hash(item) val alreadySeen = cells.forall(i => get(i) > 0L) decrementRandomCells() cells.foreach(set) @@ -73,89 +89,104 @@ case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) } /** - * Return true if this SBF might contain the requested item. - * - * @param item the item to check - * @return - */ + * Return true if this SBF might contain the requested item. + * + * @param item + * the item to check + * @return + */ def mightContain(item: T): Boolean = hash(item).forall(i => get(i) > 0L) /** - * Merge another filter into this filter. - * - * @param another the other filter - * @return - */ + * Merge another filter into this filter. + * + * @param another + * the other filter + * @return + */ def merge(another: StableBloomFilter[T]): StableBloomFilter[T] = { - require(another.M == M && another.d == d && another.FPR == FPR, "Can only merge SBFs with same settings") + require( + another.M == M && another.d == d && another.FPR == FPR, + "Can only merge SBFs with same settings" + ) another.storage.zipWithIndex.foreach { case (s, i) => storage(i) |= s } this } /** - * Decrement P cells randomly. As recommended in the DR paper, we only generate a single random index, then - * decrement that cell and the next P-1 cells (wrapping around if needed). - */ + * Decrement P cells randomly. As recommended in the DR paper, we only + * generate a single random index, then decrement that cell and the next + * P-1 cells (wrapping around if needed). + */ private def decrementRandomCells(): Unit = { val p = (random.nextDouble() * m).toLong Range(0, P).map(i => (i + p) % m).foreach(decrement) } /** - * Gets the current value of the i'th cell. - * - * @param i the cell to get (in [0, m)) - * @return - */ + * Gets the current value of the i'th cell. + * + * @param i + * the cell to get (in [0, m)) + * @return + */ def get(i: Long) = { val (x, j) = offset(i) getBitsValue(x, j) } /** - * Decrement a cell by one. - * - * @param i the cell to decrement (in [0,m)) - */ + * Decrement a cell by one. + * + * @param i + * the cell to decrement (in [0,m)) + */ private def decrement(i: Long): Unit = { - val (x, j) = offset(i) + val (x, j) = offset(i) val current = getBitsValue(x, j) if (current > 0) storage(x) -= (1L << j) } /** - * Set a cell's value to Max - * - * @param i the cell to set (in [0,m)) - */ + * Set a cell's value to Max + * + * @param i + * the cell to set (in [0,m)) + */ private def set(i: Long): Unit = { val (x, j) = offset(i) storage(x) |= (Max.toLong << j) } /** - * Extract the Int value of d bits (bits j to j+d-1) from stored element - * x. - * - * @param x the index into storage - * @param j the LSB to start from - * @return Int - */ + * Extract the Int value of d bits (bits j to + * j+d-1) from stored element x. + * + * @param x + * the index into storage + * @param j + * the LSB to start from + * @return + * Int + */ private def getBitsValue(x: Int, j: Int) = (storage(x) & (Max.toLong << j)) >>> j /** - * Converts a cell number into a tuple of (x:Int, j:Int), allowing other methods to get and set - * cell values. - * - * x in the integer offset within storage that contains cell i. - * j is the relative offset (in [0,63]) of the LSB of cell i within storage[x]. - * - * @param i the cell number in [0,m) - * @return (Int, Int) - */ + * Converts a cell number into a tuple of (x:Int, j:Int), + * allowing other methods to get and set cell values. + * + * x in the integer offset within storage that contains cell + * i. j is the relative offset (in [0,63]) of + * the LSB of cell i within storage[x]. + * + * @param i + * the cell number in [0,m) + * @return + * (Int, Int) + */ private def offset(i: Long): (Int, Int) = { // the cell covers d bits starting at b (within our total M bits) val b = (i - 1) * d @@ -167,21 +198,22 @@ case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) (x, j) } - /** Computes K hash functions of a filter item. - * - * @param item the item to hash - * @return - */ + /** + * Computes K hash functions of a filter item. + * + * @param item + * the item to hash + * @return + */ private def hash(item: T) = { val hash128 = hasher.hashObject(item, funnel).asBytes() - val hash1 = ByteBuffer.wrap(hash128, 0, 8).getLong - val hash2 = ByteBuffer.wrap(hash128, 8, 8).getLong - Range(1, K + 1).map( - i => - (hash1 + i * hash2 match { - case combined if combined < 0 => ~combined - case combined => combined - }) % m + val hash1 = ByteBuffer.wrap(hash128, 0, 8).getLong + val hash2 = ByteBuffer.wrap(hash128, 8, 8).getLong + Range(1, K + 1).map(i => + (hash1 + i * hash2 match { + case combined if combined < 0 => ~combined + case combined => combined + }) % m ) } @@ -189,21 +221,30 @@ case class StableBloomFilter[T](funnel: Funnel[T], m: Long, d: Int, FPR: Double) object StableBloomFilter { val STORAGE_BITS = java.lang.Long.SIZE - 1 - val LN2 = Math.log(2) - val LN2_SQUARED = LN2 * LN2 + val LN2 = Math.log(2) + val LN2_SQUARED = LN2 * LN2 - /** Return a builder for constructing an instance of StableBloomFilter[T] */ + /** + * Return a builder for constructing an instance of StableBloomFilter[T] + */ def builder[T](funnel: Funnel[T]) = StableBloomFilterBuilder[T](funnel) - /** Return the optimal number of cells to decrement each time a new item is inserted - * in the filter. This quantity is represented by the symbol P in the DR paper (eqn 17). - * - * @param m number of cells in the SBF - * @param K number of hash functions - * @param d bits per cell (Max = 2**d - 1) - * @param FPS false positive rate - * @return P optimal number of cells to decrement - */ + /** + * Return the optimal number of cells to decrement each time a new item + * is inserted in the filter. This quantity is represented by the symbol + * P in the DR paper (eqn 17). + * + * @param m + * number of cells in the SBF + * @param K + * number of hash functions + * @param d + * bits per cell (Max = 2**d - 1) + * @param FPS + * false positive rate + * @return + * P optimal number of cells to decrement + */ def optimalP(m: Long, K: Int, d: Int, FPS: Double) = { val Max = (1L << d) - 1 @@ -216,7 +257,7 @@ object StableBloomFilter { (1d / (denom1 * denom2)).toInt match { case x if x <= 0 => 1 - case x => x + case x => x } } } 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/AvroCodingException.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala index 048495c..8ddd508 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala @@ -1,4 +1,6 @@ package io.epiphanous.flinkrunner.avro -class AvroCodingException(message: String = "Failure during Avro coding", cause: Throwable = None.orNull) - extends Exception(message, cause) +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/ConfluentSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala index bd0dab9..cbbd517 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala @@ -4,9 +4,10 @@ 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.api.common.typeinfo.TypeInformation import org.apache.flink.runtime.concurrent.Executors.directExecutionContext import org.http4s.EntityDecoder import org.http4s.circe.jsonOf @@ -18,8 +19,8 @@ import java.util.concurrent.TimeUnit import scala.concurrent.ExecutionContext import scala.util.{Failure, Success, Try} -class ConfluentSchemaRegistryClient()(implicit - config: FlinkConfig, +class ConfluentSchemaRegistryClient[ADT <: FlinkEvent: TypeInformation]( + config: FlinkConfig[ADT])(implicit decoder: Decoder[ConfluentSchemaRegistryResponse]) extends AvroSchemaRegistryClient[ConfluentSchemaRegistryContext] with StringUtils @@ -104,7 +105,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")) 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..17d0389 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -2,26 +2,22 @@ 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.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 +124,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 +134,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 +145,35 @@ 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(name: String) = - factory.getKafkaDeserializationSchema(name, this) + def getKafkaDeserializationSchema[E <: ADT](name: String) = + factory.getKafkaDeserializationSchema[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 getKinesisSerializationSchema[E <: ADT](name: String) = + factory.getKinesisSerializationSchema[E](name, this) - def getEncoder(name: String) = factory.getEncoder(name, this) + def getEncoder[E <: ADT](name: String) = + factory.getEncoder[E](name, this) - def getAddToJdbcBatchFunction(name: String) = - factory.getAddToJdbcBatchFunction(name, this) + def getAddToJdbcBatchFunction[E <: ADT](name: String) = + factory.getAddToJdbcBatchFunction[E](name, this) - def getBucketAssigner(name: String) = - factory.getBucketAssigner(name, this) + def getBucketAssigner[E <: ADT](name: String) = + factory.getBucketAssigner[E](name, this) - def getAvroCoder(name: String) = factory.getAvroCoder(name, this) + def getAvroCoder(name: String) = + factory.getAvroCoder(name, this) def getSourceConfig(name: String): SourceConfig = SourceConfig(name, this) @@ -232,9 +208,6 @@ class FlinkConfig( else StreamExecutionEnvironment.getExecutionEnvironment - // use event time - env.setStreamTimeCharacteristic(timeCharacteristic) - // set parallelism env.setParallelism(globalParallelism) @@ -250,44 +223,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..f235cf4 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala @@ -15,7 +15,9 @@ sealed trait SinkConfig { } 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..85d081b 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala @@ -1,9 +1,9 @@ package io.epiphanous.flinkrunner.model import io.epiphanous.flinkrunner.model.FlinkConnectorName._ -import org.apache.flink.streaming.api.TimeCharacteristic import java.util.Properties +import scala.concurrent.duration.DurationInt import scala.util.Try sealed trait SourceConfig { @@ -13,20 +13,21 @@ sealed trait SourceConfig { def label: String = s"$connector/$name" - def timeCharacteristic: TimeCharacteristic - def watermarkStrategy: String + def maxAllowedLateness: Long + def properties: 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 +43,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 +52,8 @@ object SourceConfig { connector, name, config.getString(s"$p.stream"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case File => @@ -60,8 +61,8 @@ object SourceConfig { connector, name, config.getString(s"$p.path"), - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case Socket => @@ -70,8 +71,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 +80,8 @@ object SourceConfig { connector, name, name, - timeCharacteristic, watermarkStrategy, + maxAllowedLateness, config.getProperties(s"$p.config") ) case other => @@ -101,8 +102,8 @@ final case class KafkaSourceConfig( name: String, topic: String, isKeyed: Boolean, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -110,8 +111,8 @@ final case class KinesisSourceConfig( connector: FlinkConnectorName = Kinesis, name: String, stream: String, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -119,8 +120,8 @@ final case class FileSourceConfig( connector: FlinkConnectorName = File, name: String, path: String, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -129,8 +130,8 @@ final case class SocketSourceConfig( name: String, host: String, port: Int, - timeCharacteristic: TimeCharacteristic, watermarkStrategy: String, + maxAllowedLateness: Long, properties: Properties) extends SourceConfig @@ -138,7 +139,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..07b27a7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala @@ -4,7 +4,7 @@ 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 io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} import org.apache.flink.runtime.concurrent.Executors.directExecutionContext import org.apache.flink.streaming.api.scala.async.{ AsyncFunction, @@ -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 { 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/util/BoundedLatenessGeneratorTest.scala b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala index 0448d76..429c56f 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala @@ -1,6 +1,7 @@ package io.epiphanous.flinkrunner.util import io.epiphanous.flinkrunner.UnitSpec +import io.epiphanous.flinkrunner.model.FlinkEvent import org.apache.flink.api.common.eventtime.{Watermark, WatermarkOutput} import java.util.UUID.randomUUID @@ -12,7 +13,10 @@ class BoundedLatenessGeneratorTest extends UnitSpec { 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 From ca1edc8ccd4c85ed4092c070ac8f35a49aea66d4 Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Tue, 30 Nov 2021 09:33:08 -0500 Subject: [PATCH 2/8] begin refactor of confluent avro support --- build.sbt | 77 +++++++------ .../epiphanous/flinkrunner/FlinkRunner.scala | 60 +++------- .../flinkrunner/FlinkRunnerFactory.scala | 4 +- .../avro/ConfluentSchemaRegistryClient.scala | 19 +-- .../avro/RegisteredAvroSchema.scala | 5 +- .../CirceJsonDeserializationSchema.scala | 72 ++++++++++++ .../serde/CirceJsonSerializationSchema.scala | 74 ++++++++++++ ...onfluentAvroKafkaSerializationSchema.scala | 47 ++++++++ ...roRegistryKafkaDeserializationSchema.scala | 108 ++++++++++++++++++ .../epiphanous/flinkrunner/model/MyADT.scala | 28 +++++ ...ConfluentAvroSerializationSchemaTest.scala | 14 +++ ...yADTConfluentAvroSerializationSchema.scala | 8 ++ 12 files changed, 423 insertions(+), 93 deletions(-) create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala diff --git a/build.sbt b/build.sbt index 7f9488f..60abd1b 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.13.3" + val logback = "1.2.7" val scalaLogging = "3.9.4" val scalaTest = "3.2.10" val scalaCheck = "1.15.4" @@ -35,53 +36,57 @@ 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, + "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/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index 7d384a3..e4374e1 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -72,24 +72,9 @@ class FlinkRunner[ADT <: FlinkEvent]( 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[List[_], 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. * @@ -97,17 +82,18 @@ class FlinkRunner[ADT <: FlinkEvent]( * a function from a stream to unit that receives results from running * flink job */ - def process1( + 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 { + factory.getJobInstance(config.jobName, this).run() match { case Left(results) => callback(results) case Right(_) => () } @@ -185,7 +171,7 @@ class FlinkRunner[ADT <: FlinkEvent]( * @return * BoundedLatenessGenerator[E] */ - def boundedOutofOrdernessWatermarks[E <: ADT: TypeInformation]() + def boundedOutOfOrderWatermarks[E <: ADT: TypeInformation]() : WatermarkStrategy[E] = WatermarkStrategy .forBoundedOutOfOrderness(config.maxLateness) @@ -205,8 +191,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * Assign timestamps/watermarks if we're using event time * @param in * the input stream to watermark - * @param env - * implicit stream execution environment * @tparam E * event type * @return @@ -217,10 +201,10 @@ class FlinkRunner[ADT <: FlinkEvent]( srcConfig: SourceConfig ): DataStream[E] = in.assignTimestampsAndWatermarks(srcConfig.watermarkStrategy match { - case "bounded out of orderness" => - boundedOutofOrdernessWatermarks() - case "ascending timestamps" => ascendingTimestampsWatermarks() - case _ => boundedLatenessWatermarks(in.name) + case "bounded out of order" => + boundedOutOfOrderWatermarks() + case "ascending timestamps" => ascendingTimestampsWatermarks() + case _ => boundedLatenessWatermarks(in.name) }).name(s"wm:${in.name}") .uid(s"wm:${in.name}") @@ -406,7 +390,7 @@ class FlinkRunner[ADT <: FlinkEvent]( file.getAbsolutePath } - val runner = this + val runner: FlinkRunner[ADT] = this implicit class EventStreamOps[E <: ADT: TypeInformation]( stream: DataStream[E]) { @@ -422,7 +406,7 @@ class FlinkRunner[ADT <: FlinkEvent]( .uid(s"cast types $name") } - def toSink(sinkName: String = "") = + def toSink(sinkName: String = ""): Object = runner.toSink[E](stream, sinkName) } @@ -434,8 +418,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * 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 @@ -444,7 +426,7 @@ class FlinkRunner[ADT <: FlinkEvent]( def toSink[E <: ADT: TypeInformation]( stream: DataStream[E], sinkName: String = "" - ) = { + ): Object = { val name = if (sinkName.isEmpty) config.getSinkNames.head else sinkName config.getSinkConfig(name) match { case s: KafkaSinkConfig => toKafka[E](stream, s) @@ -468,8 +450,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * the data stream * @param sinkConfig * a sink configuration - * @param config - * implicit job args * @tparam E * stream element type * @return @@ -500,8 +480,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * the data stream * @param sinkConfig * a sink configuration - * @param config - * implicit job args * @tparam E * stream element type * @return @@ -535,8 +513,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * the data stream * @param sinkConfig * a sink configuration - * @param config - * implicit job args * @tparam E * stream element type * @return @@ -565,8 +541,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * the data stream * @param sinkConfig * a sink configuration - * @param config - * implicit job args * @tparam E * stream element type * @return @@ -661,8 +635,6 @@ class FlinkRunner[ADT <: FlinkEvent]( * the data stream * @param sinkConfig * a sink configuration - * @param config - * implicit job args * @tparam E * stream element type * @return @@ -697,7 +669,7 @@ class FlinkRunner[ADT <: FlinkEvent]( */ def toCassandraSink[E <: ADT: TypeInformation]( stream: DataStream[E], - sinkConfig: CassandraSinkConfig) = + sinkConfig: CassandraSinkConfig): CassandraSink[E] = CassandraSink .addSink(stream) .setHost(sinkConfig.host) diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala index 3814d1d..3943eb2 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala @@ -27,9 +27,9 @@ trait FlinkRunnerFactory[ADT <: FlinkEvent] { optConfig: Option[String] = None) = new FlinkConfig[ADT](args, this, sources, optConfig) - def getJobInstance( + def getJobInstance[DS, OUT <: ADT]( name: String, - config: FlinkConfig[ADT]): BaseFlinkJob[_, _, ADT] + runner: FlinkRunner[ADT]): BaseFlinkJob[DS, OUT, ADT] def getDeserializationSchema[E <: ADT]( name: String, diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala index cbbd517..dbd45c9 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala @@ -209,14 +209,17 @@ class ConfluentSchemaRegistryClient[ADT <: FlinkEvent: TypeInformation]( */ 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/RegisteredAvroSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala index 0f0a335..cab791f 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/RegisteredAvroSchema.scala @@ -28,8 +28,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 +54,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/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/ConfluentAvroKafkaSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala new file mode 100644 index 0000000..84b81fa --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala @@ -0,0 +1,47 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + KafkaSinkConfig, + KafkaSourceConfig +} +import org.apache.avro.specific.SpecificRecord +import org.apache.flink.api.common.serialization.SerializationSchema +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema +import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema +import org.apache.kafka.clients.producer.ProducerRecord + +import java.lang + +/** + * 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 ConfluentAvroKafkaSerializationSchema[ + E <: ADT, + ADT <: FlinkEvent]( + sinkName: String, + config: FlinkConfig[ADT] +) extends KafkaSerializationSchema[E] + with LazyLogging { + + val sinkConfig: KafkaSinkConfig = + config.getSourceConfig(sinkName).asInstanceOf[KafkaSinkConfig] + + override def serialize( + element: E, + timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = ??? +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala new file mode 100644 index 0000000..3c18fbb --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala @@ -0,0 +1,108 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + KafkaSourceConfig +} +import org.apache.avro.specific.SpecificRecord +import org.apache.flink.api.common.serialization.DeserializationSchema +import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema +import org.apache.kafka.clients.consumer.ConsumerRecord + +/** + * A schema to deserialize bytes from kafka into an ADT event using a + * confluent avro schema registry. An implementing class must provide a + * Flink [[ConfluentRegistryAvroDeserializationSchema]] to interface with + * the schema registry. That registry is specific to a type that implements + * Avro's [[SpecificRecord]] interface. The implementing class must also + * provide a [[deserializeSpecificRecord]] method that deserializes an + * array of bytes into a specific record type as well as a + * [[fromSpecificRecord]] method that converts that type into a type that + * is a member of the ADT. + * + * @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 ConfluentAvroRegistryKafkaDeserializationSchema[ + E <: ADT, + ADT <: FlinkEvent +]( + sourceName: String, + config: FlinkConfig[ADT] +) extends KafkaDeserializationSchema[E] + with LazyLogging { + + val sourceConfig: KafkaSourceConfig = + config.getSourceConfig(sourceName).asInstanceOf[KafkaSourceConfig] + + /** + * Implementing classes must provide a a confluent schema registry + * deserialization schema for specific records of type T. + * @tparam K + * specific record type + * @return + * ConfluentRegistryAvroDeserializationSchema[K] + */ + def schemaRegistryKeyDeserializer[K] + : ConfluentRegistryAvroDeserializationSchema[K] + + /** + * A helper method to use the provided schema registry deserialization + * schema to deserialize a kafka message into a specific record instance. + * @param message + * the kafka message + * @return + * an instance of specific record type T + */ + def deserializeSpecificRecord[T <: SpecificRecord]( + message: Array[Byte], + isKey: Boolean = false): T = ??? + ///schemaRegistryDeserializer.deserialize(message) + + /** + * Convert a deserialized specific record instance into an instance of + * our produced event type. Must be defined by implementing classes. + * @param key + * an optional key of type K + * @param value + * a value of specific record type V + * @tparam K + * the type of the key + * @tparam V + * the type of the value, subtype of avro specific record + * @return + * an instance of the flink runner ADT + */ + def fromSpecificRecord[K, V <: SpecificRecord]( + key: Option[K], + value: V): E + + def deserializeKey[K](key: Array[Byte]): K + + override def deserialize( + record: ConsumerRecord[Array[Byte], Array[Byte]]): E = ??? +// { +// val key = +// if (sourceConfig.isKeyed) Some(deserializeKey(record.key())) +// else None +// fromSpecificRecord( +// key, +// schemaRegistryDeserializer.deserialize(record.value()) +// ) +// } + + override def isEndOfStream(nextElement: E): Boolean = false + + override def getProducedType: TypeInformation[E] = + TypeInformation.of(new TypeHint[E] {}) +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala new file mode 100644 index 0000000..ec1383e --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala @@ -0,0 +1,28 @@ +package io.epiphanous.flinkrunner.model + +import java.time.Instant +import java.util.UUID + +sealed trait MyADT extends FlinkEvent + +case class A( + id: String = UUID.randomUUID().toString, + a: String = "A", + value: Int = 0, + modified: Instant) + extends MyADT { + override def $id: String = id + override def $key: String = a + override def $timestamp: Long = modified.toEpochMilli +} + +case class B( + id: String = UUID.randomUUID().toString, + b: String = "B", + value: Double = 0d, + modified: Instant) + extends MyADT { + override def $id: String = id + override def $key: String = b + override def $timestamp: Long = modified.toEpochMilli +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala new file mode 100644 index 0000000..dbe4e54 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala @@ -0,0 +1,14 @@ +package io.epiphanous.flinkrunner.serde + +import io.epiphanous.flinkrunner.UnitSpec +import io.epiphanous.flinkrunner.model.FlinkEvent + +class ConfluentAvroSerializationSchemaTest extends UnitSpec { + +// val ss = new ConfluentAvroSerializationSchema() + + behavior of "ConfluentAvroSerializationSchema" + + it should "" + +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala new file mode 100644 index 0000000..6f0446d --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala @@ -0,0 +1,8 @@ +package io.epiphanous.flinkrunner.serde + +import io.epiphanous.flinkrunner.model.{FlinkConfig, MyADT} + +class MyADTConfluentAvroSerializationSchema( + name: String, + config: FlinkConfig[MyADT]) +// extends ConfluentAvroSerializationSchema {} From 4890d748213be647fc19fec9d1a2ce76517f9d5f Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Thu, 2 Dec 2021 13:42:36 -0500 Subject: [PATCH 3/8] implement serdes and upgrade flink to 1.14 --- build.sbt | 23 ++-- .../epiphanous/flinkrunner/FlinkRunner.scala | 111 +++++++++-------- .../flinkrunner/FlinkRunnerFactory.scala | 10 ++ .../membership/StableBloomFilter.scala | 2 +- .../avro/ConfluentSchemaRegistryClient.scala | 7 +- .../flinkrunner/model/FlinkConfig.scala | 8 ++ .../operator/EnrichmentAsyncFunction.scala | 5 +- ...onfluentAvroKafkaSerializationSchema.scala | 47 -------- ...roRegistryKafkaDeserializationSchema.scala | 108 ----------------- ...stryKafkaRecordDeserializationSchema.scala | 112 ++++++++++++++++++ ...gistryKafkaRecordSerializationSchema.scala | 109 +++++++++++++++++ .../util/BoundedLatenessGeneratorTest.scala | 26 ++-- 12 files changed, 327 insertions(+), 241 deletions(-) delete mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala delete mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala create mode 100644 src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala diff --git a/build.sbt b/build.sbt index 60abd1b..c565166 100644 --- a/build.sbt +++ b/build.sbt @@ -27,7 +27,7 @@ resolvers += "Local Maven Repository" at "file://" + Path.userHome.absolutePath resolvers += "Confluent Repository" at "https://packages.confluent.io/maven/" val V = new { - val flink = "1.13.3" + val flink = "1.14.0" val logback = "1.2.7" val scalaLogging = "3.9.4" val scalaTest = "3.2.10" @@ -77,16 +77,17 @@ val circeDeps = Seq( ).map(d => "io.circe" %% s"circe-$d" % V.circe) val otherDeps = Seq( - "io.confluent" % "kafka-schema-registry-client" % V.schemaRegistry, - "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 +// "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/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index e4374e1..1192bdb 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -15,6 +15,8 @@ import org.apache.flink.api.common.serialization.{ 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.{ @@ -33,17 +35,7 @@ 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.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.serialization.KinesisSerializationSchema import org.apache.flink.streaming.connectors.kinesis.{ FlinkKinesisConsumer, FlinkKinesisProducer @@ -247,18 +239,27 @@ class FlinkRunner[ADT <: FlinkEvent]( */ def fromKafka[E <: ADT: TypeInformation]( srcConfig: KafkaSourceConfig - ): DataStream[E] = { - val consumer = - new FlinkKafkaConsumer[E]( - srcConfig.topic, - config - .getKafkaDeserializationSchema[E](srcConfig.name) - .asInstanceOf[KafkaDeserializationSchema[E]], - srcConfig.properties - ) + ): DataStream[E] = env - .addSource(consumer) - } + .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. @@ -277,13 +278,11 @@ class FlinkRunner[ADT <: FlinkEvent]( new FlinkKinesisConsumer[E]( srcConfig.stream, config - .getKinesisDeserializationSchema(srcConfig.name) - .asInstanceOf[KinesisDeserializationSchema[E]], + .getKinesisDeserializationSchema[E](srcConfig.name), srcConfig.properties ) env .addSource(consumer) - .name(srcConfig.label) } /** @@ -304,8 +303,7 @@ class FlinkRunner[ADT <: FlinkEvent]( case other => other } val ds = config - .getDeserializationSchema(srcConfig.name) - .asInstanceOf[DeserializationSchema[E]] + .getDeserializationSchema[E](srcConfig.name) env .readTextFile(path) .name(s"raw:${srcConfig.label}") @@ -427,15 +425,22 @@ class FlinkRunner[ADT <: FlinkEvent]( stream: DataStream[E], sinkName: String = "" ): Object = { - 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) + 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}" @@ -460,18 +465,18 @@ class FlinkRunner[ADT <: FlinkEvent]( sinkConfig: KafkaSinkConfig ): DataStreamSink[E] = stream - .addSink( - new FlinkKafkaProducer[E]( - sinkConfig.topic, - config - .getKafkaSerializationSchema(sinkConfig.name) - .asInstanceOf[KafkaSerializationSchema[E]], - sinkConfig.properties, - Semantic.AT_LEAST_ONCE - ) + .sinkTo( + KafkaSink + .builder() + .setKafkaProducerConfig(sinkConfig.properties) + .setRecordSerializer( + config + .getKafkaRecordSerializationSchema[E]( + sinkConfig.name + ) + ) + .build() ) - .uid(sinkConfig.label) - .name(sinkConfig.label) /** * Send stream to a kinesis sink. @@ -503,8 +508,6 @@ class FlinkRunner[ADT <: FlinkEvent]( sink.setDefaultPartition("0") sink } - .uid(sinkConfig.label) - .name(sinkConfig.label) /** * Send stream to a socket sink. @@ -531,8 +534,6 @@ class FlinkRunner[ADT <: FlinkEvent]( .asInstanceOf[AddToJdbcBatchFunction[E]] ) ) - .uid(sinkConfig.label) - .name(sinkConfig.label) /** * Send stream to a rolling file sink. @@ -625,7 +626,7 @@ class FlinkRunner[ADT <: FlinkEvent]( s"Unknown file sink encoder format: '$encoderFormat'" ) } - stream.addSink(sink).uid(sinkConfig.label).name(sinkConfig.label) + stream.addSink(sink) } /** @@ -652,8 +653,6 @@ class FlinkRunner[ADT <: FlinkEvent]( .getSerializationSchema(sinkConfig.name) .asInstanceOf[SerializationSchema[E]] ) - .uid(sinkConfig.label) - .name(sinkConfig.label) /** * Send stream to a cassandra sink. @@ -675,8 +674,6 @@ class FlinkRunner[ADT <: FlinkEvent]( .setHost(sinkConfig.host) .setQuery(sinkConfig.query) .build() - .uid(sinkConfig.label) - .name(sinkConfig.label) /** * Send stream to an elasticsearch sink. @@ -723,7 +720,7 @@ class FlinkRunner[ADT <: FlinkEvent]( indexer.add(req) } ).build() - stream.addSink(esSink).uid(sinkConfig.label).name(sinkConfig.label) + stream.addSink(esSink) } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala index 3943eb2..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, @@ -40,6 +42,14 @@ trait FlinkRunnerFactory[ADT <: FlinkEvent] { config: FlinkConfig[ADT]): KafkaDeserializationSchema[E] = ??? + def getKafkaRecordSerializationSchema[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): KafkaRecordSerializationSchema[E] = ??? + + def getKafkaRecordDeserializationSchema[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): KafkaRecordDeserializationSchema[E] = ??? + def getKinesisDeserializationSchema[E <: ADT]( name: String, config: FlinkConfig[ADT]): KinesisDeserializationSchema[E] = ??? 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/avro/ConfluentSchemaRegistryClient.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala index dbd45c9..b41c16b 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala @@ -8,7 +8,7 @@ import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} import io.epiphanous.flinkrunner.util.StringUtils import org.apache.avro.Schema.Parser import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.runtime.concurrent.Executors.directExecutionContext +import org.apache.flink.util.concurrent.Executors import org.http4s.EntityDecoder import org.http4s.circe.jsonOf import org.http4s.client.Client @@ -16,7 +16,7 @@ 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[ADT <: FlinkEvent: TypeInformation]( @@ -37,7 +37,8 @@ class ConfluentSchemaRegistryClient[ADT <: FlinkEvent: TypeInformation]( 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) diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala index 17d0389..72d5685 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -5,6 +5,7 @@ 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.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment @@ -151,6 +152,10 @@ class FlinkConfig[ADT <: FlinkEvent]( def getKafkaDeserializationSchema[E <: ADT](name: String) = factory.getKafkaDeserializationSchema[E](name, this) + def getKafkaRecordDeserializationSchema[E <: ADT]( + name: String): KafkaRecordDeserializationSchema[E] = + factory.getKafkaRecordDeserializationSchema[E](name, this) + def getKinesisDeserializationSchema[E <: ADT](name: String) = factory.getKinesisDeserializationSchema[E](name, this) @@ -160,6 +165,9 @@ class FlinkConfig[ADT <: FlinkEvent]( def getKafkaSerializationSchema[E <: ADT](name: String) = factory.getKafkaSerializationSchema[E](name, this) + def getKafkaRecordSerializationSchema[E <: ADT](name: String) = + factory.getKafkaRecordSerializationSchema[E](name, this) + def getKinesisSerializationSchema[E <: ADT](name: String) = factory.getKinesisSerializationSchema[E](name, this) diff --git a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala index 07b27a7..8c54587 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala @@ -5,7 +5,7 @@ import com.google.common.cache.{CacheBuilder, CacheLoader} import com.typesafe.scalalogging.LazyLogging import io.circe.Decoder import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} -import org.apache.flink.runtime.concurrent.Executors.directExecutionContext +import org.apache.flink.util.concurrent.Executors import org.apache.flink.streaming.api.scala.async.{ AsyncFunction, ResultFuture @@ -75,7 +75,8 @@ abstract class EnrichmentAsyncFunction[ 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/ConfluentAvroKafkaSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala deleted file mode 100644 index 84b81fa..0000000 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroKafkaSerializationSchema.scala +++ /dev/null @@ -1,47 +0,0 @@ -package io.epiphanous.flinkrunner.serde - -import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{ - FlinkConfig, - FlinkEvent, - KafkaSinkConfig, - KafkaSourceConfig -} -import org.apache.avro.specific.SpecificRecord -import org.apache.flink.api.common.serialization.SerializationSchema -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroSerializationSchema -import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema -import org.apache.kafka.clients.producer.ProducerRecord - -import java.lang - -/** - * 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 ConfluentAvroKafkaSerializationSchema[ - E <: ADT, - ADT <: FlinkEvent]( - sinkName: String, - config: FlinkConfig[ADT] -) extends KafkaSerializationSchema[E] - with LazyLogging { - - val sinkConfig: KafkaSinkConfig = - config.getSourceConfig(sinkName).asInstanceOf[KafkaSinkConfig] - - override def serialize( - element: E, - timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = ??? -} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala deleted file mode 100644 index 3c18fbb..0000000 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaDeserializationSchema.scala +++ /dev/null @@ -1,108 +0,0 @@ -package io.epiphanous.flinkrunner.serde - -import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{ - FlinkConfig, - FlinkEvent, - KafkaSourceConfig -} -import org.apache.avro.specific.SpecificRecord -import org.apache.flink.api.common.serialization.DeserializationSchema -import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema -import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema -import org.apache.kafka.clients.consumer.ConsumerRecord - -/** - * A schema to deserialize bytes from kafka into an ADT event using a - * confluent avro schema registry. An implementing class must provide a - * Flink [[ConfluentRegistryAvroDeserializationSchema]] to interface with - * the schema registry. That registry is specific to a type that implements - * Avro's [[SpecificRecord]] interface. The implementing class must also - * provide a [[deserializeSpecificRecord]] method that deserializes an - * array of bytes into a specific record type as well as a - * [[fromSpecificRecord]] method that converts that type into a type that - * is a member of the ADT. - * - * @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 ConfluentAvroRegistryKafkaDeserializationSchema[ - E <: ADT, - ADT <: FlinkEvent -]( - sourceName: String, - config: FlinkConfig[ADT] -) extends KafkaDeserializationSchema[E] - with LazyLogging { - - val sourceConfig: KafkaSourceConfig = - config.getSourceConfig(sourceName).asInstanceOf[KafkaSourceConfig] - - /** - * Implementing classes must provide a a confluent schema registry - * deserialization schema for specific records of type T. - * @tparam K - * specific record type - * @return - * ConfluentRegistryAvroDeserializationSchema[K] - */ - def schemaRegistryKeyDeserializer[K] - : ConfluentRegistryAvroDeserializationSchema[K] - - /** - * A helper method to use the provided schema registry deserialization - * schema to deserialize a kafka message into a specific record instance. - * @param message - * the kafka message - * @return - * an instance of specific record type T - */ - def deserializeSpecificRecord[T <: SpecificRecord]( - message: Array[Byte], - isKey: Boolean = false): T = ??? - ///schemaRegistryDeserializer.deserialize(message) - - /** - * Convert a deserialized specific record instance into an instance of - * our produced event type. Must be defined by implementing classes. - * @param key - * an optional key of type K - * @param value - * a value of specific record type V - * @tparam K - * the type of the key - * @tparam V - * the type of the value, subtype of avro specific record - * @return - * an instance of the flink runner ADT - */ - def fromSpecificRecord[K, V <: SpecificRecord]( - key: Option[K], - value: V): E - - def deserializeKey[K](key: Array[Byte]): K - - override def deserialize( - record: ConsumerRecord[Array[Byte], Array[Byte]]): E = ??? -// { -// val key = -// if (sourceConfig.isKeyed) Some(deserializeKey(record.key())) -// else None -// fromSpecificRecord( -// key, -// schemaRegistryDeserializer.deserialize(record.value()) -// ) -// } - - override def isEndOfStream(nextElement: E): Boolean = false - - override def getProducedType: TypeInformation[E] = - TypeInformation.of(new TypeHint[E] {}) -} 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..6e2ac0b --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -0,0 +1,112 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient +import io.confluent.kafka.serializers.{ + KafkaAvroDeserializer, + KafkaAvroDeserializerConfig +} +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + KafkaSourceConfig +} +import io.epiphanous.flinkrunner.serde.ConfluentAvroRegistryKafkaRecordDeserializationSchema.DEFAULT_CACHE_CAPACITY +import org.apache.avro.specific.SpecificRecord +import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema +import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema +import org.apache.flink.util.Collector +import org.apache.kafka.clients.consumer.ConsumerRecord + +import java.util +import scala.collection.JavaConverters.mapAsJavaMapConverter +import scala.collection.mutable + +/** + * 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 = + config.getSourceConfig(sourceName).asInstanceOf[KafkaSourceConfig] + + val topic: String = sourceConfig.topic + + val url: String = + sourceConfig.properties.getProperty("schema.registry.url") + val cacheCapacity: Int = sourceConfig.properties + .getProperty("schema.registry.cache.capacity", DEFAULT_CACHE_CAPACITY) + .toInt + val useSpecificAvroReader: Boolean = sourceConfig.properties + .getProperty("use.specific.avro.reader", "true") + .toBoolean + val useLogicalTypes: Boolean = sourceConfig.properties + .getProperty("use.logical.type.converters", "true") + .toBoolean + + /** create deserializer config */ + val deserializerConfig: util.Map[String, Boolean] = Map( + KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG -> useSpecificAvroReader, + KafkaAvroDeserializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG -> useLogicalTypes + ).asJava + + /** our schema registry client */ + val schemaRegistryClient = + new CachedSchemaRegistryClient(url, cacheCapacity) + + /** map to store the value, and optionally, key deserializers */ + val deserializers: mutable.Map[String, KafkaAvroDeserializer] = + mutable.Map( + "value" -> new KafkaAvroDeserializer( + schemaRegistryClient, + deserializerConfig + ) + ) + + /** add the key deserializer if needed */ + if (sourceConfig.isKeyed) { + val keyDeserializer = new KafkaAvroDeserializer(schemaRegistryClient) + keyDeserializer.configure(deserializerConfig, true) + deserializers += ("key" -> keyDeserializer) + } + + /** + * Convert a kafka consumer record instance into an instance of our + * produced event type. Must be defined by implementing classes. + * @param record + * a kafka consumer record + * @return + * an instance of the flink runner ADT + */ + def fromConsumerRecord( + record: ConsumerRecord[Array[Byte], Array[Byte]]): E + + override def deserialize( + record: ConsumerRecord[Array[Byte], Array[Byte]], + out: Collector[E]): Unit = fromConsumerRecord(record) + + override def getProducedType: TypeInformation[E] = + TypeInformation.of(new TypeHint[E] {}) +} + +object ConfluentAvroRegistryKafkaRecordDeserializationSchema { + val DEFAULT_CACHE_CAPACITY = "1000" +} 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..120a2c0 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -0,0 +1,109 @@ +package io.epiphanous.flinkrunner.serde + +import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient +import io.confluent.kafka.serializers.{ + KafkaAvroSerializer, + KafkaAvroSerializerConfig +} +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkEvent, + KafkaSinkConfig +} +import io.epiphanous.flinkrunner.serde.ConfluentAvroRegistryKafkaRecordSerializationSchema.DEFAULT_CACHE_CAPACITY +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} +import scala.collection.JavaConverters.mapAsJavaMapConverter +import scala.collection.mutable + +/** + * 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 = + config.getSourceConfig(sinkName).asInstanceOf[KafkaSinkConfig] + + val url: String = + sinkConfig.properties.getProperty("schema.registry.url") + val cacheCapacity: Int = sinkConfig.properties + .getProperty("schema.registry.cache.capacity", DEFAULT_CACHE_CAPACITY) + .toInt + val removeJavaProps: Boolean = sinkConfig.properties + .getProperty("serializer.remove.java.props", "true") + .toBoolean + val useLogicalTypes: Boolean = sinkConfig.properties + .getProperty("serializer.use.logical.type.converters", "true") + .toBoolean + + /** create serializer config */ + val serializerConfig: util.Map[String, Boolean] = Map( + KafkaAvroSerializerConfig.AVRO_REMOVE_JAVA_PROPS_CONFIG -> removeJavaProps, + KafkaAvroSerializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG -> useLogicalTypes + ).asJava + + /** our schema registry client */ + val schemaRegistryClient = + new CachedSchemaRegistryClient(url, cacheCapacity) + + /** map to store the value, and optionally, key serializers */ + val serializers: mutable.Map[String, KafkaAvroSerializer] = + mutable.Map( + "value" -> new KafkaAvroSerializer( + schemaRegistryClient, + serializerConfig + ) + ) + + /** add the key serializer if needed */ + if (sinkConfig.isKeyed) { + val keySerializer = new KafkaAvroSerializer(schemaRegistryClient) + keySerializer.configure(serializerConfig, true) + serializers += ("key" -> keySerializer) + } + + /** + * Convert an element into a producer record of byte arrays. Must be + * defined by implementing classes. + * @param element + * an instance of the flinkrunner ADT + * @return + * ProducerRecord of bytes + */ + def toProducerRecord( + element: E): ProducerRecord[Array[Byte], Array[Byte]] + + override def serialize( + element: E, + context: KafkaRecordSerializationSchema.KafkaSinkContext, + timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = + toProducerRecord(element) + +} + +object ConfluentAvroRegistryKafkaRecordSerializationSchema { + val DEFAULT_CACHE_CAPACITY = "1000" +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala index 429c56f..27c459a 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/util/BoundedLatenessGeneratorTest.scala @@ -1,16 +1,16 @@ package io.epiphanous.flinkrunner.util import io.epiphanous.flinkrunner.UnitSpec -import io.epiphanous.flinkrunner.model.FlinkEvent 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]( @@ -18,13 +18,14 @@ class BoundedLatenessGeneratorTest extends UnitSpec { 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 @@ -33,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 @@ -46,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( @@ -55,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( @@ -64,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" @@ -88,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 From 3ffd924717f44aada77a2b9ddaa058449ada4f3e Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Thu, 2 Dec 2021 13:49:36 -0500 Subject: [PATCH 4/8] deprecated old avro code --- .../epiphanous/flinkrunner/avro/AvroCoder.scala | 4 ++++ .../flinkrunner/avro/AvroCodingException.scala | 4 ++++ .../avro/AvroSchemaRegistryClient.scala | 4 ++++ .../avro/ConfluentSchemaRegistryClient.scala | 4 ++++ .../avro/ConfluentSchemaRegistryContext.scala | 4 ++++ .../avro/ConfluentSchemaRegistryResponse.scala | 16 ++++++++++++++++ .../flinkrunner/avro/RegisteredAvroSchema.scala | 4 ++++ .../avro/TestSchemaRegistryClient.scala | 4 ++++ 8 files changed, 44 insertions(+) diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCoder.scala index bd37f52..8a2dc09 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 { diff --git a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala index 8ddd508..4a1c997 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/AvroCodingException.scala @@ -1,5 +1,9 @@ package io.epiphanous.flinkrunner.avro +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) class AvroCodingException( message: String = "Failure during Avro coding", cause: Throwable = None.orNull) 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 b41c16b..91cbcac 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/avro/ConfluentSchemaRegistryClient.scala @@ -19,6 +19,10 @@ import java.util.concurrent.TimeUnit import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} import scala.util.{Failure, Success, Try} +@deprecated( + "Use the ConfluentAvroRegistryKafkaRecordSerialization and Deserialization classes instead", + "4.0.0" +) class ConfluentSchemaRegistryClient[ADT <: FlinkEvent: TypeInformation]( config: FlinkConfig[ADT])(implicit decoder: Decoder[ConfluentSchemaRegistryResponse]) 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 cab791f..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, 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 = From 5f24dd25ae00ab900a5251332d5a07808d4e1efc Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Fri, 3 Dec 2021 13:02:19 -0500 Subject: [PATCH 5/8] refactor avro registry serdes --- src/main/resources/flink-runner.conf | 3 +- .../flinkrunner/model/FlinkConfig.scala | 4 + .../flinkrunner/model/SinkConfig.scala | 5 + .../flinkrunner/model/SourceConfig.scala | 5 + ...stryKafkaRecordDeserializationSchema.scala | 105 +++++++-------- ...gistryKafkaRecordSerializationSchema.scala | 124 ++++++++++-------- .../epiphanous/flinkrunner/model/MyADT.scala | 28 ---- .../flinkrunner/model/MyAvroADT.scala | 120 +++++++++++++++++ .../flinkrunner/model/NoJobFactory.scala | 10 ++ ...ryKafkaRecordSerializationSchemaTest.scala | 109 +++++++++++++++ ...ConfluentAvroSerializationSchemaTest.scala | 14 -- ...gistryKafkaRecordSerializationSchema.scala | 69 ++++++++++ ...yADTConfluentAvroSerializationSchema.scala | 8 -- 13 files changed, 441 insertions(+), 163 deletions(-) delete mode 100644 src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/model/NoJobFactory.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala delete mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala create mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala delete mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala 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/model/FlinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala index 72d5685..07cc530 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -180,6 +180,10 @@ class FlinkConfig[ADT <: FlinkEvent]( 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) diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala index f235cf4..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,6 +14,9 @@ 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 { diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala index 85d081b..6c68c0d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.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 import scala.concurrent.duration.DurationInt import scala.util.Try @@ -18,6 +20,9 @@ sealed trait SourceConfig { def maxAllowedLateness: Long def properties: Properties + + def propertiesMap: util.HashMap[String, String] = + Maps.newHashMap(Maps.fromProperties(properties)) } object SourceConfig { diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala index 6e2ac0b..e9b8cf6 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -1,27 +1,20 @@ package io.epiphanous.flinkrunner.serde import com.typesafe.scalalogging.LazyLogging -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient -import io.confluent.kafka.serializers.{ - KafkaAvroDeserializer, - KafkaAvroDeserializerConfig -} +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient +import io.confluent.kafka.serializers.KafkaAvroDeserializer import io.epiphanous.flinkrunner.model.{ FlinkConfig, + FlinkConnectorName, FlinkEvent, KafkaSourceConfig } -import io.epiphanous.flinkrunner.serde.ConfluentAvroRegistryKafkaRecordDeserializationSchema.DEFAULT_CACHE_CAPACITY -import org.apache.avro.specific.SpecificRecord import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema import org.apache.flink.util.Collector import org.apache.kafka.clients.consumer.ConsumerRecord import java.util -import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.mutable /** * A schema to deserialize bytes from kafka into an ADT event using a @@ -45,68 +38,64 @@ abstract class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ ) extends KafkaRecordDeserializationSchema[E] with LazyLogging { - val sourceConfig: KafkaSourceConfig = - config.getSourceConfig(sourceName).asInstanceOf[KafkaSourceConfig] - - val topic: String = sourceConfig.topic + 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 url: String = - sourceConfig.properties.getProperty("schema.registry.url") - val cacheCapacity: Int = sourceConfig.properties - .getProperty("schema.registry.cache.capacity", DEFAULT_CACHE_CAPACITY) - .toInt - val useSpecificAvroReader: Boolean = sourceConfig.properties - .getProperty("use.specific.avro.reader", "true") - .toBoolean - val useLogicalTypes: Boolean = sourceConfig.properties - .getProperty("use.logical.type.converters", "true") - .toBoolean + val schemaRegistryProps: util.HashMap[String, String] = + sourceConfig.propertiesMap - /** create deserializer config */ - val deserializerConfig: util.Map[String, Boolean] = Map( - KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG -> useSpecificAvroReader, - KafkaAvroDeserializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG -> useLogicalTypes - ).asJava + val topic: String = sourceConfig.topic - /** our schema registry client */ - val schemaRegistryClient = - new CachedSchemaRegistryClient(url, cacheCapacity) + /** + * 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 - /** map to store the value, and optionally, key deserializers */ - val deserializers: mutable.Map[String, KafkaAvroDeserializer] = - mutable.Map( - "value" -> new KafkaAvroDeserializer( - schemaRegistryClient, - deserializerConfig - ) - ) + val valueDeserializer = new KafkaAvroDeserializer( + schemaRegistryClient, + schemaRegistryProps + ) - /** add the key deserializer if needed */ - if (sourceConfig.isKeyed) { - val keyDeserializer = new KafkaAvroDeserializer(schemaRegistryClient) - keyDeserializer.configure(deserializerConfig, true) - deserializers += ("key" -> keyDeserializer) - } + val keyDeserializer: Option[KafkaAvroDeserializer] = + if (sourceConfig.isKeyed) { + val ks = new KafkaAvroDeserializer(schemaRegistryClient) + ks.configure(schemaRegistryProps, true) + Some(ks) + } else None /** - * Convert a kafka consumer record instance into an instance of our - * produced event type. Must be defined by implementing classes. - * @param record - * a kafka consumer record + * 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 fromConsumerRecord( - record: ConsumerRecord[Array[Byte], Array[Byte]]): E + def fromKeyValue(key: Option[AnyRef], value: AnyRef): E override def deserialize( record: ConsumerRecord[Array[Byte], Array[Byte]], - out: Collector[E]): Unit = fromConsumerRecord(record) + 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] {}) } - -object ConfluentAvroRegistryKafkaRecordDeserializationSchema { - val DEFAULT_CACHE_CAPACITY = "1000" -} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala index 120a2c0..f39007a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -1,25 +1,20 @@ package io.epiphanous.flinkrunner.serde import com.typesafe.scalalogging.LazyLogging -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient -import io.confluent.kafka.serializers.{ - KafkaAvroSerializer, - KafkaAvroSerializerConfig -} +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient +import io.confluent.kafka.serializers.KafkaAvroSerializer import io.epiphanous.flinkrunner.model.{ FlinkConfig, + FlinkConnectorName, FlinkEvent, KafkaSinkConfig } -import io.epiphanous.flinkrunner.serde.ConfluentAvroRegistryKafkaRecordSerializationSchema.DEFAULT_CACHE_CAPACITY 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} -import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.mutable /** * A schema to serialize an ADT event using a confluent avro schema @@ -44,66 +39,87 @@ abstract class ConfluentAvroRegistryKafkaRecordSerializationSchema[ ) extends KafkaRecordSerializationSchema[E] with LazyLogging { - val sinkConfig: KafkaSinkConfig = - config.getSourceConfig(sinkName).asInstanceOf[KafkaSinkConfig] - - val url: String = - sinkConfig.properties.getProperty("schema.registry.url") - val cacheCapacity: Int = sinkConfig.properties - .getProperty("schema.registry.cache.capacity", DEFAULT_CACHE_CAPACITY) - .toInt - val removeJavaProps: Boolean = sinkConfig.properties - .getProperty("serializer.remove.java.props", "true") - .toBoolean - val useLogicalTypes: Boolean = sinkConfig.properties - .getProperty("serializer.use.logical.type.converters", "true") - .toBoolean - - /** create serializer config */ - val serializerConfig: util.Map[String, Boolean] = Map( - KafkaAvroSerializerConfig.AVRO_REMOVE_JAVA_PROPS_CONFIG -> removeJavaProps, - KafkaAvroSerializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG -> useLogicalTypes - ).asJava + 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] + } - /** our schema registry client */ - val schemaRegistryClient = - new CachedSchemaRegistryClient(url, cacheCapacity) + val schemaRegistryProps: util.HashMap[String, String] = + sinkConfig.propertiesMap /** map to store the value, and optionally, key serializers */ - val serializers: mutable.Map[String, KafkaAvroSerializer] = - mutable.Map( - "value" -> new KafkaAvroSerializer( - schemaRegistryClient, - serializerConfig - ) - ) + val valueSerializer = new KafkaAvroSerializer( + schemaRegistryClient, + schemaRegistryProps + ) /** add the key serializer if needed */ - if (sinkConfig.isKeyed) { - val keySerializer = new KafkaAvroSerializer(schemaRegistryClient) - keySerializer.configure(serializerConfig, true) - serializers += ("key" -> keySerializer) - } + 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) +// ) +// } /** - * Convert an element into a producer record of byte arrays. Must be - * defined by implementing classes. + * 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 - * ProducerRecord of bytes + * (Option[AnyRef], AnyRef) */ - def toProducerRecord( - element: E): ProducerRecord[Array[Byte], Array[Byte]] + def toKeyValue(element: E): (Option[AnyRef], AnyRef) override def serialize( element: E, context: KafkaRecordSerializationSchema.KafkaSinkContext, - timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = - toProducerRecord(element) - -} + 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) + } -object ConfluentAvroRegistryKafkaRecordSerializationSchema { - val DEFAULT_CACHE_CAPACITY = "1000" } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala deleted file mode 100644 index ec1383e..0000000 --- a/src/test/scala/io/epiphanous/flinkrunner/model/MyADT.scala +++ /dev/null @@ -1,28 +0,0 @@ -package io.epiphanous.flinkrunner.model - -import java.time.Instant -import java.util.UUID - -sealed trait MyADT extends FlinkEvent - -case class A( - id: String = UUID.randomUUID().toString, - a: String = "A", - value: Int = 0, - modified: Instant) - extends MyADT { - override def $id: String = id - override def $key: String = a - override def $timestamp: Long = modified.toEpochMilli -} - -case class B( - id: String = UUID.randomUUID().toString, - b: String = "B", - value: Double = 0d, - modified: Instant) - extends MyADT { - override def $id: String = id - override def $key: String = b - override def $timestamp: Long = modified.toEpochMilli -} 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/ConfluentAvroSerializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala deleted file mode 100644 index dbe4e54..0000000 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroSerializationSchemaTest.scala +++ /dev/null @@ -1,14 +0,0 @@ -package io.epiphanous.flinkrunner.serde - -import io.epiphanous.flinkrunner.UnitSpec -import io.epiphanous.flinkrunner.model.FlinkEvent - -class ConfluentAvroSerializationSchemaTest extends UnitSpec { - -// val ss = new ConfluentAvroSerializationSchema() - - behavior of "ConfluentAvroSerializationSchema" - - it should "" - -} 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/serde/MyADTConfluentAvroSerializationSchema.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala deleted file mode 100644 index 6f0446d..0000000 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroSerializationSchema.scala +++ /dev/null @@ -1,8 +0,0 @@ -package io.epiphanous.flinkrunner.serde - -import io.epiphanous.flinkrunner.model.{FlinkConfig, MyADT} - -class MyADTConfluentAvroSerializationSchema( - name: String, - config: FlinkConfig[MyADT]) -// extends ConfluentAvroSerializationSchema {} From 977b831d4ff6d590061a4f5861ddcb8891c1f167 Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Fri, 3 Dec 2021 18:27:58 -0500 Subject: [PATCH 6/8] simplified --- .../CirceJsonDeserializationSchema.scala | 18 ++-- .../serde/CirceJsonSerializationSchema.scala | 10 +- ...stryKafkaRecordDeserializationSchema.scala | 76 ++++++++------- ...gistryKafkaRecordSerializationSchema.scala | 96 ++++++++----------- ...ryKafkaRecordSerializationSchemaTest.scala | 47 ++++++--- ...gistryKafkaRecordSerializationSchema.scala | 69 ------------- 6 files changed, 126 insertions(+), 190 deletions(-) delete mode 100644 src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala index a7f0d59..f47244a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonDeserializationSchema.scala @@ -21,12 +21,12 @@ import java.nio.charset.StandardCharsets * @tparam ADT * the algebraic data type of our events */ -class CirceJsonDeserializationSchema[E <: ADT, ADT <: FlinkEvent]( +class CirceJsonDeserializationSchema[ADT <: FlinkEvent]( sourceName: String, config: FlinkConfig[ADT])(implicit - circeDecoder: Decoder[E], - ev: Null <:< E) - extends DeserializationSchema[E] + circeDecoder: Decoder[ADT], + ev: Null <:< ADT) + extends DeserializationSchema[ADT] with LazyLogging { val sourceConfig: SourceConfig = config.getSourceConfig(sourceName) @@ -39,9 +39,9 @@ class CirceJsonDeserializationSchema[E <: ADT, ADT <: FlinkEvent]( * @return * an instance of an ADT event type */ - override def deserialize(bytes: Array[Byte]): E = { + override def deserialize(bytes: Array[Byte]): ADT = { val payload = new String(bytes, StandardCharsets.UTF_8) - decode[E](payload).toOption match { + decode[ADT](payload).toOption match { case Some(event) => event case other => logger.error( @@ -59,14 +59,14 @@ class CirceJsonDeserializationSchema[E <: ADT, ADT <: FlinkEvent]( * @return * false */ - override def isEndOfStream(nextEvent: E): Boolean = false + override def isEndOfStream(nextEvent: ADT): Boolean = false /** * Compute the produced type when deserializing a byte array * @return * TypeInformation[E] */ - override def getProducedType: TypeInformation[E] = - TypeInformation.of(new TypeHint[E] {}) + override def getProducedType: TypeInformation[ADT] = + TypeInformation.of(new TypeHint[ADT] {}) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala index 48b8796..0ddfe86 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/CirceJsonSerializationSchema.scala @@ -26,10 +26,10 @@ import java.nio.charset.StandardCharsets * @tparam ADT * the flink runner ADT */ -class CirceJsonSerializationSchema[E <: ADT, ADT <: FlinkEvent]( +class CirceJsonSerializationSchema[ADT <: FlinkEvent]( sinkName: String, - config: FlinkConfig[ADT])(implicit circeEncoder: Encoder[E]) - extends SerializationSchema[E] + config: FlinkConfig[ADT])(implicit circeEncoder: Encoder[ADT]) + extends SerializationSchema[ADT] with LazyLogging { val sourceConfig: SinkConfig = config.getSinkConfig(sinkName) @@ -45,7 +45,7 @@ class CirceJsonSerializationSchema[E <: ADT, ADT <: FlinkEvent]( * @return * a json encoded byte array */ - override def serialize(event: E): Array[Byte] = + override def serialize(event: ADT): Array[Byte] = toJson(event).getBytes(StandardCharsets.UTF_8) /** @@ -61,7 +61,7 @@ class CirceJsonSerializationSchema[E <: ADT, ADT <: FlinkEvent]( * a json-encoded string */ def toJson( - event: E, + event: ADT, pretty: Boolean = configPretty, sortKeys: Boolean = configSort): String = { val j = event.asJson diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala index e9b8cf6..9fe9c95 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -17,25 +17,50 @@ 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. + * A deserialization schema that uses the provided confluent avro schema + * registry client and `fromKV` partial function to deserialize a kafka + * key/value pair into an instance of a flink runner ADT. * + * In order to decouple the shape of the flink runner ADT types from the + * types that are serialized in kafka, a user of this class must provide a + * `fromKV` partial function that maps from the specific key and value pair + * (key is optional) deserialized from the kafka source into an instance of + * the flink runner ADT. + * + * Usually, `fromKV` is as simple as providing a set of cases. Consider the + * following example, where `A` and `B` are subclasses of the flink runner + * ADT, and `ASpecific` and `BSpecific` are corresponding Avro + * `SpecificRecord` classes generated from avro schemas. In this case, we + * ignore the key and have defined our ADT types to be wrappers around the + * deserialized records. However, you can use the deserialized key and + * value in any way that makes sense for your application. + * {{{ + * { + * // (key,value) => ADT + * case (_, a:ASpecific) => A(a) + * case (_, b:BSpecific) => B(b) + * } + * }}} * @param sourceName - * name of the source stream + * name of the kafka source * @param config * flink runner config - * @tparam E - * the event type we are producing here, which is a member of the ADT + * @param schemaRegistryClient + * the schema registry client + * @param fromKV + * a partial function that should return a flink runner adt instance when + * passed a deserialized kafka key/value pair * @tparam ADT - * the flink runner ADT + * the flink runner ADT type */ -abstract class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ - E <: ADT, +class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ ADT <: FlinkEvent ]( sourceName: String, - config: FlinkConfig[ADT] -) extends KafkaRecordDeserializationSchema[E] + config: FlinkConfig[ADT], + schemaRegistryClient: SchemaRegistryClient, + fromKV: PartialFunction[(Option[AnyRef], AnyRef), ADT] +) extends KafkaRecordDeserializationSchema[ADT] with LazyLogging { val sourceConfig: KafkaSourceConfig = { @@ -52,13 +77,6 @@ abstract class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ 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 @@ -71,31 +89,15 @@ abstract class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ 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 = { + out: Collector[ADT]): 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)) + if (Option(value).nonEmpty) out.collect(fromKV(key, value)) } - override def getProducedType: TypeInformation[E] = - TypeInformation.of(new TypeHint[E] {}) + override def getProducedType: TypeInformation[ADT] = + TypeInformation.of(new TypeHint[ADT] {}) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala index f39007a..0f1dfaa 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -18,25 +18,49 @@ 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. + * registry. + */ + +/** + * A serialization schema that uses the provided confluent avro schema + * registry client to serialize an instance of a flink runner ADT into + * kafka. In order to decouple the shape of the flink runner ADT types from + * the types that are serialized in kafka, and to support providing both a + * key and a value to kafka, a user of this class must provide a `toKV` + * partial function that maps from the flink runner ADT instance to the + * specific key and value pair (key is optional) that will be serialized + * into the kafka sink. + * + * Usually, `toKV` is as simple as providing a set of cases like so, where + * `A` and `B` are subclasses of the flink runner ADT. + * {{{ + * { + * // key value + * case a:A => (Some(a.id), a.value) + * case b:B => (Some(b.id), b.value) + * } + * }}} * @param sinkName - * name of the sink stream + * name of the kafka sink we serialize into * @param config * flink runner config - * @tparam E - * the event type we are serializing from, which is a member of the ADT + * @param schemaRegistryClient + * an instance of a confluent schema registry client to use, for instance + * a CachedSchemaRegistryClient or a + * MockSchemaRegistryClient for testing + * @param toKV + * a partial function that maps an instance of the flink runner ADT into + * a key/value pair that will be serialized into the kafka sink * @tparam ADT - * the flink runner ADT + * the flink runner ADT type */ -abstract class ConfluentAvroRegistryKafkaRecordSerializationSchema[ - E <: ADT, +case class ConfluentAvroRegistryKafkaRecordSerializationSchema[ ADT <: FlinkEvent]( sinkName: String, - config: FlinkConfig[ADT] -) extends KafkaRecordSerializationSchema[E] + config: FlinkConfig[ADT], + schemaRegistryClient: SchemaRegistryClient, + toKV: PartialFunction[ADT, (Option[AnyRef], AnyRef)] +) extends KafkaRecordSerializationSchema[ADT] with LazyLogging { val sinkConfig: KafkaSinkConfig = { @@ -67,55 +91,11 @@ abstract class ConfluentAvroRegistryKafkaRecordSerializationSchema[ 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, + element: ADT, context: KafkaRecordSerializationSchema.KafkaSinkContext, timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = { - val (k, v) = toKeyValue(element) + val (k, v) = toKV(element) val key = keySerializer.flatMap(ks => k.map(kk => ks.serialize(topic, kk))) val value = valueSerializer.serialize(topic, v) diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala index f597218..101f704 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala @@ -1,6 +1,12 @@ package io.epiphanous.flinkrunner.serde -import io.confluent.kafka.schemaregistry.client.SchemaMetadata +import io.confluent.kafka.schemaregistry.avro.AvroSchema + +import scala.language.higherKinds +import io.confluent.kafka.schemaregistry.client.{ + MockSchemaRegistryClient, + SchemaMetadata +} import io.epiphanous.flinkrunner.UnitSpec import io.epiphanous.flinkrunner.model._ import org.apache.avro.Schema @@ -12,8 +18,8 @@ import java.time.Instant class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest extends UnitSpec { - val factory = new NoJobFactory[MyAvroADT] - val optConfig: String = + val factory = new NoJobFactory[MyAvroADT] + val optConfig: String = s""" |sinks { | test { @@ -29,17 +35,23 @@ class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest | } |} |""".stripMargin - val config = new FlinkConfig[MyAvroADT]( + val config = new FlinkConfig[MyAvroADT]( Array.empty[String], factory, Map.empty, Some(optConfig) ) - val serde = - new MyADTConfluentAvroRegistryKafkaRecordSerializationSchema[ - MyAvroADT]( + val schemaRegistryClient = new MockSchemaRegistryClient() + + val serde = + new ConfluentAvroRegistryKafkaRecordSerializationSchema[MyAvroADT]( "test", // sink name must match this - config + config, + schemaRegistryClient, + toKV = { + case a: AWrapper => (Some(a.$id), a.value) + case b: BWrapper => (Some(b.$id), b.value) + } ) // helper to return the class name of the object passed in (without a $ at the end) @@ -73,9 +85,20 @@ class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest } // some test fixtures - val aRecord: ARecord = ARecord("a123", 17, 32.2, Instant.now()) - val aWrapper: AWrapper = AWrapper(aRecord) - val aName: String = className(aRecord) + val aRecord: ARecord = ARecord("a123", 17, 32.2, Instant.now()) + val aWrapper: AWrapper = AWrapper(aRecord) + val aName: String = className(aRecord) + + val stringSchema: AvroSchema = new AvroSchema("""{"type":"string"}""") + schemaRegistryClient.register( + s"test-key", + stringSchema + ) + schemaRegistryClient.register( + aName, + new AvroSchema(ARecord.SCHEMA$) + ) + val keySchemaInfo: SchemaMetadata = serde.schemaRegistryClient.getLatestSchemaMetadata("test-key") val aSchemaInfo: SchemaMetadata = @@ -94,7 +117,7 @@ class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest } it should "serialize to a producer record" in { - val (aKey, aValue) = serde.toKeyValue(aWrapper) + val (aKey, aValue) = serde.toKV(aWrapper) val aWrapperKeyExpectedBytes: Option[Array[Byte]] = aKey.map(k => binaryEncode(k, keySchemaInfo)) val aWrapperValueExpectedBytes: Array[Byte] = diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala deleted file mode 100644 index 9b2acc4..0000000 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/MyADTConfluentAvroRegistryKafkaRecordSerializationSchema.scala +++ /dev/null @@ -1,69 +0,0 @@ -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) - } -} From 988a283c7105ba27d7e26f0bfb07cb6f577519ca Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Sat, 4 Dec 2021 09:30:15 -0500 Subject: [PATCH 7/8] allow multiple events to be generated from a single kafka key/value record --- ...stryKafkaRecordDeserializationSchema.scala | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala index 9fe9c95..57c0d5a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -9,6 +9,7 @@ import io.epiphanous.flinkrunner.model.{ FlinkEvent, KafkaSourceConfig } +import org.apache.avro.specific.SpecificRecord 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 @@ -19,13 +20,19 @@ import java.util /** * A deserialization schema that uses the provided confluent avro schema * registry client and `fromKV` partial function to deserialize a kafka - * key/value pair into an instance of a flink runner ADT. + * key/value pair into instances of a flink runner ADT. * * In order to decouple the shape of the flink runner ADT types from the * types that are serialized in kafka, a user of this class must provide a * `fromKV` partial function that maps from the specific key and value pair - * (key is optional) deserialized from the kafka source into an instance of - * the flink runner ADT. + * (key is optional) deserialized from the kafka source into a sequence of + * instances of the flink runner ADT. This means a single kafka record + * could generate multiple events. + * + * Note: that the number and size of the produced events should be + * relatively small. Depending on the source implementation records can be + * buffered in memory or collecting records might delay emitting the next + * checkpoint barrier. * * Usually, `fromKV` is as simple as providing a set of cases. Consider the * following example, where `A` and `B` are subclasses of the flink runner @@ -34,11 +41,12 @@ import java.util * ignore the key and have defined our ADT types to be wrappers around the * deserialized records. However, you can use the deserialized key and * value in any way that makes sense for your application. + * * {{{ * { * // (key,value) => ADT - * case (_, a:ASpecific) => A(a) - * case (_, b:BSpecific) => B(b) + * case (_, a:ASpecific) => Seq(A(a)) + * case (_, b:BSpecific) => Seq(B(b)) * } * }}} * @param sourceName @@ -48,18 +56,17 @@ import java.util * @param schemaRegistryClient * the schema registry client * @param fromKV - * a partial function that should return a flink runner adt instance when - * passed a deserialized kafka key/value pair + * a partial function that should return a sequence of zero or more flink + * runner adt instances when passed a deserialized kafka key/value pair * @tparam ADT * the flink runner ADT type */ class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ - ADT <: FlinkEvent -]( + ADT <: FlinkEvent]( sourceName: String, config: FlinkConfig[ADT], schemaRegistryClient: SchemaRegistryClient, - fromKV: PartialFunction[(Option[AnyRef], AnyRef), ADT] + fromKV: PartialFunction[(Option[AnyRef], AnyRef), Seq[ADT]] ) extends KafkaRecordDeserializationSchema[ADT] with LazyLogging { @@ -95,7 +102,7 @@ class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ val key = keyDeserializer.map(ds => ds.deserialize(topic, record.key())) val value = valueDeserializer.deserialize(topic, record.value()) - if (Option(value).nonEmpty) out.collect(fromKV(key, value)) + if (Option(value).nonEmpty) fromKV(key, value).map(out.collect) } override def getProducedType: TypeInformation[ADT] = From 000e3d65450fb58340c1d5a5fefc43ee1ea3bea7 Mon Sep 17 00:00:00 2001 From: Robert Lyons Date: Fri, 10 Dec 2021 06:43:53 -0500 Subject: [PATCH 8/8] added rabbitmq source/sink and modernized kafka src/sink, jdbc sink, elastic sink --- README.md | 16 +- build.sbt | 25 +- .../epiphanous/flinkrunner/FlinkRunner.scala | 227 ++++++++++++++---- .../flinkrunner/FlinkRunnerFactory.scala | 12 + .../membership/StableBloomFilter.scala | 2 +- .../flinkrunner/flink/BaseFlinkJob.scala | 13 +- .../flinkrunner/model/FlinkConfig.scala | 158 +++++++++--- .../model/FlinkConnectorName.scala | 2 + .../model/RabbitMQConnectionInfo.scala | 67 ++++++ .../flinkrunner/model/SinkConfig.scala | 26 +- .../flinkrunner/model/SourceConfig.scala | 54 +++++ .../operator/EnrichmentAsyncFunction.scala | 2 +- .../io/epiphanous/flinkrunner/package.scala | 7 - ...stryKafkaRecordDeserializationSchema.scala | 3 +- ...gistryKafkaRecordSerializationSchema.scala | 2 - .../flinkrunner/util/JdbcSink.scala | 1 + 16 files changed, 494 insertions(+), 123 deletions(-) create mode 100644 src/main/scala/io/epiphanous/flinkrunner/model/RabbitMQConnectionInfo.scala delete mode 100644 src/main/scala/io/epiphanous/flinkrunner/package.scala diff --git a/README.md b/README.md index 8970feb..a03a60b 100644 --- a/README.md +++ b/README.md @@ -41,7 +41,7 @@ ## Maven Dependency `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. +built against Flink 1.14 with Scala 2.12 and JDK 11. ```sbtshell libraryDependencies += "io.epiphanous" %% "flinkrunner" % @@ -201,10 +201,10 @@ abstract class BaseFlinkJob[DS, OUT <: FlinkEvent: TypeInformation] extends Lazy * @param config implicit flink job config * @return data output stream */ - def flow()(implicit config: FlinkConfig, env: SEE): DataStream[OUT] = + def flow()(implicit config: FlinkConfig, env: StreamExecutionEnvironment): DataStream[OUT] = source |> transform |# maybeSink - def run()(implicit config: FlinkConfig, env: SEE): Either[Iterator[OUT], Unit] = { + def run()(implicit config: FlinkConfig, env: StreamExecutionEnvironment): Either[Iterator[OUT], Unit] = { logger.info(s"\nSTARTING FLINK JOB: ${config.jobName} ${config.jobArgs.mkString(" ")}\n") @@ -222,7 +222,7 @@ abstract class BaseFlinkJob[DS, OUT <: FlinkEvent: TypeInformation] extends Lazy * 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 + def source()(implicit config: FlinkConfig, env: StreamExecutionEnvironment): DS /** * Primary method to transform the source data stream into the output data stream. The output of @@ -232,7 +232,7 @@ abstract class BaseFlinkJob[DS, OUT <: FlinkEvent: TypeInformation] extends Lazy * @param config implicit flink job config * @return output data stream */ - def transform(in: DS)(implicit config: FlinkConfig, env: SEE): DataStream[OUT] + def transform(in: DS)(implicit config: FlinkConfig, env: StreamExecutionEnvironment): DataStream[OUT] /** * Writes the transformed data stream to configured output sinks. @@ -240,7 +240,7 @@ abstract class BaseFlinkJob[DS, OUT <: FlinkEvent: TypeInformation] extends Lazy * @param out a transformed stream from [[transform()]] * @param config implicit flink job config */ - def sink(out: DataStream[OUT])(implicit config: FlinkConfig, env: SEE): Unit = + def sink(out: DataStream[OUT])(implicit config: FlinkConfig, env: StreamExecutionEnvironment): Unit = config.getSinkNames.foreach(name => out.toSink(name)) /** @@ -250,7 +250,7 @@ abstract class BaseFlinkJob[DS, OUT <: FlinkEvent: TypeInformation] extends Lazy * @param out the output data stream to pass into [[sink()]] * @param config implicit flink job config */ - def maybeSink(out: DataStream[OUT])(implicit config: FlinkConfig, env: SEE): Unit = + def maybeSink(out: DataStream[OUT])(implicit config: FlinkConfig, env: StreamExecutionEnvironment): Unit = if (!config.mockEdges) sink(out) } @@ -274,7 +274,7 @@ abstract class FlinkJob[IN <: FlinkEvent: TypeInformation, OUT <: FlinkEvent: Ty * Returns source data stream to pass into [[transform()]]. This can be overridden by subclasses. * @return input data stream */ - def source()(implicit config: FlinkConfig, env: SEE): DataStream[IN] = + def source()(implicit config: FlinkConfig, env: StreamExecutionEnvironment): DataStream[IN] = fromSource[IN](getEventSourceName) |# maybeAssignTimestampsAndWatermarks } diff --git a/build.sbt b/build.sbt index c565166..2efabb2 100644 --- a/build.sbt +++ b/build.sbt @@ -45,16 +45,21 @@ val V = new { 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 + "org.apache.flink" %% "flink-scala" % V.flink % Provided, + "org.apache.flink" %% "flink-streaming-scala" % V.flink % Provided, + "org.apache.flink" %% "flink-cep-scala" % V.flink % Provided, + "org.apache.flink" %% "flink-table-planner" % V.flink % Provided, + "org.apache.flink" %% "flink-connector-kafka" % V.flink, + "org.apache.flink" %% "flink-connector-kinesis" % V.flink, + "org.apache.flink" %% "flink-connector-cassandra" % V.flink, + "org.apache.flink" %% "flink-connector-elasticsearch7" % V.flink, + "org.apache.flink" %% "flink-connector-jdbc" % V.flink, + "org.apache.flink" %% "flink-connector-rabbitmq" % V.flink, + "org.apache.flink" % "flink-connector-files" % V.flink, + "org.apache.flink" %% "flink-table-api-scala-bridge" % V.flink, + "org.apache.flink" %% "flink-statebackend-rocksdb" % V.flink, + "org.apache.flink" % "flink-avro-confluent-registry" % V.flink, + "org.apache.flink" %% "flink-test-utils" % V.flink % Test ) val loggingDeps = Seq( diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index 1192bdb..0fd2826 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -2,11 +2,7 @@ package io.epiphanous.flinkrunner import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.operator.AddToJdbcBatchFunction -import io.epiphanous.flinkrunner.util.{ - BoundedLatenessWatermarkStrategy, - JdbcSink -} +import io.epiphanous.flinkrunner.util.BoundedLatenessWatermarkStrategy import org.apache.flink.api.common.eventtime.WatermarkStrategy import org.apache.flink.api.common.functions.RuntimeContext import org.apache.flink.api.common.serialization.{ @@ -15,6 +11,12 @@ import org.apache.flink.api.common.serialization.{ SerializationSchema } import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.connector.jdbc.{ + JdbcConnectionOptions, + JdbcExecutionOptions, + JdbcSink, + JdbcStatementBuilder +} import org.apache.flink.connector.kafka.sink.KafkaSink import org.apache.flink.connector.kafka.source.KafkaSource import org.apache.flink.core.fs.Path @@ -33,6 +35,7 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.{ } import org.apache.flink.streaming.api.scala.{DataStream, _} import org.apache.flink.streaming.connectors.cassandra.CassandraSink +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase.FlushBackoffType 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 @@ -40,12 +43,15 @@ import org.apache.flink.streaming.connectors.kinesis.{ FlinkKinesisConsumer, FlinkKinesisProducer } +import org.apache.flink.streaming.connectors.rabbitmq.{RMQSink, RMQSource} +import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment 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 java.sql.PreparedStatement import scala.collection.JavaConverters._ import scala.util.matching.Regex @@ -59,9 +65,11 @@ class FlinkRunner[ADT <: FlinkEvent]( optConfig: Option[String] = None) extends LazyLogging { - val config: FlinkConfig[ADT] = + val config: FlinkConfig[ADT] = factory.getFlinkConfig(args, sources, optConfig) - val env: SEE = config.configureStreamExecutionEnvironment + val env: StreamExecutionEnvironment = + config.configureStreamExecutionEnvironment + val tableEnv: StreamTableEnvironment = StreamTableEnvironment.create(env) /** * Invoke a job based on the job name and arguments passed in. If the job @@ -220,6 +228,7 @@ class FlinkRunner[ADT <: FlinkEvent]( val stream = (src match { case src: KafkaSourceConfig => fromKafka(src) case src: KinesisSourceConfig => fromKinesis(src) + case src: RabbitMQSourceConfig => fromRabbitMQ(src) case src: FileSourceConfig => fromFile(src) case src: SocketSourceConfig => fromSocket(src) case src: CollectionSourceConfig => fromCollection(src) @@ -239,19 +248,23 @@ class FlinkRunner[ADT <: FlinkEvent]( */ def fromKafka[E <: ADT: TypeInformation]( srcConfig: KafkaSourceConfig - ): DataStream[E] = + ): DataStream[E] = { + val ksb = KafkaSource + .builder[E]() + .setProperties(srcConfig.properties) + .setStartingOffsets(srcConfig.startingOffsets) + .setDeserializer( + config + .getKafkaRecordDeserializationSchema[E]( + srcConfig.name + ) + ) + val kafkaSrcBuilder = + if (srcConfig.bounded) ksb.setBounded(srcConfig.stoppingOffsets) + else ksb env .fromSource( - KafkaSource - .builder[E]() - .setProperties(srcConfig.properties) - .setDeserializer( - config - .getKafkaRecordDeserializationSchema[E]( - srcConfig.name - ) - ) - .build(), + kafkaSrcBuilder.build(), srcConfig.watermarkStrategy match { case "bounded out of order" => boundedOutOfOrderWatermarks[E]() @@ -260,6 +273,7 @@ class FlinkRunner[ADT <: FlinkEvent]( }, srcConfig.label ) + } /** * Configure stream from kinesis. @@ -273,17 +287,15 @@ class FlinkRunner[ADT <: FlinkEvent]( */ def fromKinesis[E <: ADT: TypeInformation]( srcConfig: KinesisSourceConfig - ): DataStream[E] = { - val consumer = + ): DataStream[E] = + env.addSource( new FlinkKinesisConsumer[E]( srcConfig.stream, config .getKinesisDeserializationSchema[E](srcConfig.name), srcConfig.properties ) - env - .addSource(consumer) - } + ) /** * Configure stream from file source. @@ -361,6 +373,32 @@ class FlinkRunner[ADT <: FlinkEvent]( .deserialize(bytes) ) + /** + * Configure a stream from rabbitmq. + * @param srcConfig + * a RabbitMQSourceConfig instance + * @tparam E + * instance type of flink runner ADT + * @return + * DataStream[E] + */ + def fromRabbitMQ[E <: ADT: TypeInformation]( + srcConfig: RabbitMQSourceConfig): DataStream[E] = { + val name = srcConfig.name + val connConfig = srcConfig.connectionInfo.rmqConfig + val deserializationSchema = config.getRMQDeserializationSchema[E](name) + env + .addSource( + new RMQSource( + connConfig, + srcConfig.queue, + srcConfig.useCorrelationId, + deserializationSchema + ) + ) + .setParallelism(1) // required to get exactly once semantics + } + /** * Returns the actual path to a resource file named filename or * filename.gz. @@ -433,6 +471,7 @@ class FlinkRunner[ADT <: FlinkEvent]( toKafka[E](stream, s).uid(label).name(label) case s: KinesisSinkConfig => toKinesis[E](stream, s).uid(label).name(label) + case s: RabbitMQSinkConfig => toRabbitMQ[E](stream, s) case s: FileSinkConfig => toFile[E](stream, s).uid(label).name(label) case s: SocketSinkConfig => toSocket[E](stream, s).uid(label).name(label) @@ -510,30 +549,60 @@ class FlinkRunner[ADT <: FlinkEvent]( } /** - * Send stream to a socket sink. - * + * A jdbc sink. * @param stream - * the data stream + * a data stream * @param sinkConfig - * a sink configuration + * a JdbcSinkConfig object * @tparam E - * stream element type + * the type of elements in the data stream * @return - * DataStreamSink[E] + * DataStreamSink */ 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]] - ) + ): DataStreamSink[E] = { + val sinkProps = sinkConfig.properties + val addToJdbcBatchFunction = + config.getAddToJdbcBatchFunction[E](sinkConfig.name) + val statementBuilder = { + new JdbcStatementBuilder[E] { + override def accept(ps: PreparedStatement, element: E): Unit = + addToJdbcBatchFunction.addToJdbcStatement(element, ps) + } + } + val executionOptions = JdbcExecutionOptions + .builder() + .withMaxRetries( + Option(sinkProps.getProperty("max.retries")) + .map(o => o.toInt) + .getOrElse(JdbcExecutionOptions.DEFAULT_MAX_RETRY_TIMES) + ) + .withBatchSize( + Option(sinkProps.getProperty("batch.size")) + .map(o => o.toInt) + .getOrElse(JdbcExecutionOptions.DEFAULT_SIZE) ) + .withBatchIntervalMs( + Option(sinkProps.getProperty("batch.interval.ms")) + .map(o => o.toLong) + .getOrElse(60L) + ) + .build() + val connectionOptions = + new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() + .withUrl(sinkConfig.url) + .build() + stream.addSink( + JdbcSink.sink( + sinkConfig.query, + statementBuilder, + executionOptions, + connectionOptions + ) + ) + } /** * Send stream to a rolling file sink. @@ -691,21 +760,17 @@ class FlinkRunner[ADT <: FlinkEvent]( stream: DataStream[E], sinkConfig: ElasticsearchSinkConfig ): DataStreamSink[E] = { - val hosts = sinkConfig.transports.map { s => + 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]( + val esSinkBuilder = 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 - ) - ) + .filterNot(f => f.getName.startsWith("$")) .foldLeft(Map.empty[String, Any]) { case (a, f) => f.setAccessible(true) val name = f.getName @@ -719,8 +784,76 @@ class FlinkRunner[ADT <: FlinkEvent]( val req = Requests.indexRequest(sinkConfig.index).source(data) indexer.add(req) } - ).build() - stream.addSink(esSink) + ) + + val props = sinkConfig.properties + Option(props.getProperty("bulk.flush.backoff.enable")) + .map(_.toBoolean) + .foreach(esSinkBuilder.setBulkFlushBackoff) + Option(props.getProperty("bulk.flush.backoff.type")) + .map(_.toUpperCase() match { + case "CONSTANT" => FlushBackoffType.CONSTANT + case "EXPONENTIAL" => FlushBackoffType.EXPONENTIAL + case t => + logger.warn( + s"invalid bulk.flush.backoff.type value '$t'; using CONSTANT" + ) + FlushBackoffType.CONSTANT + }) + .foreach(esSinkBuilder.setBulkFlushBackoffType) + Option( + props.getProperty("bulk.flush.backoff.delay") + ).map(_.toLong) + .foreach(esSinkBuilder.setBulkFlushBackoffDelay) + Option(props.getProperty("bulk.flush.backoff.retries")) + .map(_.toInt) + .foreach(esSinkBuilder.setBulkFlushBackoffRetries) + Option(props.getProperty("bulk.flush.max.actions")) + .map(_.toInt) + .foreach(esSinkBuilder.setBulkFlushMaxActions) + Option(props.getProperty("bulk.flush.max.size.mb")) + .map(_.toInt) + .foreach(esSinkBuilder.setBulkFlushMaxSizeMb) + Option(props.getProperty("bulk.flush.interval.ms")) + .map(_.toLong) + .foreach(esSinkBuilder.setBulkFlushInterval) + stream.addSink(esSinkBuilder.build()) + } + + /** + * Configure streaming to a rabbitmq sink. + * @param stream + * the stream to send to the sink + * @param sinkConfig + * a RabbitMQSinkConfig instance + * @tparam E + * type of the flink runner ADT events in the stream + * @return + * DataStreamSink[E] + */ + def toRabbitMQ[E <: ADT: TypeInformation]( + stream: DataStream[E], + sinkConfig: RabbitMQSinkConfig + ): DataStreamSink[E] = { + + val name = sinkConfig.name + val connConfig = sinkConfig.connectionInfo.rmqConfig + val serializationSchema = + config.getSerializationSchema[E](sinkConfig.name) + + stream.addSink( + config.getRabbitPublishOptions[E](sinkConfig.name) match { + case Some(p) => new RMQSink(connConfig, serializationSchema, p) + case None => + sinkConfig.queue match { + case Some(q) => new RMQSink(connConfig, q, serializationSchema) + case None => + throw new RuntimeException( + s"RabbitMQ config requires either a queue name or publishing options for sink $name" + ) + } + } + ) } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala index 1623da8..2b063c7 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunnerFactory.scala @@ -20,6 +20,10 @@ import org.apache.flink.streaming.connectors.kinesis.serialization.{ KinesisDeserializationSchema, KinesisSerializationSchema } +import org.apache.flink.streaming.connectors.rabbitmq.{ + RMQDeserializationSchema, + RMQSinkPublishOptions +} trait FlinkRunnerFactory[ADT <: FlinkEvent] { @@ -81,4 +85,12 @@ trait FlinkRunnerFactory[ADT <: FlinkEvent] { def getAvroCoder(name: String, config: FlinkConfig[ADT]): AvroCoder[_] = ??? + + def getRMQDeserializationSchema[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): RMQDeserializationSchema[E] = ??? + + def getRabbitPublishOptions[E <: ADT]( + name: String, + config: FlinkConfig[ADT]): Option[RMQSinkPublishOptions[E]] = None } 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 7a10fde..62e1d46 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/algorithm/membership/StableBloomFilter.scala @@ -1,7 +1,7 @@ package io.epiphanous.flinkrunner.algorithm.membership -import com.google.common.hash.{Funnel, HashFunction} import com.google.common.hash.Hashing.murmur3_128 +import com.google.common.hash.{Funnel, HashFunction} import java.nio.ByteBuffer import scala.util.Random diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala index 53fbcb6..9f065b4 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/BaseFlinkJob.scala @@ -1,12 +1,16 @@ package io.epiphanous.flinkrunner.flink import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.FlinkRunner import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} 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.scala.DataStream +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} +import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment import scala.util.Try @@ -28,8 +32,9 @@ abstract class BaseFlinkJob[ ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) extends LazyLogging { - val config: FlinkConfig[ADT] = runner.config - val env: SEE = runner.env + val config: FlinkConfig[ADT] = runner.config + val env: StreamExecutionEnvironment = runner.env + val tableEnv: StreamTableEnvironment = runner.tableEnv /** * A pipeline for transforming a single stream. Passes the output of diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala index 07cc530..a0b4771 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -1,13 +1,35 @@ package io.epiphanous.flinkrunner.model -import com.typesafe.config.{ConfigFactory, ConfigObject} +import com.typesafe.config.{Config, ConfigFactory, ConfigObject} import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.FlinkRunnerFactory +import io.epiphanous.flinkrunner.avro.AvroCoder import io.epiphanous.flinkrunner.model.ConfigToProps.RichConfigObject -import io.epiphanous.flinkrunner.{FlinkRunnerFactory, SEE} +import io.epiphanous.flinkrunner.operator.AddToJdbcBatchFunction +import org.apache.flink.api.common.RuntimeExecutionMode +import org.apache.flink.api.common.serialization.{ + DeserializationSchema, + Encoder, + SerializationSchema +} import org.apache.flink.api.java.utils.ParameterTool +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend +import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.connectors.kafka.{ + KafkaDeserializationSchema, + KafkaSerializationSchema +} +import org.apache.flink.streaming.connectors.kinesis.serialization.{ + KinesisDeserializationSchema, + KinesisSerializationSchema +} +import org.apache.flink.streaming.connectors.rabbitmq.{ + RMQDeserializationSchema, + RMQSinkPublishOptions +} import java.io.File import java.time.Duration @@ -35,7 +57,7 @@ class FlinkConfig[ADT <: FlinkEvent]( (n, a, ParameterTool.fromArgs(a)) } - val _config = { + val _config: Config = { val sc = Seq(ConfigFactory.load(), ConfigFactory.load("flink-runner.conf")) val ocf = @@ -49,17 +71,18 @@ class FlinkConfig[ADT <: FlinkEvent]( ) } - def getCollectionSource(name: String) = + def getCollectionSource(name: String): Seq[Array[Byte]] = sources.getOrElse( name, throw new RuntimeException(s"missing collection source $name") ) - val systemName = _config.getString("system.name") + val systemName: String = _config.getString("system.name") - val jobs = _config.getObject("jobs").unwrapped().keySet().asScala.toSet + val jobs: Set[String] = + _config.getObject("jobs").unwrapped().keySet().asScala.toSet - def getJobConfig(name: String) = _config.getConfig(s"jobs.$name") + def getJobConfig(name: String): Config = _config.getConfig(s"jobs.$name") private def _s(path: String): (String, String) = { val jpath = _j(path) @@ -86,36 +109,55 @@ class FlinkConfig[ADT <: FlinkEvent]( case (_, p) => _config.getString(p) } + def getStringOpt(path: String): Option[String] = Try( + getString(path) + ).toOption + def getStringList(path: String): List[String] = _s(path) match { case ("a", p) => jobParams.get(p).split("[, ]+").toList case (_, p) => _config.getStringList(p).asScala.toList } + def getStringListOpt(path: String): List[String] = + Try(getStringList(path)).getOrElse(List.empty[String]) + def getInt(path: String): Int = _s(path) match { case ("a", p) => jobParams.getInt(p) case (_, p) => _config.getInt(p) } + def getIntOpt(path: String): Option[Int] = Try(getInt(path)).toOption + def getLong(path: String): Long = _s(path) match { case ("a", p) => jobParams.getLong(p) case (_, p) => _config.getLong(p) } + def getLongOpt(path: String): Option[Long] = Try(getLong(path)).toOption + def getBoolean(path: String): Boolean = _s(path) match { case ("a", p) => jobParams.getBoolean(p) case (_, p) => _config.getBoolean(p) } + def getBooleanOpt(path: String): Option[Boolean] = Try( + getBoolean(path) + ).toOption + def getDouble(path: String): Double = _s(path) match { case ("a", p) => jobParams.getDouble(p) case (_, p) => _config.getDouble(p) } + def getDoubleOpt(path: String): Option[Double] = Try( + getDouble(path) + ).toOption + def getDuration(path: String): Duration = _s(path) match { case ("a", p) => @@ -125,6 +167,10 @@ class FlinkConfig[ADT <: FlinkEvent]( case (_, p) => _config.getDuration(p) } + def getDurationOpt(path: String): Option[Duration] = Try( + getDuration(path) + ).toOption + def getProperties(path: String): Properties = (_s(path) match { case ("a", p) => @@ -146,45 +192,62 @@ class FlinkConfig[ADT <: FlinkEvent]( // def getJobInstance = factory.getJobInstance(jobName, this) - def getDeserializationSchema[E <: ADT](name: String) = + def getDeserializationSchema[E <: ADT]( + name: String): DeserializationSchema[E] = factory.getDeserializationSchema[E](name, this) - def getKafkaDeserializationSchema[E <: ADT](name: String) = + def getKafkaDeserializationSchema[E <: ADT]( + name: String): KafkaDeserializationSchema[E] = factory.getKafkaDeserializationSchema[E](name, this) def getKafkaRecordDeserializationSchema[E <: ADT]( name: String): KafkaRecordDeserializationSchema[E] = factory.getKafkaRecordDeserializationSchema[E](name, this) - def getKinesisDeserializationSchema[E <: ADT](name: String) = + def getKinesisDeserializationSchema[E <: ADT]( + name: String): KinesisDeserializationSchema[E] = factory.getKinesisDeserializationSchema[E](name, this) - def getSerializationSchema[E <: ADT](name: String) = + def getSerializationSchema[E <: ADT]( + name: String): SerializationSchema[E] = factory.getSerializationSchema[E](name, this) - def getKafkaSerializationSchema[E <: ADT](name: String) = + def getKafkaSerializationSchema[E <: ADT]( + name: String): KafkaSerializationSchema[E] = factory.getKafkaSerializationSchema[E](name, this) - def getKafkaRecordSerializationSchema[E <: ADT](name: String) = + def getKafkaRecordSerializationSchema[E <: ADT]( + name: String): KafkaRecordSerializationSchema[E] = factory.getKafkaRecordSerializationSchema[E](name, this) - def getKinesisSerializationSchema[E <: ADT](name: String) = + def getKinesisSerializationSchema[E <: ADT]( + name: String): KinesisSerializationSchema[E] = factory.getKinesisSerializationSchema[E](name, this) - def getEncoder[E <: ADT](name: String) = + def getEncoder[E <: ADT](name: String): Encoder[E] = factory.getEncoder[E](name, this) - def getAddToJdbcBatchFunction[E <: ADT](name: String) = + def getAddToJdbcBatchFunction[E <: ADT]( + name: String): AddToJdbcBatchFunction[E] = factory.getAddToJdbcBatchFunction[E](name, this) - def getBucketAssigner[E <: ADT](name: String) = + def getBucketAssigner[E <: ADT]( + name: String): BucketAssigner[E, String] = factory.getBucketAssigner[E](name, this) + def getRMQDeserializationSchema[E <: ADT]( + name: String): RMQDeserializationSchema[E] = + factory.getRMQDeserializationSchema(name, this) + + def getRabbitPublishOptions[E <: ADT]( + name: String): Option[RMQSinkPublishOptions[E]] = + factory.getRabbitPublishOptions[E](name, this) + @deprecated( "Use the ConfluentAvroRegistryKafkaRecordSerialization and ...Deserialization classes instead", "4.0.0" ) - def getAvroCoder(name: String) = + def getAvroCoder(name: String): AvroCoder[_] = factory.getAvroCoder(name, this) def getSourceConfig(name: String): SourceConfig = @@ -208,12 +271,13 @@ class FlinkConfig[ADT <: FlinkEvent]( getObject("sinks").unwrapped().keySet().asScala.toSeq } - lazy val environment = getString("environment") - lazy val isDev = environment.startsWith("dev") - lazy val isStage = environment.startsWith("stag") - lazy val isProd = environment.startsWith("prod") + lazy val environment: String = + getStringOpt("environment").getOrElse("production") + lazy val isDev: Boolean = environment.startsWith("dev") + lazy val isStage: Boolean = environment.startsWith("stag") + lazy val isProd: Boolean = environment.startsWith("prod") - def configureStreamExecutionEnvironment: SEE = { + def configureStreamExecutionEnvironment: StreamExecutionEnvironment = { val env = if (isDev) StreamExecutionEnvironment.createLocalEnvironment(1) @@ -242,10 +306,12 @@ class FlinkConfig[ADT <: FlinkEvent]( env.getCheckpointConfig.setCheckpointStorage(checkpointUrl) } + env.setRuntimeMode(executionRuntimeMode) + env } - def getWatermarkStrategy(ws: String) = + def getWatermarkStrategy(ws: String): String = ws.toLowerCase.replaceAll("[^a-z]", "") match { case "boundedlateness" => "bounded lateness" case "boundedoutoforderness" => "bounded out of orderness" @@ -257,24 +323,36 @@ class FlinkConfig[ADT <: FlinkEvent]( ) } - lazy val watermarkStrategy = getWatermarkStrategy( + lazy val watermarkStrategy: String = getWatermarkStrategy( getString("watermark.strategy") ) - lazy val systemHelp = _config.getString("system.help") - lazy val jobHelp = getString("help") - lazy val jobDescription = getString("description") - lazy val globalParallelism = getInt("global.parallelism") - lazy val checkpointInterval = getLong("checkpoint.interval") - lazy val checkpointMinPause = getDuration("checkpoint.min.pause") - lazy val checkpointMaxConcurrent = getInt("checkpoint.max.concurrent") - lazy val checkpointUrl = getString("checkpoint.url") - lazy val checkpointFlash = getBoolean("checkpoint.flash") - lazy val stateBackend = getString("state.backend").toLowerCase - lazy val checkpointIncremental = getBoolean("checkpoint.incremental") - lazy val showPlan = getBoolean("show.plan") - lazy val mockEdges = isDev && getBoolean("mock.edges") - lazy val maxLateness = getDuration("max.lateness") - lazy val maxIdleness = getDuration("max.idleness") + lazy val systemHelp: String = _config.getString("system.help") + lazy val jobHelp: String = getString("help") + lazy val jobDescription: String = getString("description") + lazy val globalParallelism: Int = getInt("global.parallelism") + lazy val checkpointInterval: Long = getLong("checkpoint.interval") + lazy val checkpointMinPause: Duration = getDuration( + "checkpoint.min.pause" + ) + lazy val checkpointMaxConcurrent: Int = getInt( + "checkpoint.max.concurrent" + ) + lazy val checkpointUrl: String = getString("checkpoint.url") + lazy val checkpointFlash: Boolean = getBoolean("checkpoint.flash") + lazy val stateBackend: String = getString("state.backend").toLowerCase + lazy val checkpointIncremental: Boolean = getBoolean( + "checkpoint.incremental" + ) + lazy val showPlan: Boolean = getBoolean("show.plan") + lazy val mockEdges: Boolean = isDev && getBoolean("mock.edges") + lazy val maxLateness: Duration = getDuration("max.lateness") + lazy val maxIdleness: Duration = getDuration("max.idleness") + lazy val executionRuntimeMode: RuntimeExecutionMode = + getStringOpt("execution.runtime-mode").map(_.toUpperCase) match { + case Some("BATCH") => RuntimeExecutionMode.BATCH + case Some("AUTOMATIC") => RuntimeExecutionMode.AUTOMATIC + case _ => RuntimeExecutionMode.STREAMING + } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala index ba582b6..a0c23bc 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala @@ -24,4 +24,6 @@ object FlinkConnectorName extends Enum[FlinkConnectorName] { case object Collection extends FlinkConnectorName + case object RabbitMQ extends FlinkConnectorName + } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/RabbitMQConnectionInfo.scala b/src/main/scala/io/epiphanous/flinkrunner/model/RabbitMQConnectionInfo.scala new file mode 100644 index 0000000..16b98a9 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/RabbitMQConnectionInfo.scala @@ -0,0 +1,67 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig + +import java.util.Properties + +case class RabbitMQConnectionInfo( + uri: String, + networkRecoveryInterval: Option[Int] = None, + automaticRecovery: Option[Boolean] = None, + topologyRecovery: Option[Boolean] = None, + connectionTimeout: Option[Int] = None, + requestedChannelMax: Option[Int] = None, + requestedFrameMax: Option[Int] = None, + requestedHeartbeat: Option[Int] = None, + prefetchCount: Option[Int] = None, + deliveryTimeout: Option[Long] = None) { + def rmqConfig: RMQConnectionConfig = { + var cb = + new RMQConnectionConfig.Builder().setUri(uri) + cb = automaticRecovery + .map(cb.setAutomaticRecovery) + .getOrElse(cb) + cb = connectionTimeout + .map(cb.setConnectionTimeout) + .getOrElse(cb) + cb = deliveryTimeout + .map(cb.setDeliveryTimeout) + .getOrElse(cb) + cb = networkRecoveryInterval + .map(cb.setNetworkRecoveryInterval) + .getOrElse(cb) + cb = prefetchCount + .map(cb.setPrefetchCount) + .getOrElse(cb) + cb = requestedChannelMax + .map(cb.setRequestedChannelMax) + .getOrElse(cb) + cb = requestedFrameMax + .map(cb.setRequestedFrameMax) + .getOrElse(cb) + cb = requestedHeartbeat + .map(cb.setRequestedHeartbeat) + .getOrElse(cb) + cb = topologyRecovery + .map(cb.setTopologyRecoveryEnabled) + .getOrElse(cb) + cb.build() + } +} +object RabbitMQConnectionInfo { + def apply(uri: String, c: Properties): RabbitMQConnectionInfo = { + RabbitMQConnectionInfo( + uri, + Option(c.getProperty("network.recovery.interval")) + .map(_.toInt), + Option(c.getProperty("automatic.recovery")).map(_.toBoolean), + Option(c.getProperty("topology.recovery")).map(_.toBoolean), + Option(c.getProperty("connection.timeout")).map(_.toInt), + Option(c.getProperty("requested.channel.max")).map(_.toInt), + Option(c.getProperty("requested.frame.max")).map(_.toInt), + Option(c.getProperty("requested.heartbeat")).map(_.toInt), + Option(c.getProperty("prefetch.count")).map(_.toInt), + Option(c.getProperty("delivery.timeout")).map(_.toLong) + ) + } +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala index 68b0b39..c90825a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SinkConfig.scala @@ -63,6 +63,7 @@ object SinkConfig { JdbcSinkConfig( connector, name, + config.getString(s"$p.url"), config.getString(s"$p.query"), config.getProperties(s"$p.config") ) @@ -84,7 +85,19 @@ object SinkConfig { config.getProperties(s"$p.config") ) - case other => + case RabbitMQ => + val c = config.getProperties(s"$p.config") + val uri = config.getString(s"$p.uri") + RabbitMQSinkConfig( + connector, + name, + uri, + config.getBoolean(s"$p.use.correlation.id"), + RabbitMQConnectionInfo(uri, c), + Option(c.getProperty("queue")), + c + ) + case other => throw new RuntimeException( s"$other $name connector not valid sink (job ${config.jobName}" ) @@ -140,6 +153,7 @@ final case class SocketSinkConfig( final case class JdbcSinkConfig( connector: FlinkConnectorName = Jdbc, name: String, + url: String, query: String, properties: Properties) extends SinkConfig @@ -160,3 +174,13 @@ final case class ElasticsearchSinkConfig( `type`: String, properties: Properties) extends SinkConfig + +final case class RabbitMQSinkConfig( + connector: FlinkConnectorName = RabbitMQ, + name: String, + uri: String, + useCorrelationId: Boolean, + connectionInfo: RabbitMQConnectionInfo, + queue: Option[String], + properties: Properties) + extends SinkConfig diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala index 6c68c0d..cad9f79 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SourceConfig.scala @@ -2,6 +2,10 @@ package io.epiphanous.flinkrunner.model import com.google.common.collect.Maps import io.epiphanous.flinkrunner.model.FlinkConnectorName._ +import org.apache.flink.connector.kafka.source.enumerator.initializer.{ + NoStoppingOffsetsInitializer, + OffsetsInitializer +} import java.util import java.util.Properties @@ -50,6 +54,27 @@ object SourceConfig { config.getBoolean(s"$p.isKeyed"), watermarkStrategy, maxAllowedLateness, + config.getBooleanOpt(s"$p.bounded").getOrElse(false), + config.getStringOpt(s"$p.starting.offset") match { + case Some(o) if o.equalsIgnoreCase("earliest") => + OffsetsInitializer.earliest() + case Some(o) if o.equalsIgnoreCase("latest") => + OffsetsInitializer.latest() + case Some(o) if o.equalsIgnoreCase("committed") => + OffsetsInitializer.committedOffsets() + case Some(o) if o.matches("[0-9]+") => + OffsetsInitializer.timestamp(o.toLong) + case _ => OffsetsInitializer.latest() + }, + config.getStringOpt(s"$p.stopping.offset") match { + case Some(o) if o.equalsIgnoreCase("latest") => + OffsetsInitializer.latest() + case Some(o) if o.equalsIgnoreCase("committed") => + OffsetsInitializer.committedOffsets() + case Some(o) if o.matches("[0-9]+") => + OffsetsInitializer.timestamp(o.toLong) + case _ => new NoStoppingOffsetsInitializer() + }, config.getProperties(s"$p.config") ) case Kinesis => @@ -89,6 +114,20 @@ object SourceConfig { maxAllowedLateness, config.getProperties(s"$p.config") ) + case RabbitMQ => + val c = config.getProperties(s"$p.config") + val uri = config.getString(s"$p.uri") + RabbitMQSourceConfig( + connector, + name, + uri, + config.getBoolean(s"$p.use.correlation.id"), + config.getString(s"$p.queue"), + watermarkStrategy, + maxAllowedLateness, + RabbitMQConnectionInfo(uri, c), + c + ) case other => throw new RuntimeException( s"$other $name connector not valid source (job ${config.jobName}" @@ -109,6 +148,9 @@ final case class KafkaSourceConfig( isKeyed: Boolean, watermarkStrategy: String, maxAllowedLateness: Long, + bounded: Boolean = false, + startingOffsets: OffsetsInitializer, + stoppingOffsets: OffsetsInitializer, properties: Properties) extends SourceConfig @@ -148,3 +190,15 @@ final case class CollectionSourceConfig( maxAllowedLateness: Long, properties: Properties) extends SourceConfig + +final case class RabbitMQSourceConfig( + connector: FlinkConnectorName = RabbitMQ, + name: String, + uri: String, + useCorrelationId: Boolean, + queue: String, + watermarkStrategy: String, + maxAllowedLateness: Long, + connectionInfo: RabbitMQConnectionInfo, + properties: Properties) + extends SourceConfig diff --git a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala index 8c54587..6976a21 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/operator/EnrichmentAsyncFunction.scala @@ -5,11 +5,11 @@ import com.google.common.cache.{CacheBuilder, CacheLoader} import com.typesafe.scalalogging.LazyLogging import io.circe.Decoder import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkEvent} -import org.apache.flink.util.concurrent.Executors import org.apache.flink.streaming.api.scala.async.{ AsyncFunction, ResultFuture } +import org.apache.flink.util.concurrent.Executors import org.http4s.EntityDecoder import org.http4s.circe.jsonOf import org.http4s.client.blaze.BlazeClientBuilder diff --git a/src/main/scala/io/epiphanous/flinkrunner/package.scala b/src/main/scala/io/epiphanous/flinkrunner/package.scala deleted file mode 100644 index c2f8340..0000000 --- a/src/main/scala/io/epiphanous/flinkrunner/package.scala +++ /dev/null @@ -1,7 +0,0 @@ -package io.epiphanous - -import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment - -package object flinkrunner { - type SEE = StreamExecutionEnvironment -} diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala index 57c0d5a..2b2f646 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -9,7 +9,6 @@ import io.epiphanous.flinkrunner.model.{ FlinkEvent, KafkaSourceConfig } -import org.apache.avro.specific.SpecificRecord 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 @@ -102,7 +101,7 @@ class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ val key = keyDeserializer.map(ds => ds.deserialize(topic, record.key())) val value = valueDeserializer.deserialize(topic, record.value()) - if (Option(value).nonEmpty) fromKV(key, value).map(out.collect) + if (Option(value).nonEmpty) fromKV(key, value).foreach(out.collect) } override def getProducedType: TypeInformation[ADT] = diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala index 0f1dfaa..21537a0 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -9,9 +9,7 @@ import io.epiphanous.flinkrunner.model.{ 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} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala b/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala index bfa4657..92c5efc 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/JdbcSink.scala @@ -36,6 +36,7 @@ import scala.util.{Failure, Success, Try} * @tparam E * the class of sink elements. */ +@deprecated("Use org.apache.flink.connector.jdbc.JdbcSink instead", "4.0") class JdbcSink[E: TypeInformation]( sinkConfig: JdbcSinkConfig, batchFunction: AddToJdbcBatchFunction[E]