diff --git a/README.md b/README.md index 48deefa1..830ceb76 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,7 @@ ## Dependencies -`Flinkrunner 4` +`FlinkRunner 4` is [available on maven central](https://mvnrepository.com/artifact/io.epiphanous/flinkrunner_2.12) , built against Flink 1.16 with Scala 2.12 and JDK 11. You can add it to your sbt project: @@ -49,22 +49,25 @@ libraryDependencies += "io.epiphanous" %% "flinkrunner" % ``` replacing `` with the currently released version -of [`Flinkrunner` on maven](https://mvnrepository.com/artifact/io.epiphanous/flinkrunner_2.12) +of [`FlinkRunner` on maven](https://mvnrepository.com/artifact/io.epiphanous/flinkrunner_2.12) . ### Connectors -Flinkrunner is built to support many common data sources and sinks, including: - -| Connector | Dependency | Source | Sink | -|:--------------|:-------------------------------|:------:|:----:| -| file system | flink-connector-files | yes | yes | -| kafka | flink-connector-kafka | yes | yes | -| kinesis | flink-connector-kinesis | yes | yes | -| cassandra | flink-connector-cassandra | no | yes | -| elasticsearch | flink-connector-elasticsearch7 | no | yes | -| jdbc | flink-connector-jdbc | no | yes | -| rabbit mq | flink-connector-rabbitmq | yes | yes | +FlinkRunner is built to support many common data sources and sinks, including: + +| Connector | Dependency | Source | Sink | +|:-----------------------|:--------------------------------------------|:------:|:----:| +| file system | flink-connector-files | yes | yes | +| kafka | flink-connector-kafka | yes | yes | +| kinesis streams source | flink-connector-kinesis | yes | no | +| kinesis streams sink | flink-connector-aws-kinesis-streams | no | yes | +| kinesis firehose | flink-connector-aws-kinesis-firehose | no | yes | +| iceberg | iceberg-flink-runtime- | yes | yes | +| cassandra | flink-connector-cassandra | no | yes | +| elasticsearch | flink-connector-elasticsearch7 | no | yes | +| jdbc | flink-connector-jdbc | no | yes | +| rabbit mq | flink-connector-rabbitmq | yes | yes | You can add a dependency for a connector by dropping the library into flink's `lib` directory during deployment of your jobs. You should make sure to match the library's @@ -111,8 +114,8 @@ replacing `` with the version of flink used in `FlinkRunner`. ### Avro Support -Flinkrunner supports reading and writing avro messages with kafka and file systems. For -kafka sources and sinks, Flinkrunner uses binary encoding with Confluent schema registry. +FlinkRunner supports reading and writing avro messages with kafka and file systems. For +kafka sources and sinks, FlinkRunner uses binary encoding with Confluent schema registry. For file sources and sinks, you can select either standard or parquet avro encoding. Add the following dependencies if you need Avro and Confluent schema registry support: @@ -135,7 +138,7 @@ registry support. #### Schema Registry Support -Flinkrunner provides automatic support for serializing and deserializing in and out of +FlinkRunner provides automatic support for serializing and deserializing in and out of kafka using Confluent's [Avro schema registry](https://docs.confluent.io/platform/current/schema-registry/index.html) . To make use of this support, you need to do three things: @@ -181,14 +184,17 @@ build: Add use the `format = parquet` directive in your file sink configuration (more details below). +The [iceberg](https://iceberg.apache.org) source and sink also support parquet output by +default. + ### Logging -`Flinkrunner` uses [scala-logging](https://github.com/lightbend/scala-logging) for logging +`FlinkRunner` uses [scala-logging](https://github.com/lightbend/scala-logging) for logging on top of slf4j. In your implementation, you must provide a logging backend compatible with slf4j, such as [logback](https://logback.qos.ch/): ```sbtshell -libraryDependencies += "ch.qos.logback" % "logback-classic" % "1.2.11" +libraryDependencies += "ch.qos.logback" % "logback-classic" % ``` ### Complex Event Processing @@ -201,13 +207,16 @@ If you want to use the complex event processing library, add this dependency: ## What is FlinkRunner? -`FlinkRunner` helps you think about your flink jobs at a high level, so you can focus on -the event pipeline, not the plumbing. +`FlinkRunner` helps you think about your datastream api flink jobs at a high level, so you +can focus on the event pipeline, not the plumbing. It is not as high-level as Flink SQL, +but when you need to write multiple related data stream API jobs, FlinkRunner helps you to +avoid repetitive boiler plate code, simplifies configuration and simplifies usage of many +common flink streaming patterns. You have a set of related flink jobs that deal in a related set of data event -types. `Flinkrunner` helps you build one application to run those related jobs and +types. `FlinkRunner` helps you build one application to run those related jobs and coordinate the types. It also simplifies setting up common sources and sinks, so you can -control them purely with configuration, not code. `Flinkrunner` supports a variety of +control them purely with configuration, not code. `FlinkRunner` supports a variety of sources and sinks out of the box, including `kafka`, `kinesis`, `jdbc` , `elasticsearch 7+` (sink only), `cassandra` (sink only), `filesystems` (including `s3` using `parquet` as well as `delimited` and `json` text @@ -220,7 +229,7 @@ makes it easy to test your transformation logic with property-based testing. First, you need to define an [*algebraic data type*](http://tpolecat.github.io/presentations/algebraic_types.html#1) (ADT) containing the types that will be used in your flink jobs. Your top level type should inherit -from `FlinkEvent`. This requires your types to implement the following members: +the `FlinkEvent` trait. This requires your types to implement the following members: * `$id: String` - a unique id for the event * `$timestamp: Long` - the event time (millis since unix epoch) @@ -230,18 +239,25 @@ Additionally, a `FlinkEvent` has three additional members that you can optionall to enable/configure various features. * `$bucketId: String` - for bucketing files written to a file sink -* `$dedupId: String` - for use with Flinkrunner's deduplication filter -* `$active:Boolean` - for use with Flinkrunner's filterByControl source stream algorithm +* `$dedupId: String` - for use with FlinkRunner's deduplication filter +* `$active:Boolean` - for use with FlinkRunner's filterByControl source stream algorithm -``` +```scala sealed trait MyEventADT extends FlinkEvent -final case class MyEventA(a:Int, $id:String, $key:String, $timestamp:Long) + +final case class MyEventA(a:Int, $id:String, $key:String, $timestamp:Long) + extends MyEventADT + final case class MyEventB(b:Int, $id:String, $key:String, $timestamp:Long) + extends MyEventADT ``` Next, you should create your own runner subclass of the abstract `FlinkRunner` base class. -``` +```scala +import io.epiphanous.flinkrunner.model.FlinkConfig +import io.epiphanous.flinkrunner.FlinkRunner + class MyRunner(config:FlinkConfig) extends FlinkRunner[MyEventADT](config) { override def invoke(jobName:String):Unit = jobName matches { @@ -268,6 +284,17 @@ While this sounds useless at first blush, and your `transform` method will usual more interesting things, identity transforms like this can be useful to copy data from one storage system to another. +You could something a little more exciting, say transform a stream of `MyEventA` +events into a stream of `MyEventB` events: + +```scala +class MyJob2(runner:FlinkRunner[MyEventADT]) extends StreamJob[MyEventB](runner) { + + override def transform:DataStream[MyEventB] = + singleSource[MyEventA]().map { a:MyEventA => MyEventB(b = a.a * 2) } +} +``` + Next, you need to configure your job. The following configuration defines a file source that reads csv files from one s3 bucket and a file sink that writes json files to a different s3 bucket. @@ -277,13 +304,13 @@ jobs { MyJob1 { sources { csv-file-source { - path = "s3://my-event-a-csv-files" + path = "s3://my-events-csv-files" format = csv } } sinks { json-file-sink { - path = "s3://my-event-a-json-files" + path = "s3://my-events-json-files" format = json } } @@ -291,6 +318,8 @@ jobs { } ``` +Note that this configuration would work for either `MyJob1` or `MyJob2`. + Next, wire up your runner to a main method. ``` @@ -309,7 +338,7 @@ flink run myRunner.jar MyJob1 ## Flink Jobs -`Flinkrunner` provides a `StreamJob` base class from which you can build and run your +`FlinkRunner` provides a `StreamJob` base class from which you can build and run your flink jobs. If your output event types require avro support, you should instead use the `AvroStreamJob` base class to build and run your flink job. @@ -322,10 +351,10 @@ class StreamJob[ ``` A `StreamJob` must specify the output stream event type in its definition. That output -stream event type must be a subclass of your `Flinkrunner` algebraic data type (ADT). Your -job class will be passed an instance of your `Flinkrunner` type, from which you can access +stream event type must be a subclass of your `FlinkRunner` algebraic data type (ADT). Your +job class will be passed an instance of your `FlinkRunner` type, from which you can access -* `runner.config`: `Flinkrunner` configuration (instance of `FlinkConfig`). +* `runner.config`: `FlinkRunner` configuration (instance of `FlinkConfig`). * `runner.env`: Flink's stream execution environment (instance of `StreamExecutionEnvironment`), for interfacing with the DataStream API. * `runner.tableEnv`: Flink's stream table environment (instance of @@ -343,17 +372,17 @@ create source streams from your configuration: type within your runner's `ADT`. The configured `name` parameter defaults to the first configured source for convenience. -* `connectedSource[IN1 <: ADT, IN2 <: ADT, KEY](name1:String, name2:String, key1:IN1=>KEY, key2:IN2=>KEY): ConnectedStreams[IN1,IN2]`: - connects the two streams, configured under the provided names, producing a single stream - of type `ConnectedStreams[IN1,IN2]`. A connected stream is effectively a union of the - two types of stream. An event on the connected stream is either of type `IN1` or `IN2`. - The key functions are used for any `keyBy` operations done on the connected stream. It - is important to realize a connected stream is NOT a join between the two streams, and - the keys are not used to perform a join. The use case for a connected stream is where - the data on one stream dynamically impacts how you want to process the other stream. - -* `filterByControlSource[CONTROL <: ADT, DATA <: ADT, KEY](controlName:String, dataName:String, key1:CONTROL=>KEY, key2:DATA=>KEY): DataStream[DATA]`: - is a special instance of connected sources, where the first source is a control stream +* `connectedSource[IN1 <: ADT, IN2 <: ADT, KEY](source1:DataStream[IN1], source2:DataStream[IN2], key1:IN1=>KEY, key2:IN2=>KEY): ConnectedStreams[IN1,IN2]`: + connects the two provided source streams producing a single stream of + type `ConnectedStreams[IN1,IN2]`. A connected stream combines the two streams. An event + on the connected stream is either of type `IN1` or `IN2`. The key functions are used for + any `keyBy` operations done on the connected stream. It is important to realize a + connected stream is NOT a join between the two streams, and the keys are not used to + perform a join. The use case for a connected stream is where the data on one stream + dynamically impacts how you want to process the other stream. + +* `filterByControlSource[CONTROL <: ADT, DATA <: ADT, KEY](control:DataStream[CONTROL], data:DataStream[DATA], key1:CONTROL=>KEY, key2:DATA=>KEY): DataStream[DATA]`: + is a special instance of a connected source, where the first source is a control stream and the second source is a data stream. The control events indicate when the data event stream should be considered active, meaning any data events seen should be emitted. When the control type's `$active` @@ -365,9 +394,9 @@ create source streams from your configuration: * `broadcastConnectedSource[ IN <: ADT: TypeInformation, BC <: ADT: TypeInformation, KEY: TypeInformation]( - keyedSourceName: String, broadcastSourceName: String, keyedSourceGetKeyFunc: IN => KEY) - : BroadcastConnectedStream[IN, BC]`: another special connected source that implements - Flink's [Broadcast State Pattern](https://nightlies.apache. + keyedSource: DataStream[IN], broadcastSource: DataStream[BC], keyedSourceGetKeyFunc: + IN => KEY): BroadcastConnectedStream[IN, BC]`: another special connected source that + implements Flink's [Broadcast State Pattern](https://nightlies.apache. org/flink/flink-docs-master/docs/dev/datastream/fault-tolerance/broadcast_state/). This `StreamJob` method keys and connects a regular data stream, with a so-called broadcast data stream. A broadcast stream sends copies of all of its elements to all @@ -382,7 +411,7 @@ create source streams from your configuration: sources are Confluent schema-registry aware avro encoded Kafka streams, you should use the avro aware versions of these factory methods. For instance, the `singleAvroSource()` method can be used to produce such an input datatream. The signatures of these methods are -more complicated and rely on you to use Flinkrunner's `EmbeddedAvroRecord` and +more complicated and rely on you to use FlinkRunner's `EmbeddedAvroRecord` and `EmbeddedAvroRecordFactory` traits when implementing your event types. ``` @@ -390,8 +419,7 @@ singleAvroSource[ IN <: ADT with EmbeddedAvroRecord[INA], INA <: GenericRecord]( name: String)(implicit - fromKV: EmbeddedAvroRecordInfo[INA] => IN, - avroParquetRecordFormat: StreamFormat[INA]): DataStream[IN] + fromKV: EmbeddedAvroRecordInfo[INA] => IN): DataStream[IN] ``` Besides source factory methods, `StreamJob` also provides a method to easily perform @@ -471,19 +499,11 @@ case class EmbeddedAvroRecordInfo[A <: GenericRecord]( headers: Map[String, String] = Map.empty) ``` -#### EmbeddedAvroParquetRecordFactory - -An avro parquet writer factory for types that wrap an avro record. - -#### EmbeddedAvroParquetRecordFormat - -A StreamFormat to read parquet avro files containing a type that embeds an avro record. +## FlinkRunner Configuration -## Flinkrunner Configuration - -`Flinkrunner` uses [lightbend config](https://lightbend.github.io/config/) for its +`FlinkRunner` uses [lightbend config](https://lightbend.github.io/config/) for its configuration, integrating environment variables and command line arguments to provide -easy, environment specific, 12-factor style configuration. `Flinkrunner` makes this +easy, environment specific, 12-factor style configuration. `FlinkRunner` makes this configuration accessible through a `FlinkConfig` object that you construct and pass to your runner. @@ -499,6 +519,8 @@ trait SourceConfig[ADT <: FlinkEvent] #### Hybrid Source +#### Iceberg Source + #### Kinesis Source #### RabbitMQ Source @@ -517,6 +539,10 @@ trait SinkConfig[ADT <: FlinkEvent] #### File Sink +#### Firehose Sink + +#### Iceberg Sink + #### Jdbc Sink #### Kafka Sink diff --git a/build.sbt b/build.sbt index 4349b8fd..e2399f1f 100644 --- a/build.sbt +++ b/build.sbt @@ -29,66 +29,76 @@ resolvers += "Local Maven Repository" at "file://" + Path.userHome.absolutePath resolvers += "Confluent Repository" at "https://packages.confluent.io/maven/" val V = new { - val flink = "1.16.0" - val logback = "1.2.11" + val flink = "1.16.1" + val flinkMinor = "1.16" + val logback = "1.4.6" val scalaLogging = "3.9.5" - val scalaTest = "3.2.14" - val scalaTestPlus = "3.2.14.0" + val scalaTest = "3.2.15" + val scalaTestPlus = "3.2.15.0" val scalaCheck = "1.17.0" - val testContainersScala = "0.40.11" - val jackson = "2.14.1" + val testContainersScala = "0.40.12" + val jackson = "2.14.2" val circe = "0.14.2" val http4s = "0.23.12" - val enumeratum = "1.7.0" + val enumeratum = "1.7.2" val typesafeConfig = "1.4.2" val guava = "31.1-jre" val squants = "1.8.3" val confluentAvroSerde = "7.1.1" val parquet = "1.12.3" - val awsSdk = "1.12.351" - val jdbcMysql = "8.0.30" - val jdbcPg = "42.5.1" + val awsSdk = "1.12.429" + val jdbcMysql = "8.0.32" + val jdbcPg = "42.5.4" val jdbcMssql = "11.2.0.jre11" val hadoop = "3.3.2" val cassandraDriver = "3.11.3" val uuidCreator = "5.2.0" + val iceberg = "1.2.0" val jna = "5.12.1" // needed for testcontainers in some jvms + val awsSdk2 = "2.20.26" + val dropWizard = "4.2.17" } val flinkDeps = Seq( // scala - "org.apache.flink" %% "flink-scala" % V.flink, - "org.apache.flink" %% "flink-streaming-scala" % V.flink, + "org.apache.flink" %% "flink-scala" % V.flink, + "org.apache.flink" %% "flink-streaming-scala" % V.flink, // rocksdb - "org.apache.flink" % "flink-statebackend-rocksdb" % V.flink, + "org.apache.flink" % "flink-statebackend-rocksdb" % V.flink, + // sql parser + "org.apache.flink" % "flink-sql-parser" % V.flink, // queryable state - "org.apache.flink" % "flink-queryable-state-runtime" % V.flink % Provided, + "org.apache.flink" % "flink-queryable-state-runtime" % V.flink % Provided, // complex event processing - "org.apache.flink" % "flink-cep" % V.flink % Provided, + "org.apache.flink" % "flink-cep" % V.flink % Provided, // connectors - "org.apache.flink" % "flink-connector-base" % V.flink % Provided, // ds hybrid source - "org.apache.flink" % "flink-connector-files" % V.flink % Provided, // ds text files - "org.apache.flink" % "flink-parquet" % V.flink % Provided, // parquet bulk sink - "org.apache.flink" % "flink-connector-kafka" % V.flink % Provided, - "org.apache.flink" % "flink-connector-kinesis" % V.flink % Provided, - "org.apache.flink" %% "flink-connector-cassandra" % V.flink % Provided, - "org.apache.flink" % "flink-connector-elasticsearch7" % V.flink % Provided, - "org.apache.flink" % "flink-connector-jdbc" % V.flink % Provided, - "org.apache.flink" % "flink-connector-rabbitmq" % V.flink % Provided, + "org.apache.flink" % "flink-connector-base" % V.flink % Provided, // ds hybrid source + "org.apache.flink" % "flink-connector-files" % V.flink % Provided, // ds text files + "org.apache.flink" % "flink-parquet" % V.flink % Provided, // parquet bulk sink + "org.apache.flink" % "flink-connector-kafka" % V.flink % Provided, + "org.apache.flink" % "flink-connector-kinesis" % V.flink % Provided, + "org.apache.flink" % "flink-connector-aws-kinesis-streams" % V.flink % Provided, + "org.apache.flink" % "flink-connector-aws-kinesis-firehose" % V.flink % Provided, + "org.apache.flink" %% "flink-connector-cassandra" % V.flink % Provided, + "org.apache.flink" % "flink-connector-elasticsearch7" % V.flink % Provided, + "org.apache.flink" % "flink-connector-jdbc" % V.flink % Provided, + "org.apache.flink" % "flink-connector-rabbitmq" % V.flink % Provided, // avro support - "org.apache.flink" % "flink-avro" % V.flink % Provided, // ds and table avro format - "org.apache.flink" % "flink-avro-confluent-registry" % V.flink % Provided, // ds and table avro registry format + "org.apache.flink" % "flink-avro" % V.flink % Provided, // ds and table avro format + "org.apache.flink" % "flink-avro-confluent-registry" % V.flink % Provided, // ds and table avro registry format // table api support - "org.apache.flink" %% "flink-table-api-scala-bridge" % V.flink, // table api scala - "org.apache.flink" % "flink-table-planner-loader" % V.flink % Provided, // table api - "org.apache.flink" % "flink-table-runtime" % V.flink % Provided, // table runtime - "org.apache.flink" % "flink-csv" % V.flink % Provided, // table api csv format - "org.apache.flink" % "flink-json" % V.flink % Provided, // table api json format - "org.apache.flink" % "flink-clients" % V.flink, + "org.apache.flink" %% "flink-table-api-scala-bridge" % V.flink, // table api scala + "org.apache.flink" % "flink-table-planner-loader" % V.flink % Provided, // table api + "org.apache.flink" % "flink-table-runtime" % V.flink % Provided, // table runtime + "org.apache.flink" % "flink-csv" % V.flink % Provided, // table api csv format + "org.apache.flink" % "flink-json" % V.flink % Provided, // table api json format + "org.apache.flink" % "flink-clients" % V.flink, + // dropwizard metrics support + "org.apache.flink" % "flink-metrics-dropwizard" % V.flink % Provided, // test support - "org.apache.flink" % "flink-test-utils" % V.flink % Test, - "org.apache.flink" % "flink-runtime-web" % V.flink % Test + "org.apache.flink" % "flink-test-utils" % V.flink % Test, + "org.apache.flink" % "flink-runtime-web" % V.flink % Test ) val loggingDeps = Seq( @@ -110,32 +120,39 @@ val circeDeps = Seq( ).map(d => "io.circe" %% s"circe-$d" % V.circe) val otherDeps = Seq( - "com.github.f4b6a3" % "uuid-creator" % V.uuidCreator, - "org.apache.hadoop" % "hadoop-client" % V.hadoop % Provided, - "io.confluent" % "kafka-avro-serializer" % V.confluentAvroSerde % Provided, - "com.amazonaws" % "aws-java-sdk-core" % V.awsSdk % Provided, - "com.beachape" %% "enumeratum" % V.enumeratum, - "com.typesafe" % "config" % V.typesafeConfig, - "com.google.guava" % "guava" % V.guava, - "org.typelevel" %% "squants" % V.squants, - "org.scalactic" %% "scalactic" % V.scalaTest, - "org.scalatest" %% "scalatest" % V.scalaTest % Test, - "org.scalatestplus" %% "scalacheck-1-17" % V.scalaTestPlus % Test, - "org.scalacheck" %% "scalacheck" % V.scalaCheck, - "com.fasterxml.jackson.module" %% "jackson-module-scala" % V.jackson, - "com.github.pjfanning" %% "jackson-scala-reflect-extensions" % "2.14.0", - "com.fasterxml.jackson.dataformat" % "jackson-dataformat-csv" % V.jackson, - "com.fasterxml.jackson.datatype" % "jackson-datatype-jsr310" % V.jackson, - "com.dimafeng" %% "testcontainers-scala-scalatest" % V.testContainersScala % Test, - "com.dimafeng" %% "testcontainers-scala-mysql" % V.testContainersScala % Test, - "mysql" % "mysql-connector-java" % V.jdbcMysql % Provided, - "com.dimafeng" %% "testcontainers-scala-postgresql" % V.testContainersScala % Test, - "org.postgresql" % "postgresql" % V.jdbcPg % Provided, - "com.dimafeng" %% "testcontainers-scala-mssqlserver" % V.testContainersScala % Test, - "net.java.dev.jna" % "jna" % V.jna % Test, - "com.microsoft.sqlserver" % "mssql-jdbc" % V.jdbcMssql % Provided, - "com.dimafeng" %% "testcontainers-scala-cassandra" % V.testContainersScala % Test, - "com.datastax.cassandra" % "cassandra-driver-extras" % V.cassandraDriver % Provided + "com.amazonaws" % "aws-java-sdk-core" % V.awsSdk % Provided, + "com.amazonaws" % "aws-java-sdk-s3" % V.awsSdk % Test, + "com.beachape" %% "enumeratum" % V.enumeratum, + "com.datastax.cassandra" % "cassandra-driver-extras" % V.cassandraDriver % Provided, + "com.dimafeng" %% "testcontainers-scala-cassandra" % V.testContainersScala % Test, + "com.dimafeng" %% "testcontainers-scala-localstack-v2" % V.testContainersScala % Test, + "com.dimafeng" %% "testcontainers-scala-mssqlserver" % V.testContainersScala % Test, + "com.dimafeng" %% "testcontainers-scala-mysql" % V.testContainersScala % Test, + "com.dimafeng" %% "testcontainers-scala-postgresql" % V.testContainersScala % Test, + "com.dimafeng" %% "testcontainers-scala-scalatest" % V.testContainersScala % Test, + "com.fasterxml.jackson.dataformat" % "jackson-dataformat-csv" % V.jackson, + "com.fasterxml.jackson.datatype" % "jackson-datatype-jsr310" % V.jackson, + "com.fasterxml.jackson.module" %% "jackson-module-scala" % V.jackson, + "com.github.f4b6a3" % "uuid-creator" % V.uuidCreator, + "com.github.pjfanning" %% "jackson-scala-reflect-extensions" % "2.14.0", + "com.google.guava" % "guava" % V.guava, + "com.lihaoyi" %% "requests" % "0.8.0" % Test, + "com.microsoft.sqlserver" % "mssql-jdbc" % V.jdbcMssql % Provided, + "com.typesafe" % "config" % V.typesafeConfig, + "io.confluent" % "kafka-avro-serializer" % V.confluentAvroSerde % Provided, + "mysql" % "mysql-connector-java" % V.jdbcMysql % Provided, + "net.java.dev.jna" % "jna" % V.jna % Test, + "org.apache.hadoop" % "hadoop-client" % V.hadoop % Provided, + "org.apache.iceberg" % s"iceberg-flink-runtime-${V.flinkMinor}" % V.iceberg, + "org.postgresql" % "postgresql" % V.jdbcPg % Provided, + "org.scalacheck" %% "scalacheck" % V.scalaCheck, + "org.scalactic" %% "scalactic" % V.scalaTest, + "org.scalatest" %% "scalatest" % V.scalaTest % Test, + "org.scalatestplus" %% "scalacheck-1-17" % V.scalaTestPlus % Test, + "org.typelevel" %% "squants" % V.squants, + "software.amazon.awssdk" % "aws-sdk-java" % V.awsSdk2 % Test, + "software.amazon.awssdk" % "url-connection-client" % V.awsSdk2 % Test, + "io.dropwizard.metrics" % "metrics-core" % V.dropWizard % Provided ) ++ Seq("org.apache.parquet" % "parquet-avro" % V.parquet % Provided).map( m => diff --git a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala index b9e33702..db682a5d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/FlinkRunner.scala @@ -10,11 +10,13 @@ import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils.AvroSchemaSerializer -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.streaming.api.graph.StreamGraph import org.apache.flink.streaming.api.scala._ import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment +import org.apache.flink.table.data.RowData import scala.collection.JavaConverters._ +import scala.reflect.runtime.{universe => ru} /** FlinkRunner base class. All users of Flinkrunner will create their own * subclass. The only required parameter is a FlinkConfig object. Two @@ -36,7 +38,8 @@ import scala.collection.JavaConverters._ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( val config: FlinkConfig, val checkResultsOpt: Option[CheckResults[ADT]] = None, - val generatorFactoryOpt: Option[GeneratorFactory[ADT]] = None) + val generatorFactoryOpt: Option[GeneratorFactory[ADT]] = None, + val executeJob: Boolean = true) extends LazyLogging { val env: StreamExecutionEnvironment = @@ -56,6 +59,20 @@ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( */ def getExecutionPlan: String = env.getExecutionPlan + /** Get the stream graph for the configured job. This is primarily useful + * for testing the stream jobs constructed in flinkrunner. It will throw + * an exception if you call it before running a job against this runner. + * If you only are interested in the stream graph and don't need the job + * to be executed, you can set executeJob = false when constructing the + * FlinkRunner instance. + * @return + * JobGraph + */ + def getStreamGraph: StreamGraph = env.getStreamGraph(false) + + def getStreamNodesInfo: Seq[StreamNodeInfo] = + StreamNodeInfo.from(getStreamGraph) + /** Executes the job graph. * @return * JobExecutionResult @@ -171,6 +188,13 @@ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( sourceName: String = getDefaultSourceName): SourceConfig[ADT] = SourceConfig[ADT](sourceName, config, generatorFactoryOpt) + def _mockSource[E <: ADT: TypeInformation]( + sourceConfig: SourceConfig[ADT], + mockEvents: Seq[E]): DataStream[E] = { + val lbl = s"mock:${sourceConfig.label}" + env.fromCollection(mockEvents).name(lbl).uid(lbl) + } + /** Helper method to convert a source config into a json-encoded source * data stream. * @@ -182,25 +206,13 @@ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( * DataStream[E] */ def configToSource[E <: ADT: TypeInformation]( - sourceConfig: SourceConfig[ADT]): DataStream[E] = { + sourceConfig: SourceConfig[ADT]): DataStream[E] = checkResultsOpt .map(c => c.getInputEvents[E](sourceConfig.name)) - .getOrElse(List.empty[E]) match { - case mockEvents if mockEvents.nonEmpty => - val lbl = s"mock:${sourceConfig.label}" - env.fromCollection(mockEvents).name(lbl).uid(lbl) - case _ => - sourceConfig match { - case s: FileSourceConfig[ADT] => s.getSourceStream[E](env) - case s: KafkaSourceConfig[ADT] => s.getSourceStream[E](env) - case s: KinesisSourceConfig[ADT] => s.getSourceStream[E](env) - case s: RabbitMQSourceConfig[ADT] => s.getSourceStream[E](env) - case s: SocketSourceConfig[ADT] => s.getSourceStream[E](env) - case s: HybridSourceConfig[ADT] => s.getSourceStream[E](env) - case s: GeneratorSourceConfig[ADT] => s.getSourceStream[E](env) - } - } - } + .filter(_.nonEmpty) + .fold(sourceConfig.getSourceStream[E](env))( + _mockSource(sourceConfig, _) + ) /** Helper method to convert a source config into an avro-encoded source * data stream. At the moment this is only supported for kafka sources @@ -224,73 +236,36 @@ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( sourceConfig: SourceConfig[ADT])(implicit - fromKV: EmbeddedAvroRecordInfo[A] => E): DataStream[E] = { - + fromKV: EmbeddedAvroRecordInfo[A] => E): DataStream[E] = checkResultsOpt .map(c => c.getInputEvents[E](sourceConfig.name)) - .getOrElse(Seq.empty[E]) match { - case mockEvents if mockEvents.nonEmpty => - val lbl = s"mock:${sourceConfig.label}" - env.fromCollection(mockEvents).name(lbl).uid(lbl) - case _ => - sourceConfig match { - case s: FileSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: KafkaSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: KinesisSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: RabbitMQSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: SocketSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: HybridSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - case s: GeneratorSourceConfig[ADT] => - s.getAvroSourceStream[E, A](env) - } - } - } + .filter(_.nonEmpty) + .fold(sourceConfig.getAvroSourceStream[E, A](env))( + _mockSource(sourceConfig, _) + ) - // ********************** SINKS ********************** - - /** Create a json-encoded stream sink from configuration. + /** Helper method to convert a source configuration into a DataStream[E] * - * @param stream - * the data stream to send to sink - * @param sinkName - * the sink to send it to + * @param sourceConfig + * the source config + * @param fromRowData + * an implicit method to convert a Row into an event of type E * @tparam E - * stream element type + * the event data type * @return - * DataStream[E] */ - def toSink[E <: ADT: TypeInformation]( - stream: DataStream[E], - sinkName: String - ): DataStreamSink[E] = - configToSink[E](stream, getSinkConfig(sinkName)) + def configToRowSource[E <: ADT with EmbeddedRowType: TypeInformation]( + sourceConfig: SourceConfig[ADT])(implicit + fromRowData: RowData => E): DataStream[E] = { + checkResultsOpt + .map(c => c.getInputEvents[E](sourceConfig.name)) + .filter(_.nonEmpty) + .fold(sourceConfig.getRowSourceStream[E](env))( + _mockSource(sourceConfig, _) + ) + } - /** Create an avro-encoded stream sink from configuration. - * - * @param stream - * the data stream to send to the sink - * @param sinkName - * an optional sink name (defaults to first sink) - * @tparam E - * the event type - * @tparam A - * the avro record type - * @return - * the - */ - def toAvroSink[ - E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation]( - stream: DataStream[E], - sinkName: String - ): DataStreamSink[E] = - configToAvroSink[E, A](stream, getSinkConfig(sinkName)) + // ********************** SINKS ********************** def getSinkConfig( sinkName: String = getDefaultSinkName): SinkConfig[ADT] = @@ -304,33 +279,22 @@ abstract class FlinkRunner[ADT <: FlinkEvent: TypeInformation]( */ def writeToSink: Boolean = checkResultsOpt.forall(_.writeToSink) - def configToSink[E <: ADT: TypeInformation]( + def addSink[E <: ADT: TypeInformation]( stream: DataStream[E], - sinkConfig: SinkConfig[ADT]): DataStreamSink[E] = - sinkConfig match { - case s: CassandraSinkConfig[ADT] => s.getSink[E](stream) - case s: ElasticsearchSinkConfig[ADT] => s.getSink[E](stream) - case s: FileSinkConfig[ADT] => s.getSink[E](stream) - case s: JdbcSinkConfig[ADT] => s.getSink[E](stream) - case s: KafkaSinkConfig[ADT] => s.getSink[E](stream) - case s: KinesisSinkConfig[ADT] => s.getSink[E](stream) - case s: RabbitMQSinkConfig[ADT] => s.getSink[E](stream) - case s: SocketSinkConfig[ADT] => s.getSink[E](stream) - } + sinkName: String): Unit = + getSinkConfig(sinkName).addSink[E](stream) - def configToAvroSink[ + def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( stream: DataStream[E], - sinkConfig: SinkConfig[ADT]): DataStreamSink[E] = - sinkConfig match { - case s: CassandraSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: ElasticsearchSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: FileSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: JdbcSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: KafkaSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: KinesisSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: RabbitMQSinkConfig[ADT] => s.getAvroSink[E, A](stream) - case s: SocketSinkConfig[ADT] => s.getAvroSink[E, A](stream) - } + sinkName: String): Unit = + getSinkConfig(sinkName).addAvroSink[E, A](stream) + + def addRowSink[ + E <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag]( + stream: DataStream[E], + sinkName: String = getDefaultSinkName): Unit = + getSinkConfig(sinkName).addRowSink[E](stream) + } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/AvroStreamJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/AvroStreamJob.scala index 93994f16..b629ec3d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/AvroStreamJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/AvroStreamJob.scala @@ -25,6 +25,6 @@ abstract class AvroStreamJob[ override def sink(out: DataStream[OUT]): Unit = runner.getSinkNames.foreach(name => - runner.toAvroSink[OUT, A](out, name) + runner.addAvroSink[OUT, A](out, name) ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/StreamJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/StreamJob.scala index 1dad9b0d..eca0f33a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/flink/StreamJob.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/StreamJob.scala @@ -2,24 +2,20 @@ package io.epiphanous.flinkrunner.flink import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.model.aggregate.{ Aggregate, AggregateAccumulator, WindowedAggregationInitializer } -import io.epiphanous.flinkrunner.model.{ - EmbeddedAvroRecord, - EmbeddedAvroRecordInfo, - FlinkConfig, - FlinkEvent -} import io.epiphanous.flinkrunner.util.StreamUtils.Pipe import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.state.MapStateDescriptor -import org.apache.flink.api.common.typeinfo.{TypeHint, TypeInformation} +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala._ import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.windows.Window +import org.apache.flink.table.data.{GenericRowData, RowData} import org.apache.flink.util.Collector import squants.Quantity @@ -35,12 +31,40 @@ import squants.Quantity abstract class StreamJob[ OUT <: ADT: TypeInformation, ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) - extends LazyLogging { + extends LazyLogging + with Serializable { val config: FlinkConfig = runner.config def transform: DataStream[OUT] + def seqOrSingleSource[IN <: ADT: TypeInformation]( + seq: Seq[IN] = Seq.empty, + name: Option[String] = None): DataStream[IN] = + if (seq.nonEmpty) runner.env.fromCollection(seq) + else singleSource[IN](name.getOrElse(runner.getDefaultSourceName)) + + def seqOrSingleAvroSource[ + IN <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation]( + seq: Seq[IN] = Seq.empty, + name: Option[String] = None)(implicit + fromKV: EmbeddedAvroRecordInfo[A] => IN): DataStream[IN] = + if (seq.nonEmpty) runner.env.fromCollection[IN](seq) + else + singleAvroSource[IN, A](name.getOrElse(runner.getDefaultSourceName)) + + def seqOrSingleRowSource[ + IN <: ADT with EmbeddedRowType: TypeInformation]( + seq: Seq[IN] = Seq.empty, + name: Option[String] = None)(implicit + fromRowData: RowData => IN): DataStream[IN] = if (seq.nonEmpty) { + val rd = seq.zipWithIndex.map { case (_, i) => + GenericRowData.of(Integer.valueOf(i)) + } + runner.env.fromCollection(rd).map(g => seq(g.getInt(0))) + } else singleRowSource[IN](name.getOrElse(runner.getDefaultSourceName)) + /** Configure a single input source stream. * @param name * name of the configured source @@ -53,12 +77,54 @@ abstract class StreamJob[ name: String = runner.getDefaultSourceName): DataStream[IN] = runner.configToSource[IN](runner.getSourceConfig(name)) + /** Configure a single avro source stream. + * + * @param name + * configured name of the source + * @param fromKV + * an implicit method to construct an IN instance from an + * EmbeddedAvroRecordInfo[INA] instance. This is usually provided by + * having the companion class of your IN type extend + * EmbeddedAvroRecordFactory[INA]. + * @tparam IN + * the type of the input data stream element, which should extend + * EmbeddedAvroRecord[INA] + * @tparam INA + * the type of the avro embedded record contained in the input data + * stream element + * @return + * DataStream[IN] + */ + def singleAvroSource[ + IN <: ADT with EmbeddedAvroRecord[INA]: TypeInformation, + INA <: GenericRecord: TypeInformation]( + name: String = runner.getDefaultSourceName)(implicit + fromKV: EmbeddedAvroRecordInfo[INA] => IN): DataStream[IN] = + runner.configToAvroSource[IN, INA](runner.getSourceConfig(name)) + + /** Configure a single input row source into a stream of elements of type + * IN. + * + * @param name + * name of the configured source + * @param fromRow + * implicit method to convert a Row to event of type E + * @tparam IN + * the type of elements in the source stream + * @return + * DataStream[IN] + */ + def singleRowSource[IN <: ADT with EmbeddedRowType: TypeInformation]( + name: String = runner.getDefaultSourceName)(implicit + fromRowData: RowData => IN): DataStream[IN] = + runner.configToRowSource[IN](runner.getSourceConfig(name)) + /** Creates a connected data stream that joins two individual stream by a * joining key. - * @param source1Name - * name of the first configured source - * @param source2Name - * name of the second configured source + * @param source1 + * the first configured source + * @param source2 + * the second configured source * @param fun1 * a function to extract the joining key from elements in the first * source stream @@ -78,14 +144,11 @@ abstract class StreamJob[ IN1 <: ADT: TypeInformation, IN2 <: ADT: TypeInformation, KEY: TypeInformation]( - source1Name: String, - source2Name: String, + source1: DataStream[IN1], + source2: DataStream[IN2], fun1: IN1 => KEY, - fun2: IN2 => KEY): ConnectedStreams[IN1, IN2] = { - val source1 = singleSource[IN1](source1Name) - val source2 = singleSource[IN2](source2Name) + fun2: IN2 => KEY): ConnectedStreams[IN1, IN2] = source1.connect(source2).keyBy[KEY](fun1, fun2) - } /** A specialized connected source that combines a control stream with a * data stream. The control stream indicates when the data stream should @@ -94,10 +157,10 @@ abstract class StreamJob[ * elements are emitted. Otherwise, data elements are ignored. The * result is a stream of active data elements filtered dynamically by * the control stream. - * @param controlName - * name of the configured control stream - * @param dataName - * name of the configured data stream + * @param controlSource + * the configured control stream + * @param dataSource + * the configured data stream * @param fun1 * a function to compute a joining key from elements in the control * stream @@ -117,16 +180,19 @@ abstract class StreamJob[ CONTROL <: ADT: TypeInformation, DATA <: ADT: TypeInformation, KEY: TypeInformation]( - controlName: String, - dataName: String, + controlSource: DataStream[CONTROL], + dataSource: DataStream[DATA], fun1: CONTROL => KEY, fun2: DATA => KEY): DataStream[DATA] = { val controlLockoutDuration = - config.getDuration("control.lockout.duration").toMillis + config + .getDurationOpt("control.lockout.duration") + .map(_.toMillis) + .getOrElse(0L) connectedSource[CONTROL, DATA, KEY]( - controlName, - dataName, + controlSource, + dataSource, fun1, fun2 ).map[Either[CONTROL, DATA]]( @@ -137,7 +203,7 @@ abstract class StreamJob[ cd match { case Left(control) => ( - false, + false, // controls are never emitted if ( lastControlOpt.forall { case (_, active) => control.$active != active @@ -167,19 +233,16 @@ abstract class StreamJob[ * * https://nightlies.apache.org/flink/flink-docs-master/docs/dev/datastream/fault-tolerance/broadcast_state/. * - * @param keyedSourceName - * name of the configured keyed source - * @param broadcastSourceName - * name of the configured broadcast source - * @param keyedSourceGetKeyFunc - * a function to extract a key from elements in the keyed source - * stream + * @param keyedSource + * the configured keyed source + * @param broadcastSource + * the configured broadcast source * @tparam IN * the type of elements in the keyed source stream * @tparam BC * the type of elements in the broadcast source stream * @tparam KEY - * the type of the key + * the type of the key of the keyed source * @return * BroadcastConnectedStream[IN,BC] */ @@ -187,258 +250,17 @@ abstract class StreamJob[ IN <: ADT: TypeInformation, BC <: ADT: TypeInformation, KEY: TypeInformation]( - keyedSourceName: String, - broadcastSourceName: String, - keyedSourceGetKeyFunc: IN => KEY) - : BroadcastConnectedStream[IN, BC] = { - val keyedSource = - singleSource[IN](keyedSourceName) - .keyBy[KEY](keyedSourceGetKeyFunc) - val broadcastSource = - singleSource[BC](broadcastSourceName).broadcast( + keyedSource: DataStream[IN], + broadcastSource: DataStream[BC]): BroadcastConnectedStream[IN, BC] = + keyedSource.connect( + broadcastSource.broadcast( new MapStateDescriptor[KEY, BC]( - s"$keyedSourceName-$broadcastSourceName-state", + s"${keyedSource.name}-${broadcastSource.name}-state", createTypeInformation[KEY], createTypeInformation[BC] ) ) - keyedSource.connect(broadcastSource) - } - - /** Configure a single avro source stream. - * @param name - * configured name of the source - * @param fromKV - * an implicit method to construct an IN instance from an - * EmbeddedAvroRecordInfo[INA] instance. This is usually provided by - * having the companion class of your IN type extend - * EmbeddedAvroRecordFactory[INA]. - * @tparam IN - * the type of the input data stream element, which should extend - * EmbeddedAvroRecord[INA] - * @tparam INA - * the type of the avro embedded record contained in the input data - * stream element - * @return - * DataStream[IN] - */ - def singleAvroSource[ - IN <: ADT with EmbeddedAvroRecord[INA]: TypeInformation, - INA <: GenericRecord: TypeInformation]( - name: String = runner.getDefaultSourceName)(implicit - fromKV: EmbeddedAvroRecordInfo[INA] => IN): DataStream[IN] = - runner.configToAvroSource[IN, INA](runner.getSourceConfig(name)) - - /** Create a connected data stream joining two avro source streams by a - * joining key. - * @param source1Name - * the first configured avro source - * @param source2Name - * the second configured avro source - * @param in1GetKeyFunc - * a function to get the key from first source elements - * @param in2GetKeyFunc - * a function to the key from second source elements - * @param fromKV1 - * an implicit method to construct an IN1 instance from an - * EmbeddedAvroRecordInfo[IN1A] instance. This is usually provided by - * having the companion class of your IN1 type extend - * EmbeddedAvroRecordFactory[IN1A]. - * @param fromKV2 - * an implicit method to construct an IN1 instance from an - * EmbeddedAvroRecordInfo[IN1A] instance. This is usually provided by - * having the companion class of your IN1 type extend - * EmbeddedAvroRecordFactory[IN1A]. - * @tparam IN1 - * the type of elements in the first data stream - * @tparam IN1A - * the avro type embedded within elements of the first data stream - * @tparam IN2 - * the type of elements in the second data stream - * @tparam IN2A - * the avro type embedded within elements of the second data stream - * @tparam KEY - * the type of the joining key - * @return - * ConnectedStreams[IN1,IN2] - */ - def connectedAvroSource[ - IN1 <: ADT with EmbeddedAvroRecord[IN1A]: TypeInformation, - IN1A <: GenericRecord: TypeInformation, - IN2 <: ADT with EmbeddedAvroRecord[IN2A]: TypeInformation, - IN2A <: GenericRecord: TypeInformation, - KEY: TypeInformation]( - source1Name: String, - source2Name: String, - in1GetKeyFunc: IN1 => KEY, - in2GetKeyFunc: IN2 => KEY)(implicit - fromKV1: EmbeddedAvroRecordInfo[IN1A] => IN1, - fromKV2: EmbeddedAvroRecordInfo[IN2A] => IN2) - : ConnectedStreams[IN1, IN2] = { - val source1 = singleAvroSource[IN1, IN1A](source1Name) - val source2 = singleAvroSource[IN2, IN2A](source2Name) - source1.connect(source2).keyBy[KEY](in1GetKeyFunc, in2GetKeyFunc) - } - - /** A specialized connected avro source that combines an avro control - * stream with an avro data stream. The control stream indicates when - * the data stream should be considered active (by the control element's - * `\$active` method). When the control stream indicates the data stream - * is active, data elements are emitted. Otherwise, data elements are - * ignored. The result is a stream of active data elements filtered - * dynamically by the control stream. - * @param controlName - * name of the configured avro control stream - * @param dataName - * name of the configured avro data stream - * @param controlGetKeyFunc - * a function to compute the joining key for the control stream - * @param dataGetKeyFunc - * a function to compute the joining key for the data stream - * @param fromKVControl - * an implicit method to construct a CONTROL instance from an - * EmbeddedAvroRecordInfo[CONTROLA] instance. This is usually provided - * by having the companion class of your CONTROL type extend - * EmbeddedAvroRecordFactory[CONTROLA]. - * @param fromKVData - * an implicit method to construct a DATA instance from an - * EmbeddedAvroRecordInfo[DATAA] instance. This is usually provided by - * having the companion class of your DATA type extend - * EmbeddedAvroRecordFactory[DATAA]. - * @tparam CONTROL - * the type of elements in the control stream - * @tparam CONTROLA - * the type of avro record embedded within elements in the control - * stream - * @tparam DATA - * the type of elements in the data stream - * @tparam DATAA - * the type of avro record embedded within elements in the data stream - * @tparam KEY - * the type of joining key - * @return - * DataStream[DATA] - */ - def filterByControlAvroSource[ - CONTROL <: ADT with EmbeddedAvroRecord[CONTROLA]: TypeInformation, - CONTROLA <: GenericRecord: TypeInformation, - DATA <: ADT with EmbeddedAvroRecord[DATAA]: TypeInformation, - DATAA <: GenericRecord: TypeInformation, - KEY: TypeInformation]( - controlName: String, - dataName: String, - controlGetKeyFunc: CONTROL => KEY, - dataGetKeyFunc: DATA => KEY)(implicit - fromKVControl: EmbeddedAvroRecordInfo[CONTROLA] => CONTROL, - fromKVData: EmbeddedAvroRecordInfo[DATAA] => DATA) - : DataStream[DATA] = { - val controlLockoutDuration = - config.getDuration("control.lockout.duration").toMillis - implicit val eitherTypeInfo: TypeInformation[Either[CONTROL, DATA]] = - TypeInformation.of(new TypeHint[Either[CONTROL, DATA]] {}) - implicit val longBoolTypeInfo: TypeInformation[(Long, Boolean)] = - TypeInformation.of(new TypeHint[(Long, Boolean)] {}) - connectedAvroSource[CONTROL, CONTROLA, DATA, DATAA, KEY]( - controlName, - dataName, - controlGetKeyFunc, - dataGetKeyFunc - ).map[Either[CONTROL, DATA]]( - (c: CONTROL) => Left(c), - (d: DATA) => Right(d) - ).keyBy[KEY]((cd: Either[CONTROL, DATA]) => - cd.fold(controlGetKeyFunc, dataGetKeyFunc) - ).filterWithState[(Long, Boolean)] { case (cd, lastControlOpt) => - cd match { - case Left(control) => - ( - false, - if ( - lastControlOpt.forall { case (_, active) => - control.$active != active - } - ) Some(control.$timestamp, control.$active) - else lastControlOpt - ) - case Right(data) => - ( - lastControlOpt.exists { case (ts, active) => - active && ((data.$timestamp - ts) >= controlLockoutDuration) - }, - lastControlOpt - ) - } - }.flatMap[DATA]( - (cd: Either[CONTROL, DATA], collector: Collector[DATA]) => - cd.foreach(d => collector.collect(d)) ) - } - - /** Creates a specialized connected avro source stream that joins a keyed - * avro data stream with an avro broadcast stream. Elements in the - * broadcast stream are connected to each of the keyed streams and can - * be processed with a special CoProcessFunction, as described in - * Flink's documentation: - * - * https://nightlies.apache.org/flink/flink-docs-master/docs/dev/datastream/fault-tolerance/broadcast_state/. - * - * @param keyedSourceName - * name of the configured keyed source - * @param broadcastSourceName - * name of the configured broadcast source - * @param keyedSourceGetKeyFunc - * a function to extract the key from elements of the keyed source - * stream - * @param fromKVIN - * an implicit method to construct an IN instance from an - * EmbeddedAvroRecordInfo[INA] instance. This is usually provided by - * having the companion class of your IN type extend - * EmbeddedAvroRecordFactory[INA]. - * @param fromKVBC - * an implicit method to construct a BC instance from an - * EmbeddedAvroRecordInfo[BCA] instance. This is usually provided by - * having the companion class of your BC type extend - * EmbeddedAvroRecordFactory[BCA]. - * @tparam IN - * type type of elements in the keyed source stream - * @tparam INA - * the type of avro record embedded in elements in the keyed source - * stream - * @tparam BC - * the type of elements in the broadcast stream - * @tparam BCA - * the type of avro record embedded in elements in the broadcast - * source stream - * @tparam KEY - * the type of key in the keyed stream - * @return - * BroadcastConnectedStream[IN,BC] - */ - def broadcastConnectedAvroSource[ - IN <: ADT with EmbeddedAvroRecord[INA]: TypeInformation, - INA <: GenericRecord: TypeInformation, - BC <: ADT with EmbeddedAvroRecord[BCA]: TypeInformation, - BCA <: GenericRecord: TypeInformation, - KEY: TypeInformation]( - keyedSourceName: String, - broadcastSourceName: String, - keyedSourceGetKeyFunc: IN => KEY)(implicit - fromKVIN: EmbeddedAvroRecordInfo[INA] => IN, - fromKVBC: EmbeddedAvroRecordInfo[BCA] => BC) - : BroadcastConnectedStream[IN, BC] = { - val keyedSource = - singleAvroSource[IN, INA](keyedSourceName) - .keyBy[KEY](keyedSourceGetKeyFunc) - val broadcastSource = - singleAvroSource[BC, BCA](broadcastSourceName).broadcast( - new MapStateDescriptor[KEY, BC]( - s"$keyedSourceName-$broadcastSourceName-state", - createTypeInformation[KEY], - createTypeInformation[BC] - ) - ) - keyedSource.connect(broadcastSource) - } /** Create a data stream of windowed aggregates of type PWF_OUT. This * output stream will be instances of the ADT to be written to a sink. @@ -498,7 +320,7 @@ abstract class StreamJob[ * a transformed stream from transform() */ def sink(out: DataStream[OUT]): Unit = - runner.getSinkNames.foreach(name => runner.toSink[OUT](out, name)) + runner.getSinkNames.foreach(name => runner.addSink[OUT](out, name)) /** The output stream will only be passed to output sink(s) if the runner * determines it's required. Some testing configurations can skip @@ -511,6 +333,8 @@ abstract class StreamJob[ def maybeSink(out: DataStream[OUT]): Unit = if (runner.writeToSink) sink(out) + def buildJobGraph: DataStream[OUT] = transform |# maybeSink + /** Runs the job, meaning it constructs the flow and executes it. */ def run(): Unit = { @@ -520,27 +344,31 @@ abstract class StreamJob[ ) // build the job graph - val stream = transform |# maybeSink + val stream = buildJobGraph if (config.showPlan) logger.info(s"\nPLAN:\n${runner.getExecutionPlan}\n") - runner.checkResultsOpt match { + if (runner.executeJob) { + runner.checkResultsOpt match { - case Some(checkResults) => - logger.info( - s"routing job ${config.jobName} results back through CheckResults<${checkResults.name}>" - ) - checkResults.checkOutputEvents[OUT]( - stream.executeAndCollect( - config.jobName, - checkResults.collectLimit + case Some(checkResults) => + logger.info( + s"routing job ${config.jobName} results back through CheckResults<${checkResults.name}>.checkOutputEvents" + ) + checkResults.checkOutputEvents[OUT]( + stream.executeAndCollect( + config.jobName, + checkResults.collectLimit + ) ) - ) - case _ => - val result = runner.execute - logger.info(result.toString) + case _ => + val result = runner.execute + logger.info(result.toString) + } + } else { + logger.info("NOT EXECUTING JOB GRAPH") } } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/flink/TableStreamJob.scala b/src/main/scala/io/epiphanous/flinkrunner/flink/TableStreamJob.scala new file mode 100644 index 00000000..7955942b --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/flink/TableStreamJob.scala @@ -0,0 +1,25 @@ +package io.epiphanous.flinkrunner.flink + +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.model.{EmbeddedRowType, FlinkEvent} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream + +import scala.reflect.runtime.{universe => ru} + +/** A job class to generate streaming output tables. + * @param runner + * an instance of [[FlinkRunner]] + * @tparam OUT + * the output type + * @tparam ADT + * the algebraic data type of the [[FlinkRunner]] instance + */ +abstract class TableStreamJob[ + OUT <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + ADT <: FlinkEvent: TypeInformation](runner: FlinkRunner[ADT]) + extends StreamJob[OUT, ADT](runner) { + + override def sink(out: DataStream[OUT]): Unit = + runner.getSinkNames.foreach(name => runner.addRowSink[OUT](out, name)) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/BasicJdbcConnectionProvider.scala b/src/main/scala/io/epiphanous/flinkrunner/model/BasicJdbcConnectionProvider.scala index 1b5c9bf7..dfca74a4 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/BasicJdbcConnectionProvider.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/BasicJdbcConnectionProvider.scala @@ -28,12 +28,14 @@ class BasicJdbcConnectionProvider( val url: String = jdbcOptions.getDbURL - jdbcOptions.getUsername.ifPresent(user => + jdbcOptions.getUsername.ifPresent { user => props.setProperty("user", user) - ) - jdbcOptions.getPassword.ifPresent(pwd => + () + } + jdbcOptions.getPassword.ifPresent { pwd => props.setProperty("password", pwd) - ) + () + } @transient var connection: Connection = _ diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/CassandraClusterBuilder.scala b/src/main/scala/io/epiphanous/flinkrunner/model/CassandraClusterBuilder.scala new file mode 100644 index 00000000..f2296154 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/CassandraClusterBuilder.scala @@ -0,0 +1,21 @@ +package io.epiphanous.flinkrunner.model + +import com.datastax.driver.core.{Cluster, CodecRegistry} +import com.datastax.driver.extras.codecs.jdk8.InstantCodec +import org.apache.flink.streaming.connectors.cassandra.ClusterBuilder + +@SerialVersionUID(938953971942797516L) +class CassandraClusterBuilder(host: String, port: Int) + extends ClusterBuilder { + // not really a bug --| + // V + override def buildCluster(builder: Cluster.Builder): Cluster = + builder + .addContactPoint(host) + .withPort(port) + .withoutJMXReporting() + .withCodecRegistry( + new CodecRegistry().register(InstantCodec.instance) + ) + .build() +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/D64.scala b/src/main/scala/io/epiphanous/flinkrunner/model/D64.scala index a5e673c5..bb853452 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/D64.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/D64.scala @@ -26,7 +26,7 @@ object D64 { val len = data.length var hang = 0 data.zipWithIndex.foreach { case (v, i) => - val v2 = if (v < 0) v + 0x100 else v + val v2 = if (v < 0) v.toInt + 0x100 else v.toInt (i % 3: @switch) match { case 0 => sb += chars(v2 >> 2) diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/DataTypeConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/DataTypeConfig.scala new file mode 100644 index 00000000..e07f449d --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/DataTypeConfig.scala @@ -0,0 +1,9 @@ +package io.epiphanous.flinkrunner.model + +case class DataTypeConfig( + value: Option[String] = None, + bridgedTo: Option[Class[_]] = None, + defaultDecimalPrecision: Option[Int] = None, + defaultDecimalScale: Option[Int] = None, + defaultSecondPrecision: Option[Int] = None, + defaultYearPrecision: Option[Int] = None) diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecord.scala b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecord.scala index a889b4ad..2d94bb0d 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecord.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecord.scala @@ -1,6 +1,7 @@ package io.epiphanous.flinkrunner.model import org.apache.avro.generic.GenericRecord +import org.apache.flink.types.Row /** Event types that wrap avro records should implement this trait. This * trait works with other avro related features in Flinkrunner, such as @@ -18,7 +19,8 @@ import org.apache.avro.generic.GenericRecord * @tparam A * An avro record type */ -trait EmbeddedAvroRecord[A <: GenericRecord] { +trait EmbeddedAvroRecord[A <: GenericRecord] extends EmbeddedRowType { + this: FlinkEvent => /** An optional embedded record key - if present, used as the key when * stored in kafka. Defaults to None. @@ -47,4 +49,13 @@ trait EmbeddedAvroRecord[A <: GenericRecord] { */ def toKV(config: FlinkConfig): EmbeddedAvroRecordInfo[A] = EmbeddedAvroRecordInfo($record, config, $recordKey, $recordHeaders) + + override def toRow: Row = { + val arity = $record.getSchema.getFields.size() + (0 until arity) + .foldLeft(Row.withPositions($rowKind, arity)) { case (row, pos) => + row.setField(pos, $record.get(pos)) + row + } + } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecordFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecordFactory.scala index 1a81fd9d..b1d54dd3 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecordFactory.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedAvroRecordFactory.scala @@ -2,6 +2,8 @@ package io.epiphanous.flinkrunner.model import org.apache.avro.generic.GenericRecord +import scala.language.implicitConversions + /** Companion objects of event types that wrap avro records should * implement this trait to support avro deserialization. A companion * trait, EmbeddedAvroRecord, can be used to support serializing avro @@ -23,4 +25,5 @@ trait EmbeddedAvroRecordFactory[ * New event of type E */ implicit def fromKV(recordInfo: EmbeddedAvroRecordInfo[A]): E + } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowType.scala b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowType.scala new file mode 100644 index 00000000..14317486 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowType.scala @@ -0,0 +1,38 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.avro.reflect.AvroIgnore +import org.apache.flink.types.{Row, RowKind} + +import scala.annotation.tailrec + +/** Event types that are used in TableStreamJobs should implement this + * trait. This trait endows [[FlinkEvent]] types with a `\$toRow` method, + * to convert the underlying event type to an instance of a flink [[Row]] + * object. + */ +trait EmbeddedRowType { this: FlinkEvent => + @AvroIgnore def toRow: Row = { + productIterator.zipWithIndex.foldLeft( + Row.withPositions($rowKind, productArity) + ) { case (row, (value, pos)) => + @tailrec + def set[T](a: T): Unit = a match { + case o: Option[_] => set(o.getOrElse(null)) + case null => row.setField(pos, null) + case d: Double => row.setField(pos, Double.box(d)) + case f: Float => row.setField(pos, Float.box(f)) + case i: Int => row.setField(pos, Int.box(i)) + case l: Long => row.setField(pos, Long.box(l)) + case c: Char => row.setField(pos, Char.box(c)) + case s: Short => row.setField(pos, Short.box(s)) + case b: Byte => row.setField(pos, Byte.box(b)) + case b: Boolean => row.setField(pos, Boolean.box(b)) + case _ => row.setField(pos, a) + } + set(value) + row + } + } + + @AvroIgnore def $rowKind: RowKind = RowKind.INSERT +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowTypeFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowTypeFactory.scala new file mode 100644 index 00000000..7a09b1e1 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/EmbeddedRowTypeFactory.scala @@ -0,0 +1,21 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.flink.table.data.RowData + +/** Companion objects of event types that implement EmbeddedRowType and + * require Row-based sources should implement this trait to support row + * deserialization. + * @tparam E + * a flink event type that implements EmbeddedRowType + */ +trait EmbeddedRowTypeFactory[E <: FlinkEvent with EmbeddedRowType] { + + /** Construct an event of type E from row data. + * + * @param rowData + * row data to convert to event of type E + * @return + * New event of type E + */ + implicit def fromRowData(rowData: RowData): E +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala index e4ef7d95..b3392063 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConfig.scala @@ -74,7 +74,8 @@ class FlinkConfig(args: Array[String], optConfig: Option[String] = None) if (jobParams.has(jpath)) ("a", jpath) else if (_config.hasPath(jpath)) ("c", jpath) else if (jobParams.has(path)) ("a", path) - else ("c", path) + else if (_config.hasPath(path)) ("c", path) + else ("n", path) } private def _j(path: String) = s"jobs.$jobName.$path" diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala index 855d9539..e0e64897 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/FlinkConnectorName.scala @@ -10,10 +10,13 @@ sealed trait FlinkConnectorName extends EnumEntry with Snakecase object FlinkConnectorName extends Enum[FlinkConnectorName] { val values: immutable.IndexedSeq[FlinkConnectorName] = findValues + case object Empty extends FlinkConnectorName case object Hybrid extends FlinkConnectorName case object Kinesis extends FlinkConnectorName + case object Firehose extends FlinkConnectorName + case object Kafka extends FlinkConnectorName case object File extends FlinkConnectorName @@ -30,8 +33,12 @@ object FlinkConnectorName extends Enum[FlinkConnectorName] { case object Generator extends FlinkConnectorName + case object Iceberg extends FlinkConnectorName + + case object Print extends FlinkConnectorName + val sources: immutable.Seq[FlinkConnectorName] = - values diff IndexedSeq(Cassandra, Elasticsearch) + values diff IndexedSeq(Cassandra, Elasticsearch, Firehose, Print) val sinks: immutable.Seq[FlinkConnectorName] = values diff IndexedSeq(Hybrid, Generator) @@ -60,10 +67,11 @@ object FlinkConnectorName extends Enum[FlinkConnectorName] { val connector = (connectorNameOpt match { case Some(connectorName) => withNameInsensitiveOption(connectorName) case None => - val lcName = sourceOrSinkName.toLowerCase - val lcNameSuffixed = s"${lcName}_$sourceOrSink" + val lcName = sourceOrSinkName.toLowerCase.replaceAll("-", "_") + val lcNameSuffixed = s"${lcName}_$sourceOrSink" + val lcNameUnsuffixed = lcName.replace(s"_$sourceOrSink", "") values.find { c => - Seq(lcName, lcNameSuffixed).exists( + Seq(lcName, lcNameSuffixed, lcNameUnsuffixed).exists( _.contains(c.entryName.toLowerCase) ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/IcebergCommonConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/IcebergCommonConfig.scala new file mode 100644 index 00000000..886018bd --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/IcebergCommonConfig.scala @@ -0,0 +1,78 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.hadoop.conf.Configuration +import org.apache.iceberg.catalog.{Namespace, TableIdentifier} +import org.apache.iceberg.flink.CatalogLoader + +import java.util +import scala.collection.JavaConverters._ + +case class IcebergCommonConfig( + tableIdentifier: TableIdentifier, + catalogLoader: CatalogLoader) + +object IcebergCommonConfig { + final val NESSIE_IMPL = "org.apache.iceberg.nessie.NessieCatalog" + final val ICEBERG_IMPL = "org.apache.iceberg.rest.RESTCatalog" + + def apply[ADT <: FlinkEvent]( + icebergSourceConfig: SourceOrSinkConfig[ADT]) + : IcebergCommonConfig = { + val config = icebergSourceConfig.config + val pfx = s => icebergSourceConfig.pfx(s) + val hadoopConf = new Configuration() + + val namespace: Namespace = + Namespace.of( + config + .getStringOpt(pfx("namespace")) + .getOrElse("default") + .split("\\."): _* + ) + + val tableName: String = config.getString(pfx("table")) + + val tableIdentifier: TableIdentifier = + TableIdentifier.of(namespace, tableName) + + val ( + catalogName: String, + catalogType: String, + catalogProperties: util.Map[String, String] + ) = + ( + config + .getStringOpt(pfx("catalog.name")) + .getOrElse("default"), + config.getStringOpt(pfx("catalog.type")).getOrElse("iceberg"), + config + .getProperties(pfx("catalog")) + .asScala + .filterKeys(k => !Seq("name", "type").contains(k)) + .foldLeft(Map.empty[String, String]) { case (m, kv) => m + kv } + .asJava + ) + + val catalogLoader: CatalogLoader = catalogType.toLowerCase match { + case "hive" => + catalogProperties.put("type", "hive") + CatalogLoader.hive(catalogName, hadoopConf, catalogProperties) + case "hadoop" => + catalogProperties.put("type", "hadoop") + CatalogLoader.hadoop(catalogName, hadoopConf, catalogProperties) + case impl => + CatalogLoader.custom( + catalogName, + catalogProperties, + hadoopConf, + impl match { + case "iceberg" => ICEBERG_IMPL + case "nessie" => NESSIE_IMPL + case _ => catalogType + } + ) + } + + IcebergCommonConfig(tableIdentifier, catalogLoader) + } +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/IcebergPartitionColumn.scala b/src/main/scala/io/epiphanous/flinkrunner/model/IcebergPartitionColumn.scala new file mode 100644 index 00000000..e3dff23a --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/IcebergPartitionColumn.scala @@ -0,0 +1,38 @@ +package io.epiphanous.flinkrunner.model + +import com.typesafe.config.Config +import io.epiphanous.flinkrunner.model.IcebergPartitionColumn.{ + BUCKET_PATTERN, + TRUNCATE_PATTERN +} +import org.apache.iceberg.PartitionSpec + +import scala.util.matching.Regex + +case class IcebergPartitionColumn(name: String, transform: String) { + def addToSpec(builder: PartitionSpec.Builder): PartitionSpec.Builder = { + transform.toLowerCase match { + case "identity" => builder.identity(name) + case "year" => builder.year(name) + case "month" => builder.month(name) + case "day" => builder.day(name) + case "hour" => builder.hour(name) + case TRUNCATE_PATTERN(w) => builder.truncate(name, w.toInt) + case BUCKET_PATTERN(n) => builder.bucket(name, n.toInt) + case _ => + throw new RuntimeException( + s"invalid iceberg partition tranform '$transform'" + ) + } + } +} + +object IcebergPartitionColumn { + val TRUNCATE_PATTERN: Regex = "truncate\\[(\\d+)]".r + val BUCKET_PATTERN: Regex = "bucket\\[(\\d+)]".r + def apply(config: Config): IcebergPartitionColumn = + IcebergPartitionColumn( + config.getString("column"), + config.getString("transform") + ) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/Id64.scala b/src/main/scala/io/epiphanous/flinkrunner/model/Id64.scala index 28271aed..0fa6057c 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/Id64.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/Id64.scala @@ -66,11 +66,11 @@ object Id64 { def ticksOf(id: String): Long = uuidOf(id).timestamp() def microsOf(id: String): Long = Math - .floor((ticksOf(id) - GREGORIAN_OFFSET) / 10) + .floor((ticksOf(id) - GREGORIAN_OFFSET).toDouble / 10.0) .toLong def millisOf(id: String): Long = Math - .floor((ticksOf(id) - GREGORIAN_OFFSET) / 10000) + .floor((ticksOf(id) - GREGORIAN_OFFSET).toDouble / 10000.0) .toLong def instantOf(id: String): Instant = { val t = ticksOf(id) - GREGORIAN_OFFSET diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkAvroStatementBuilder.scala b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkAvroStatementBuilder.scala new file mode 100644 index 00000000..75f1194f --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkAvroStatementBuilder.scala @@ -0,0 +1,20 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.typeinfo.TypeInformation + +import java.sql.PreparedStatement + +class JdbcSinkAvroStatementBuilder[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation, + ADT <: FlinkEvent](columns: Seq[JdbcSinkColumn]) + extends JdbcSinkStatementBuilder[E, ADT](columns) { + override def accept(statement: PreparedStatement, event: E): Unit = { + _fillInStatement( + fieldValuesOf(event.$record.asInstanceOf[Product]), + statement, + event + ) + } +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementBuilder.scala b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementBuilder.scala new file mode 100644 index 00000000..05c373fe --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementBuilder.scala @@ -0,0 +1,69 @@ +package io.epiphanous.flinkrunner.model + +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.serde.JsonRowEncoder +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.connector.jdbc.JdbcStatementBuilder + +import java.sql.{PreparedStatement, Timestamp} +import java.time.Instant + +class JdbcSinkStatementBuilder[E <: ADT, ADT <: FlinkEvent]( + columns: Seq[JdbcSinkColumn]) + extends JdbcStatementBuilder[E] + with LazyLogging { + + override def accept(statement: PreparedStatement, event: E): Unit = { + _fillInStatement( + fieldValuesOf(event), + statement, + event + ) + } + + def fieldValuesOf[T <: Product](product: T): Map[String, Any] = { + product.getClass.getDeclaredFields + .map(_.getName) + .zip(product.productIterator.toIndexedSeq) + .toMap + } + + def _matcher(value: Any): Any = { + lazy val encoder = new JsonRowEncoder[Map[String, Any]]() + value match { + case ts: Instant => Timestamp.from(ts) + case m: Map[String, Any] => + try + encoder.encode(m).get + catch { + case t: Throwable => + logger.error(s"Failed to json encode map: $m\n${t.getMessage}") + null + } + case _ => value + } + } + + def _fillInStatement( + data: Map[String, Any], + statement: PreparedStatement, + element: E): Unit = { + columns.zipWithIndex.map(x => (x._1, x._2 + 1)).foreach { + case (column, i) => + data.get(column.name) match { + case Some(v) => + val value = v match { + case null | None => null + case Some(x) => _matcher(x) + case x => _matcher(x) + } + statement.setObject(i, value, column.dataType.jdbcType) + case None => + throw new RuntimeException( + s"value for field ${column.name} is not in $element" + ) + } + } + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementExecutorFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementExecutorFactory.scala new file mode 100644 index 00000000..de9bfe1d --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/JdbcSinkStatementExecutorFactory.scala @@ -0,0 +1,20 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.connector.jdbc.JdbcStatementBuilder +import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat.StatementExecutorFactory +import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor + +import java.util.function.Function + +class JdbcSinkStatementExecutorFactory[E <: ADT, ADT <: FlinkEvent]( + queryDml: String, + statementBuilder: JdbcStatementBuilder[E]) + extends StatementExecutorFactory[JdbcBatchStatementExecutor[E]] { + override def apply(rtc: RuntimeContext): JdbcBatchStatementExecutor[E] = + JdbcBatchStatementExecutor.simple( + queryDml, + statementBuilder, + Function.identity[E] + ) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/KinesisProperties.scala b/src/main/scala/io/epiphanous/flinkrunner/model/KinesisProperties.scala new file mode 100644 index 00000000..21d63a72 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/KinesisProperties.scala @@ -0,0 +1,158 @@ +package io.epiphanous.flinkrunner.model + +import com.amazonaws.regions.Regions +import io.epiphanous.flinkrunner.model.sink.SinkConfig +import io.epiphanous.flinkrunner.util.ConfigToProps.{ + getFromEither, + RichConfigObject +} +import org.apache.flink.connector.aws.config.AWSConfigConstants + +import java.util.Properties + +case class KinesisProperties( + stream: String, + clientProperties: Properties, + failOnError: Boolean, + maxBatchSizeInNumber: Int, + maxBatchSizeInBytes: Long, + maxBufferedRequests: Int, + maxBufferTime: Long, + maxInFlightRequests: Int, + maxRecordSizeInBytes: Option[Long]) + +object KinesisProperties { + + final val DEFAULT_REGION = Regions.US_EAST_1.getName + final val DEFAULT_FAIL_ON_ERROR = false + final val DEFAULT_MAX_BATCH_SIZE_IN_NUMBER = 500 + final val DEFAULT_MAX_BATCH_SIZE_IN_BYTES = 4 * 1024 * 1024 + final val DEFAULT_MAX_BUFFERED_REQUESTS = 10000 + final val DEFAULT_MAX_BUFFER_TIME = 5000 + final val DEFAULT_MAX_IN_FLIGHT_REQUESTS = 50 + + def fromSinkConfig[SC <: SinkConfig[_]]( + sinkConfig: SC): KinesisProperties = { + val config = sinkConfig.config + val pfx = sinkConfig.pfx() + + val awsRegion: Option[String] = getFromEither( + pfx, + Seq("aws.region", "region", AWSConfigConstants.AWS_REGION), + config.getStringOpt + ) + + val awsEndpoint: Option[String] = getFromEither( + pfx, + Seq("aws.endpoint", "endpoint", AWSConfigConstants.AWS_ENDPOINT), + config.getStringOpt + ) + + val clientProperties: Properties = + getFromEither( + pfx, + Seq("client"), + config.getObjectOption + ).asProperties + + clientProperties.putIfAbsent( + AWSConfigConstants.AWS_REGION, + DEFAULT_REGION + ) + + awsEndpoint.foreach(endpoint => + clientProperties.putIfAbsent( + AWSConfigConstants.AWS_ENDPOINT, + endpoint + ) + ) + + val stream: String = getFromEither( + pfx, + Seq( + "stream", + "stream.name", + "delivery.stream", + "delivery.stream.name" + ), + config.getStringOpt + ).getOrElse( + throw new RuntimeException( + s"kinesis stream name required but missing in sink <${sinkConfig.name}> of job <${config.jobName}>" + ) + ) + + val failOnError: Boolean = getFromEither( + pfx, + Seq("failOnError", "fail.on.error"), + config.getBooleanOpt + ).getOrElse(DEFAULT_FAIL_ON_ERROR) + + val maxInFlightRequests: Int = getFromEither( + pfx, + Seq("maxInFlightRequests", "max.in.flight.requests"), + config.getIntOpt + ).getOrElse(DEFAULT_MAX_IN_FLIGHT_REQUESTS) + + val maxBufferedRequests: Int = + getFromEither( + pfx, + Seq("maxBufferedRequests", "max.buffered.requests"), + config.getIntOpt + ).getOrElse(DEFAULT_MAX_BUFFERED_REQUESTS) + + val maxBatchSizeInNumber: Int = + getFromEither( + pfx, + Seq( + "maxBatchSizeInNumber", + "max.batch.size.in.number", + "max.batch.size.number" + ), + config.getIntOpt + ).getOrElse(DEFAULT_MAX_BATCH_SIZE_IN_NUMBER) + + val maxBatchSizeInBytes: Long = + getFromEither( + pfx, + Seq( + "maxBatchSizeInBytes", + "max.batch.size.in.bytes", + "max.batch.size.bytes" + ), + config.getLongOpt + ).getOrElse(DEFAULT_MAX_BATCH_SIZE_IN_BYTES) + + val maxBufferTime: Long = getFromEither( + pfx, + Seq("maxBufferTime", "max.buffer.time"), + config.getDurationOpt + ) + .map(_.toMillis) + .getOrElse(DEFAULT_MAX_BUFFER_TIME) + + val maxRecordSizeInBytes: Option[Long] = getFromEither( + pfx, + Seq( + "maxRecordSizeInBytes", + "maxRecordSize", + "max.record.size", + "max.record.size.in.bytes" + ), + config.getLongOpt + ) + + KinesisProperties( + stream, + clientProperties, + failOnError, + maxBatchSizeInNumber, + maxBatchSizeInBytes, + maxBufferedRequests, + maxBufferTime, + maxInFlightRequests, + maxRecordSizeInBytes + ) + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SchemaRegistryConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SchemaRegistryConfig.scala index bd53b34f..6da5d77e 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/SchemaRegistryConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SchemaRegistryConfig.scala @@ -1,6 +1,7 @@ package io.epiphanous.flinkrunner.model import com.typesafe.config.ConfigObject +import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig import io.epiphanous.flinkrunner.util.ConfigToProps.RichConfigObject import io.epiphanous.flinkrunner.util.StreamUtils.RichProps @@ -15,8 +16,10 @@ case class SchemaRegistryConfig( props: util.HashMap[String, String] = new util.HashMap()) { val isSerializing: Boolean = !isDeserializing props.put("schema.registry.url", url) - props.put("specific.avro.reader", "false") // don't make this true! - props.putIfAbsent("use.logical.type.converters", "true") + props.putIfAbsent( + KafkaAvroDeserializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG, + "true" + ) } object SchemaRegistryConfig { def apply( diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/SourceOrSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/SourceOrSinkConfig.scala new file mode 100644 index 00000000..8dcdba27 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/SourceOrSinkConfig.scala @@ -0,0 +1,55 @@ +package io.epiphanous.flinkrunner.model + +import com.google.common.hash.Hashing +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.util.StreamUtils.RichProps + +import java.nio.charset.StandardCharsets +import java.util +import java.util.Properties + +trait SourceOrSinkConfig[ADT <: FlinkEvent] extends LazyLogging { + def name: String + + def config: FlinkConfig + + def connector: FlinkConnectorName + + def _sourceOrSink: String + + def _sourceOrSinkPath: String = _sourceOrSink + "s" + + def pfx(path: String = ""): String = Seq( + Some(_sourceOrSinkPath), + Some(name), + if (path.isEmpty) None else Some(path) + ).flatten.mkString(".") + + val properties: Properties = config.getProperties(pfx("config")) + + lazy val propertiesMap: util.HashMap[String, String] = + properties.asJavaMap + + lazy val label: String = + s"${config.jobName.toLowerCase}/${connector.entryName.toLowerCase}/$name" + + lazy val stdUid: String = Hashing + .sha256() + .hashString( + label, + StandardCharsets.UTF_8 + ) + .toString + + lazy val uid: String = config.getStringOpt(pfx("uid")).getOrElse(stdUid) + + lazy val parallelism: Int = config + .getIntOpt(pfx("parallelism")) + .getOrElse(config.globalParallelism) + + def notImplementedError(method: String): Unit = + throw new RuntimeException( + s"$method is not implemented for ${connector.entryName} ${_sourceOrSink} $name" + ) + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/StreamNodeInfo.scala b/src/main/scala/io/epiphanous/flinkrunner/model/StreamNodeInfo.scala new file mode 100644 index 00000000..0e362203 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/StreamNodeInfo.scala @@ -0,0 +1,44 @@ +package io.epiphanous.flinkrunner.model + +import org.apache.flink.streaming.api.graph.StreamGraph + +import scala.collection.JavaConverters._ + +case class StreamNodeInfo( + id: Int, + name: String, + parallelism: Int, + inClasses: List[String], + outClass: Option[String]) { + val isSource: Boolean = inClasses.isEmpty + val isSink: Boolean = outClass.isEmpty + val isTransform: Boolean = !isSource && !isSink + val nodeKind: String = + if (isSource) "source" else if (isTransform) "transform" else "sink" + val simpleInClasses: List[String] = + inClasses.map(_.split("\\.").last) + val simpleOutClass: Option[String] = outClass.map(_.split("\\.").last) +} + +object StreamNodeInfo { + def from(sg: StreamGraph): Seq[StreamNodeInfo] = { + sg.getStreamNodes.asScala.map { sn => + val id = sn.getId + val name = sn.getOperatorName + val parallelism = sn.getParallelism + val inClasses = sn.getTypeSerializersIn.toList.map( + _.createInstance().getClass.getCanonicalName + ) + val outClass = + Option(sn.getTypeSerializerOut) + .map(_.createInstance().getClass.getCanonicalName) + StreamNodeInfo( + id = id, + name = name, + parallelism = parallelism, + inClasses = inClasses, + outClass = outClass + ) + }.toSeq + } +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/CassandraSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/CassandraSinkConfig.scala index 455e9873..8bfde316 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/CassandraSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/CassandraSinkConfig.scala @@ -1,14 +1,13 @@ package io.epiphanous.flinkrunner.model.sink -import com.datastax.driver.core.{Cluster, CodecRegistry} -import com.datastax.driver.extras.codecs.jdk8.InstantCodec -import io.epiphanous.flinkrunner.model.{EmbeddedAvroRecord, FlinkConfig, FlinkConnectorName, FlinkEvent} +import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.util.AvroUtils.RichGenericRecord import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.datastream.DataStreamSink import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.connectors.cassandra._ +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.types.Row /** A cassandra sink config. * @@ -37,33 +36,18 @@ case class CassandraSinkConfig[ADT <: FlinkEvent]( val port: Int = config.getIntOpt(pfx("port")).getOrElse(9042) val query: String = config.getString(pfx("query")) - /** Don't convert to single abstract method...flink will complain - */ - val clusterBuilder: ClusterBuilder = new ClusterBuilder { - override def buildCluster(builder: Cluster.Builder): Cluster = - builder - .addContactPoint(host) - .withPort(port) - .withoutJMXReporting() - .withCodecRegistry( - new CodecRegistry().register(InstantCodec.instance) - ) - .build() - } + val clusterBuilder = new CassandraClusterBuilder(host, port) - def getSink[E <: ADT: TypeInformation]( - stream: DataStream[E]): DataStreamSink[E] = { - stream - .addSink(new CassandraScalaProductSink[E](query, clusterBuilder)) - .uid(label) - .name(label) - .setParallelism(parallelism) - } + def addSink[E <: ADT: TypeInformation](stream: DataStream[E]): Unit = + CassandraSink + .addSink(stream) + .setClusterBuilder(clusterBuilder) + .setQuery(query) + .build() - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation]( - stream: DataStream[E]): DataStreamSink[E] = + A <: GenericRecord: TypeInformation](stream: DataStream[E]): Unit = { stream .addSink( new AbstractCassandraTupleSink[E]( @@ -78,6 +62,16 @@ case class CassandraSinkConfig[ADT <: FlinkEvent]( ) .uid(label) .name(label) - .setParallelism(parallelism) + } + override def _addRowSink( + stream: DataStream[Row], + rowType: RowType): Unit = { + CassandraSink + .addSink(stream) + .setClusterBuilder(clusterBuilder) + .setQuery(query) + .build() + () + } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/ElasticsearchSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/ElasticsearchSinkConfig.scala index ef222ffe..86652ee8 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/ElasticsearchSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/ElasticsearchSinkConfig.scala @@ -1,14 +1,21 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{EmbeddedAvroRecord, FlinkConfig, FlinkConnectorName, FlinkEvent} +import io.epiphanous.flinkrunner.model.{ + EmbeddedAvroRecord, + FlinkConfig, + FlinkConnectorName, + FlinkEvent +} import io.epiphanous.flinkrunner.util.AvroUtils.RichGenericRecord import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.connector.sink2.SinkWriter import org.apache.flink.connector.elasticsearch.sink -import org.apache.flink.connector.elasticsearch.sink.{Elasticsearch7SinkBuilder, ElasticsearchEmitter, FlushBackoffType} -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.connector.elasticsearch.sink.{ + Elasticsearch7SinkBuilder, + ElasticsearchEmitter, + FlushBackoffType +} import org.apache.flink.streaming.api.scala.DataStream import org.apache.http.HttpHost import org.elasticsearch.client.Requests @@ -39,8 +46,7 @@ import scala.collection.JavaConverters._ case class ElasticsearchSinkConfig[ADT <: FlinkEvent]( name: String, config: FlinkConfig -) extends SinkConfig[ADT] - with LazyLogging { +) extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.Elasticsearch @@ -70,9 +76,9 @@ case class ElasticsearchSinkConfig[ADT <: FlinkEvent]( val bulkFlushIntervalMs: Option[Long] = Option(properties.getProperty("bulk.flush.interval.ms")).map(_.toLong) - def _getSink[E <: ADT: TypeInformation]( + def _addSink[E <: ADT: TypeInformation]( dataStream: DataStream[E], - emitter: ElasticsearchEmitter[E]): DataStreamSink[E] = { + emitter: ElasticsearchEmitter[E]): Unit = { val esb = new Elasticsearch7SinkBuilder[E] .setHosts(transports: _*) @@ -92,15 +98,15 @@ case class ElasticsearchSinkConfig[ADT <: FlinkEvent]( .setParallelism(parallelism) } - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink(dataStream, getEmitter[E]) + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink(dataStream, getEmitter[E]) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink(dataStream, getAvroEmitter[E, A]) + dataStream: DataStream[E]): Unit = + _addSink(dataStream, getAvroEmitter[E, A]) def _getEmitter[E <: ADT]( getData: E => AnyRef): ElasticsearchEmitter[E] = diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/FileSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/FileSinkConfig.scala index ff833a62..8ad1c4f6 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/FileSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/FileSinkConfig.scala @@ -1,6 +1,5 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.serde._ import io.epiphanous.flinkrunner.util.AvroUtils.instanceOf @@ -11,10 +10,18 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.connector.file.sink.FileSink import org.apache.flink.core.fs.Path import org.apache.flink.core.io.SimpleVersionedSerializer -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.{CheckpointRollingPolicy, OnCheckpointRollingPolicy} -import org.apache.flink.streaming.api.functions.sink.filesystem.{BucketAssigner, OutputFileConfig} +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.{ + BasePathBucketAssigner, + DateTimeBucketAssigner +} +import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.{ + CheckpointRollingPolicy, + OnCheckpointRollingPolicy +} +import org.apache.flink.streaming.api.functions.sink.filesystem.{ + BucketAssigner, + OutputFileConfig +} import org.apache.flink.streaming.api.scala.DataStream import java.nio.charset.StandardCharsets @@ -72,8 +79,7 @@ import scala.collection.JavaConverters._ case class FileSinkConfig[ADT <: FlinkEvent]( name: String, config: FlinkConfig -) extends SinkConfig[ADT] - with LazyLogging { +) extends SinkConfig[ADT] { override def connector: FlinkConnectorName = FlinkConnectorName.File @@ -100,19 +106,17 @@ case class FileSinkConfig[ADT <: FlinkEvent]( * @return * DataStreamSink[E] */ - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - dataStream - .sinkTo( - FileSink - .forRowFormat(destination, getRowEncoder[E]) - .withBucketAssigner(getBucketAssigner) - .withBucketCheckInterval(getBucketCheckInterval) - .withRollingPolicy(getCheckpointRollingPolicy) - .withOutputFileConfig(getOutputFileConfig) - .build() - ) - .setParallelism(parallelism) + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + dataStream.sinkTo( + FileSink + .forRowFormat(destination, getRowEncoder[E]) + .withBucketAssigner(getBucketAssigner) + .withBucketCheckInterval(getBucketCheckInterval) + .withRollingPolicy(getCheckpointRollingPolicy) + .withOutputFileConfig(getOutputFileConfig) + .build() + ) /** Create an bulk avro parquet file sink and send the data stream to it. * @param dataStream @@ -125,10 +129,10 @@ case class FileSinkConfig[ADT <: FlinkEvent]( * @return * DataStream[E] */ - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = { + dataStream: DataStream[E]): Unit = { val sink = format match { case StreamFormatName.Parquet | StreamFormatName.Avro => FileSink diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfig.scala new file mode 100644 index 00000000..c0651970 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfig.scala @@ -0,0 +1,113 @@ +package io.epiphanous.flinkrunner.model.sink + +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.model._ +import io.epiphanous.flinkrunner.serde.{ + EmbeddedAvroJsonSerializationSchema, + JsonSerializationSchema +} +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.serialization.SerializationSchema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.connector.firehose.sink.KinesisFirehoseSink +import org.apache.flink.streaming.api.scala.DataStream + +/** AWS kinesis firehose sink config. + * + * Follow the instructions from the Amazon + * Kinesis Data Firehose Developer Guide to setup a Kinesis Data + * Firehose delivery stream. + * + * Configuration: + * - stream - required name of the kinesis firehose delivery stream + * - aws.region - optional aws region where kinesis is hosted (defaults + * to us-east-1) + * - aws.endpoint - optional aws kinesis endpoint (defaults to normal + * endpoint from configured kinesis region, but you can specify if you + * are using localstack) + * - client - optional kinesis client properties (aws.region or + * aws.endpoint can also be specified here) + * - max.batch.size.in.number: the maximum size of a batch of entries + * that may be sent to KDS + * - max.in.flight.requests: the maximum number of in flight requests + * that may exist, if any more in flight requests need to be initiated + * once the maximum has been reached, then it will be blocked until + * some have completed + * - max.buffered.requests: the maximum number of elements held in the + * buffer, requests to add elements will be blocked while the number + * of elements in the buffer is at the maximum + * - max.batch.size.in.bytes: the maximum size of a batch of entries + * that may be sent to KDS measured in bytes + * - max.time.in.buffer: the maximum amount of time an entry is allowed + * to live in the buffer, if any element reaches this age, the entire + * buffer will be flushed immediately + * - max.record.size.in.bytes: the maximum size of a record the sink + * will accept into the buffer, a record of size larger than this will + * be rejected when passed to the sink + * - fail.on.error: when an exception is encountered while persisting to + * Kinesis Data Streams, the job will fail immediately if failOnError + * is set + * + * @param name + * name of the sink + * @param config + * flinkrunner config + * @tparam ADT + * the flinkrunner algebraic data type + */ +case class FirehoseSinkConfig[ADT <: FlinkEvent: TypeInformation]( + name: String, + config: FlinkConfig +) extends SinkConfig[ADT] + with LazyLogging { + override def connector: FlinkConnectorName = + FlinkConnectorName.Firehose + + val props: KinesisProperties = KinesisProperties.fromSinkConfig(this) + + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink(dataStream, getSerializationSchema[E]) + + override def addAvroSink[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink( + dataStream, + getAvroSerializationSchema[E, A] + ) + + def _addSink[E <: ADT]( + dataStream: DataStream[E], + serializationSchema: SerializationSchema[E]): Unit = { + val kfs = { + val k = KinesisFirehoseSink + .builder[E]() + .setFirehoseClientProperties(props.clientProperties) + .setSerializationSchema(serializationSchema) + .setDeliveryStreamName(props.stream) + .setFailOnError(props.failOnError) + .setMaxInFlightRequests(props.maxInFlightRequests) + .setMaxBufferedRequests(props.maxBufferedRequests) + .setMaxBatchSize(props.maxBatchSizeInNumber) + .setMaxBatchSizeInBytes(props.maxBatchSizeInBytes) + .setMaxTimeInBufferMS(props.maxBufferTime) + props.maxRecordSizeInBytes + .map(k.setMaxBatchSizeInBytes) + .getOrElse(k) + }.build() + dataStream.sinkTo(kfs).setParallelism(parallelism) + () + } + + def getSerializationSchema[E <: ADT: TypeInformation] + : SerializationSchema[E] = + new JsonSerializationSchema[E, ADT](this) + + def getAvroSerializationSchema[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation] = + new EmbeddedAvroJsonSerializationSchema[E, A, ADT](this) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfig.scala new file mode 100644 index 00000000..a16187f4 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfig.scala @@ -0,0 +1,160 @@ +package io.epiphanous.flinkrunner.model.sink + +import io.epiphanous.flinkrunner.model._ +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.api.TableSchema +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.types.utils.TypeConversions +import org.apache.flink.types.Row +import org.apache.iceberg.flink.sink.FlinkSink +import org.apache.iceberg.flink.{ + FlinkSchemaUtil, + FlinkWriteOptions, + TableLoader +} +import org.apache.iceberg.{PartitionSpec, Schema, Table} + +import scala.collection.JavaConverters._ +import scala.util.Try + +/** An iceberg sink configuration. + * @param name + * sink name + * @param config + * sink configuration + * @tparam ADT + * the flinkrunner algebraic data type + */ +case class IcebergSinkConfig[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig +) extends SinkConfig[ADT] { + + override def connector: FlinkConnectorName = FlinkConnectorName.Iceberg + + val icebergCommonConfig: IcebergCommonConfig = IcebergCommonConfig(this) + + val primaryKey: Seq[String] = config.getStringListOpt(pfx("primary.key")) + + val partitionSpecConfig: Seq[IcebergPartitionColumn] = + config + .getObjectList(pfx("partition.spec")) + .map(_.toConfig) + .map(IcebergPartitionColumn.apply) + + val writeFormat: String = config + .getStringOpt(pfx("write.format")) + .map(_.toLowerCase) + .getOrElse("parquet") + + val writeParallelism: Int = + config.getIntOpt("write.parallelism").getOrElse(2) + + /** Convert row type into a flink TableSchema. TableSchema is deprecated + * in Flink, but the current Iceberg integration with Flink relies on + * it. + * @param rowType + * RowType - The type of Row objects flowing into the sink + * @return + * TableSchema + */ + def getFlinkTableSchema(rowType: RowType): TableSchema = { + val tsb = rowType.getFields.asScala.foldLeft(TableSchema.builder()) { + case (b, f) => + b.field( + f.getName, + TypeConversions.fromLogicalToDataType(f.getType) + ) + b + } + if (primaryKey.nonEmpty) tsb.primaryKey(primaryKey: _*) + tsb.build() + } + + /** Given the table schema, try to ensure the target table exists in the + * catalog + * + * TODO: this needs more work to handle altering table definition if it + * does exist but doesn't match the configured specs + * + * @param flinkTableSchema + * a flink TableSchema instance + * @return + * Try[Boolean] - true if we created the table, false otherwise + */ + def maybeCreateTable(flinkTableSchema: TableSchema): Try[Table] = { + logger.debug(s"enter maybeCreateTable $flinkTableSchema") + val t = Try { + val icebergSchema: Schema = FlinkSchemaUtil.convert(flinkTableSchema) + logger.debug(icebergSchema.toString) + val catalog = icebergCommonConfig.catalogLoader.loadCatalog() + logger.debug(catalog.toString) + val ps = + if (partitionSpecConfig.nonEmpty) + partitionSpecConfig + .foldLeft(PartitionSpec.builderFor(icebergSchema)) { + case (psb, pc) => + pc.addToSpec(psb) + } + .build() + else PartitionSpec.unpartitioned() + logger.debug(ps.toString) + logger.debug(icebergCommonConfig.tableIdentifier.toString) + if (catalog.tableExists(icebergCommonConfig.tableIdentifier)) + catalog.loadTable(icebergCommonConfig.tableIdentifier) + else + catalog.createTable( + icebergCommonConfig.tableIdentifier, + icebergSchema, + ps + ) + } + logger.debug(s"exit maybeCreateTable with $t") + t + } + + /** Add an iceberg row sink for the given avro data stream and row type + * + * @param rows + * a stream of rows + * @param rowType + * configured or inferred from avro row type + */ + override def _addRowSink( + rows: DataStream[Row], + rowType: RowType): Unit = { + val flinkTableSchema = getFlinkTableSchema(rowType) + maybeCreateTable(flinkTableSchema).fold( + err => + throw new RuntimeException( + s"Failed to create iceberg table ${icebergCommonConfig.tableIdentifier}", + err + ), + table => logger.info(s"iceberg table $table ready") + ) + FlinkSink + .forRow(rows.javaStream, flinkTableSchema) + .set(FlinkWriteOptions.WRITE_FORMAT.toString, writeFormat) + .upsert(primaryKey.nonEmpty) + .writeParallelism(writeParallelism) + .tableLoader( + TableLoader.fromCatalog( + icebergCommonConfig.catalogLoader, + icebergCommonConfig.tableIdentifier + ) + ) + .append() + + } + + override def addAvroSink[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation]( + dataStream: DataStream[E]): Unit = notImplementedError("addAvroSink") + + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + notImplementedError("addSink") +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkConfig.scala index 729d6fd8..849548ce 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkConfig.scala @@ -1,26 +1,31 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.SupportedDatabase.{Postgresql, Snowflake} +import io.epiphanous.flinkrunner.model.SupportedDatabase.{ + Postgresql, + Snowflake +} import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.model.sink.JdbcSinkConfig.{DEFAULT_CONNECTION_TIMEOUT, DEFAULT_TIMESCALE_CHUNK_TIME_INTERVAL, DEFAULT_TIMESCALE_NUMBER_PARTITIONS} -import io.epiphanous.flinkrunner.operator.CreateTableJdbcSinkFunction -import io.epiphanous.flinkrunner.serde.JsonRowEncoder +import io.epiphanous.flinkrunner.model.sink.JdbcSinkConfig.{ + DEFAULT_CONNECTION_TIMEOUT, + DEFAULT_TIMESCALE_CHUNK_TIME_INTERVAL, + DEFAULT_TIMESCALE_NUMBER_PARTITIONS +} import io.epiphanous.flinkrunner.util.SqlBuilder import org.apache.avro.generic.GenericRecord -import org.apache.flink.api.common.functions.RuntimeContext import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.createTypeInformation -import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat -import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat.StatementExecutorFactory import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor -import org.apache.flink.connector.jdbc.{JdbcConnectionOptions, JdbcExecutionOptions, JdbcStatementBuilder} -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.connector.jdbc.internal.{ + GenericJdbcSinkFunction, + JdbcOutputFormat +} +import org.apache.flink.connector.jdbc.{ + JdbcConnectionOptions, + JdbcExecutionOptions, + JdbcStatementBuilder +} import org.apache.flink.streaming.api.scala.DataStream -import java.sql.{Connection, DriverManager, PreparedStatement, Timestamp} -import java.time.Instant -import java.util.function.{Function => JavaFunction} +import java.sql.{Connection, DriverManager} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.{Failure, Success, Try} @@ -104,8 +109,7 @@ import scala.util.{Failure, Success, Try} case class JdbcSinkConfig[ADT <: FlinkEvent]( name: String, config: FlinkConfig) - extends SinkConfig[ADT] - with LazyLogging { + extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.Jdbc @@ -564,87 +568,9 @@ case class JdbcSinkConfig[ADT <: FlinkEvent]( jcoBuilder.build() } - /** Creates a statement builder based on the target table columns and the - * values in an event - * @tparam E - * the event type - * @return - * JdbcStatementBuilder[E] - */ - def getStatementBuilder[E <: ADT]: JdbcStatementBuilder[E] = - new JdbcStatementBuilder[E] { - override def accept(statement: PreparedStatement, element: E): Unit = - _fillInStatement( - fieldValuesOf(element), - statement, - element - ) - } - - def fieldValuesOf[T <: Product](product: T): Map[String, Any] = { - product.getClass.getDeclaredFields - .map(_.getName) - .zip(product.productIterator.toIndexedSeq) - .toMap - } - - def getAvroStatementBuilder[ - E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation]: JdbcStatementBuilder[E] = - new JdbcStatementBuilder[E] { - override def accept( - statement: PreparedStatement, - element: E): Unit = { - println(s"XXX: $element") - _fillInStatement[E]( - fieldValuesOf(element.$record.asInstanceOf[Product]), - statement, - element - ) - } - } - - def _fillInStatement[E <: ADT]( - data: Map[String, Any], - statement: PreparedStatement, - element: E): Unit = { - columns.zipWithIndex.map(x => (x._1, x._2 + 1)).foreach { - case (column, i) => - data.get(column.name) match { - case Some(v) => - val value = v match { - case null | None => null - case Some(x) => _matcher(x) - case x => _matcher(x) - } - statement.setObject(i, value, column.dataType.jdbcType) - case None => - throw new RuntimeException( - s"value for field ${column.name} is not in $element" - ) - } - } - } - - def _matcher(value: Any): Any = { - lazy val encoder = new JsonRowEncoder[Map[String, Any]]() - value match { - case ts: Instant => Timestamp.from(ts) - case m: Map[String, Any] => - try - encoder.encode(m).get - catch { - case _: Throwable => - println(s"Failure to encode map: $m") - null - } - case _ => value - } - } - - def _getSink[E <: ADT: TypeInformation]( + def _addSink[E <: ADT: TypeInformation]( dataStream: DataStream[E], - statementBuilder: JdbcStatementBuilder[E]): DataStreamSink[E] = { + statementBuilder: JdbcStatementBuilder[E]): Unit = { val jdbcOutputFormat = new JdbcOutputFormat[E, E, JdbcBatchStatementExecutor[E]]( new BasicJdbcConnectionProvider( @@ -652,40 +578,34 @@ case class JdbcSinkConfig[ADT <: FlinkEvent]( properties ), getJdbcExecutionOptions, - // ================================= - // NOTE: following line should NOT be converted to a Single Abstract Method - // in order to prevent flink's closure container from complaining about - // serialization. - // ================================== - new StatementExecutorFactory[JdbcBatchStatementExecutor[E]] { - override def apply( - t: RuntimeContext): JdbcBatchStatementExecutor[E] = { - JdbcBatchStatementExecutor.simple( - queryDml, - statementBuilder, - JavaFunction.identity[E] - ) - } - }, + new JdbcSinkStatementExecutorFactory[E, ADT]( + queryDml, + statementBuilder + ), JdbcOutputFormat.RecordExtractor.identity[E] ) + maybeCreateTable() dataStream .addSink( - new CreateTableJdbcSinkFunction[E, ADT](this, jdbcOutputFormat) + new GenericJdbcSinkFunction[E](jdbcOutputFormat) ) .uid(label) .name(label) + .setParallelism(parallelism) } - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink(dataStream, getStatementBuilder[E]) + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink(dataStream, new JdbcSinkStatementBuilder[E, ADT](columns)) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink(dataStream, getAvroStatementBuilder[E, A]) + dataStream: DataStream[E]): Unit = + _addSink( + dataStream, + new JdbcSinkAvroStatementBuilder[E, A, ADT](columns) + ) } object JdbcSinkConfig { diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/KafkaSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/KafkaSinkConfig.scala index b45487c3..67eb73e1 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/KafkaSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/KafkaSinkConfig.scala @@ -1,15 +1,19 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.serde.{ConfluentAvroRegistryKafkaRecordSerializationSchema, JsonKafkaRecordSerializationSchema} +import io.epiphanous.flinkrunner.serde.{ + ConfluentAvroRegistryKafkaRecordSerializationSchema, + JsonKafkaRecordSerializationSchema +} import io.epiphanous.flinkrunner.util.ConfigToProps import io.epiphanous.flinkrunner.util.ConfigToProps.getFromEither import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.connector.base.DeliveryGuarantee -import org.apache.flink.connector.kafka.sink.{KafkaRecordSerializationSchema, KafkaSink} -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.connector.kafka.sink.{ + KafkaRecordSerializationSchema, + KafkaSink +} import org.apache.flink.streaming.api.scala.DataStream import java.time.Duration @@ -29,8 +33,7 @@ import java.util.Properties case class KafkaSinkConfig[ADT <: FlinkEvent: TypeInformation]( name: String, config: FlinkConfig -) extends SinkConfig[ADT] - with LazyLogging { +) extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.Kafka @@ -129,21 +132,17 @@ case class KafkaSinkConfig[ADT <: FlinkEvent: TypeInformation]( : KafkaRecordSerializationSchema[E] = new JsonKafkaRecordSerializationSchema[E, ADT](this) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - dataStream - .sinkTo(_getSink[E](getAvroSerializationSchema[E, A])) - .setParallelism(parallelism) - - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - dataStream - .sinkTo(_getSink[E](getSerializationSchema[E])) - .setParallelism(parallelism) - - def _getSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + dataStream.sinkTo(_addSink[E](getAvroSerializationSchema[E, A])) + + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + dataStream.sinkTo(_addSink[E](getSerializationSchema[E])) + + def _addSink[E <: ADT: TypeInformation]( serializer: KafkaRecordSerializationSchema[E]): KafkaSink[E] = KafkaSink .builder() diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/KinesisSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/KinesisSinkConfig.scala index e17b8396..e92cf81a 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/KinesisSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/KinesisSinkConfig.scala @@ -1,12 +1,6 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{ - EmbeddedAvroRecord, - FlinkConfig, - FlinkConnectorName, - FlinkEvent -} +import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.serde.{ EmbeddedAvroJsonSerializationSchema, JsonSerializationSchema @@ -15,66 +9,85 @@ import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.serialization.SerializationSchema import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.connector.kinesis.sink.KinesisStreamsSink -import org.apache.flink.streaming.api.datastream.DataStreamSink import org.apache.flink.streaming.api.scala.DataStream -/** - maxBatchSize: the maximum size of a batch of entries that may be - * sent to KDS - * - maxInFlightRequests: the maximum number of in flight requests that - * may exist, if any more in flight requests need to be initiated once - * the maximum has been reached, then it will be blocked until some - * have completed - * - maxBufferedRequests: the maximum number of elements held in the +/** Kinesis Sink Config + * + * Configuration: Configuration: + * - stream - required name of the kinesis firehose delivery stream + * - aws.region - optional aws region where kinesis is hosted (defaults + * to us-east-1) + * - aws.endpoint - optional aws kinesis endpoint (defaults to normal + * endpoint from configured kinesis region, but you can specify if you + * are using localstack) + * - client - optional kinesis client properties (aws.region or + * aws.endpoint can also be specified here) + * - max.batch.size.in.number: the maximum size of a batch of entries + * that may be sent to KDS + * - max.in.flight.requests: the maximum number of in flight requests + * that may exist, if any more in flight requests need to be initiated + * once the maximum has been reached, then it will be blocked until + * some have completed + * - max.buffered.requests: the maximum number of elements held in the * buffer, requests to add elements will be blocked while the number * of elements in the buffer is at the maximum - * - maxBatchSizeInBytes: the maximum size of a batch of entries that - * may be sent to KDS measured in bytes - * - maxTimeInBufferMS: the maximum amount of time an entry is allowed + * - max.batch.size.in.bytes: the maximum size of a batch of entries + * that may be sent to KDS measured in bytes + * - max.time.in.buffer: the maximum amount of time an entry is allowed * to live in the buffer, if any element reaches this age, the entire * buffer will be flushed immediately - * - maxRecordSizeInBytes: the maximum size of a record the sink will - * accept into the buffer, a record of size larger than this will be - * rejected when passed to the sink - * - failOnError: when an exception is encountered while persisting to + * - max.record.size.in.bytes: the maximum size of a record the sink + * will accept into the buffer, a record of size larger than this will + * be rejected when passed to the sink + * - fail.on.error: when an exception is encountered while persisting to * Kinesis Data Streams, the job will fail immediately if failOnError * is set */ case class KinesisSinkConfig[ADT <: FlinkEvent: TypeInformation]( name: String, config: FlinkConfig -) extends SinkConfig[ADT] - with LazyLogging { +) extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.Kinesis - val stream: String = config.getString(pfx("stream")) + val props: KinesisProperties = KinesisProperties.fromSinkConfig(this) - def _getSink[E <: ADT: TypeInformation]( + def _addSink[E <: ADT: TypeInformation]( dataStream: DataStream[E], - serializationSchema: SerializationSchema[E]): DataStreamSink[E] = { + serializationSchema: SerializationSchema[E]): Unit = { + val ks = { + val kb = KinesisStreamsSink + .builder[E] + .setKinesisClientProperties(props.clientProperties) + .setSerializationSchema(serializationSchema) + .setPartitionKeyGenerator(element => element.$key) + .setStreamName(props.stream) + .setFailOnError(props.failOnError) + .setMaxBatchSize(props.maxBatchSizeInNumber) + .setMaxBatchSizeInBytes(props.maxBatchSizeInBytes) + .setMaxInFlightRequests(props.maxInFlightRequests) + .setMaxBufferedRequests(props.maxBufferedRequests) + .setMaxTimeInBufferMS(props.maxBufferTime) + props.maxRecordSizeInBytes + .map(kb.setMaxRecordSizeInBytes) + .getOrElse(kb) + }.build() dataStream - .sinkTo( - KinesisStreamsSink - .builder[E] - .setStreamName(stream) - .setFailOnError(true) - .setSerializationSchema(serializationSchema) - .setKinesisClientProperties(properties) - .build() - ) + .sinkTo(ks) .uid(label) .name(label) + .setParallelism(parallelism) } - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getSerializationSchema[E]) + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getSerializationSchema[E]) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getAvroSerializationSchema[E, A]) + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getAvroSerializationSchema[E, A]) def getSerializationSchema[E <: ADT: TypeInformation] : SerializationSchema[E] = diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/PrintSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/PrintSinkConfig.scala new file mode 100644 index 00000000..0128b5ab --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/PrintSinkConfig.scala @@ -0,0 +1,37 @@ +package io.epiphanous.flinkrunner.model.sink + +import io.epiphanous.flinkrunner.model.{ + EmbeddedAvroRecord, + FlinkConfig, + FlinkConnectorName, + FlinkEvent +} +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.types.Row + +case class PrintSinkConfig[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig +) extends SinkConfig[ADT] { + + override val connector: FlinkConnectorName = FlinkConnectorName.Print + + def _print[T](stream: DataStream[T]): Unit = { + stream.print() + () + } + + override def addSink[E <: ADT: TypeInformation]( + stream: DataStream[E]): Unit = _print(stream) + + override def addAvroSink[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation](stream: DataStream[E]): Unit = + _print(stream) + + override def _addRowSink(rows: DataStream[Row], rowType: RowType): Unit = + _print(rows) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/RabbitMQSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/RabbitMQSinkConfig.scala index 431ac403..ed57c68b 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/RabbitMQSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/RabbitMQSinkConfig.scala @@ -1,20 +1,23 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.serde.{EmbeddedAvroJsonSerializationSchema, JsonSerializationSchema} +import io.epiphanous.flinkrunner.serde.{ + EmbeddedAvroJsonSerializationSchema, + JsonSerializationSchema +} import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.serialization.SerializationSchema import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.datastream.DataStreamSink import org.apache.flink.streaming.api.scala.DataStream -import org.apache.flink.streaming.connectors.rabbitmq.{RMQSink, RMQSinkPublishOptions} +import org.apache.flink.streaming.connectors.rabbitmq.{ + RMQSink, + RMQSinkPublishOptions +} case class RabbitMQSinkConfig[ADT <: FlinkEvent: TypeInformation]( name: String, config: FlinkConfig) - extends SinkConfig[ADT] - with LazyLogging { + extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.RabbitMQ @@ -36,9 +39,9 @@ case class RabbitMQSinkConfig[ADT <: FlinkEvent: TypeInformation]( def getPublishOptions[E <: ADT: TypeInformation] : Option[RMQSinkPublishOptions[E]] = None - def _getSink[E <: ADT: TypeInformation]( + def _addSink[E <: ADT: TypeInformation]( dataStream: DataStream[E], - serializationSchema: SerializationSchema[E]): DataStreamSink[E] = { + serializationSchema: SerializationSchema[E]): Unit = { val connConfig = connectionInfo.rmqConfig val sink = getPublishOptions[E] match { @@ -59,14 +62,14 @@ case class RabbitMQSinkConfig[ADT <: FlinkEvent: TypeInformation]( .setParallelism(parallelism) } - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getSerializationSchema[E]) + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getSerializationSchema[E]) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getAvroSerializationSchema[E, A]) + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getAvroSerializationSchema[E, A]) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/SinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/SinkConfig.scala index 1bf238d3..1783e4fa 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/SinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/SinkConfig.scala @@ -1,16 +1,16 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model.FlinkConnectorName._ import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.util.StreamUtils._ +import io.epiphanous.flinkrunner.util.RowUtils.rowTypeOf import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.types.Row -import java.util -import java.util.Properties +import scala.reflect.runtime.{universe => ru} /** A flinkrunner sink configuration trait. All sink configs have a few * common configuration options. @@ -22,6 +22,7 @@ import java.util.Properties * - [[FlinkConnectorName.Cassandra]] * - [[FlinkConnectorName.Elasticsearch]] * - [[FlinkConnectorName.File]] + * - [[FlinkConnectorName.Firehose]] * - [[FlinkConnectorName.Jdbc]] * - [[FlinkConnectorName.Kafka]] * - [[FlinkConnectorName.Kinesis]] @@ -31,35 +32,32 @@ import java.util.Properties * @tparam ADT * the flinkrunner algebraic data type */ -trait SinkConfig[ADT <: FlinkEvent] extends LazyLogging { - def name: String - def config: FlinkConfig - def connector: FlinkConnectorName +trait SinkConfig[ADT <: FlinkEvent] extends SourceOrSinkConfig[ADT] { - def pfx(path: String = ""): String = Seq( - Some("sinks"), - Some(name), - if (path.isEmpty) None else Some(path) - ).flatten.mkString(".") + override val _sourceOrSink = "sink" - val properties: Properties = config.getProperties(pfx("config")) + def addSink[E <: ADT: TypeInformation](stream: DataStream[E]): Unit - lazy val propertiesMap: util.HashMap[String, String] = - properties.asJavaMap - - lazy val label: String = s"${connector.entryName.toLowerCase}/$name" - - lazy val parallelism: Int = config - .getIntOpt(pfx("parallelism")) - .getOrElse(config.globalParallelism) + def addAvroSink[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation + ]( + stream: DataStream[E] + ): Unit - def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] + def addRowSink[ + E <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag]( + stream: DataStream[E]): Unit = + _addRowSink( + stream + .map((e: E) => e.toRow) + .name(s"row:${stream.name}") + .uid(s"row:${stream.name}") + .setParallelism(stream.parallelism), + rowTypeOf[E] + ) - def getAvroSink[ - E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] + def _addRowSink(rows: DataStream[Row], rowType: RowType): Unit = ??? } @@ -76,6 +74,7 @@ object SinkConfig { ) match { case Kafka => KafkaSinkConfig(name, config) case Kinesis => KinesisSinkConfig(name, config) + case Firehose => FirehoseSinkConfig(name, config) case File => FileSinkConfig(name, config) case Socket => SocketSinkConfig(name, config) case Jdbc => JdbcSinkConfig(name, config) @@ -84,9 +83,11 @@ object SinkConfig { case Elasticsearch => ElasticsearchSinkConfig(name, config) case RabbitMQ => RabbitMQSinkConfig(name, config) + case Iceberg => IcebergSinkConfig(name, config) + case Print => PrintSinkConfig(name, config) case connector => throw new RuntimeException( - s"Don't know how to configure ${connector.entryName} sink connector $name (job ${config.jobName}" + s"Don't know how to configure ${connector.entryName} sink connector <$name> (in job <${config.jobName}>)" ) } } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/sink/SocketSinkConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/sink/SocketSinkConfig.scala index 4674b62d..40752ffe 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/sink/SocketSinkConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/sink/SocketSinkConfig.scala @@ -1,14 +1,17 @@ package io.epiphanous.flinkrunner.model.sink -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.serde._ import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.serialization.SerializationSchema import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.datastream.DataStreamSink +import org.apache.flink.formats.csv.CsvRowSerializationSchema +import org.apache.flink.formats.json.JsonRowSerializationSchema import org.apache.flink.streaming.api.functions.sink.SocketClientSink import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.types.Row import java.nio.charset.StandardCharsets import scala.util.{Failure, Success} @@ -16,8 +19,7 @@ import scala.util.{Failure, Success} case class SocketSinkConfig[ADT <: FlinkEvent]( name: String, config: FlinkConfig) - extends SinkConfig[ADT] - with LazyLogging { + extends SinkConfig[ADT] { override val connector: FlinkConnectorName = FlinkConnectorName.Socket @@ -45,6 +47,30 @@ case class SocketSinkConfig[ADT <: FlinkEvent]( ) } + def getRowSerializationSchema( + rowType: RowType): SerializationSchema[Row] = format match { + case StreamFormatName.Json => + JsonRowSerializationSchema + .builder() + .withTypeInfo(InternalTypeInfo.of(rowType)) + .build() + + case StreamFormatName.Csv | StreamFormatName.Tsv | + StreamFormatName.Psv | StreamFormatName.Delimited => + val delimitedConfig = DelimitedConfig.get(format, pfx(), config) + val b = + new CsvRowSerializationSchema.Builder(InternalTypeInfo.of(rowType)) + .setFieldDelimiter(delimitedConfig.columnSeparator) + .setQuoteCharacter(delimitedConfig.quoteCharacter) + .setEscapeCharacter(delimitedConfig.escapeCharacter) + if (!delimitedConfig.useQuotes) b.disableQuoteCharacter() + b.build() + case StreamFormatName.Parquet | StreamFormatName.Avro => + throw new RuntimeException( + s"invalid format ${format.entryName} for socket sink $name" + ) + } + def getSerializationSchema[E <: ADT: TypeInformation] : SerializationSchema[E] = new SerializationSchema[E] { @@ -58,33 +84,36 @@ case class SocketSinkConfig[ADT <: FlinkEvent]( } } - override def getSink[E <: ADT: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getSerializationSchema[E]) + override def _addRowSink( + stream: DataStream[Row], + rowType: RowType): Unit = + _addSink[Row](stream, getRowSerializationSchema(rowType)) + + override def addSink[E <: ADT: TypeInformation]( + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getSerializationSchema[E]) def getAvroSerializationSchema[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]: SerializationSchema[E] = new EmbeddedAvroJsonSerializationSchema[E, A, ADT](this) - override def getAvroSink[ + override def addAvroSink[ E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, A <: GenericRecord: TypeInformation]( - dataStream: DataStream[E]): DataStreamSink[E] = - _getSink[E](dataStream, getAvroSerializationSchema[E, A]) + dataStream: DataStream[E]): Unit = + _addSink[E](dataStream, getAvroSerializationSchema[E, A]) - def _getSink[E <: ADT: TypeInformation]( + def _addSink[E]( dataStream: DataStream[E], - serializationSchema: SerializationSchema[E]): DataStreamSink[E] = - dataStream - .addSink( - new SocketClientSink[E]( - host, - port, - serializationSchema, - maxRetries, - autoFlush - ) + serializationSchema: SerializationSchema[E]): Unit = + dataStream.addSink( + new SocketClientSink[E]( + host, + port, + serializationSchema, + maxRetries, + autoFlush ) - .setParallelism(parallelism) + ) } diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/EmptySourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/EmptySourceConfig.scala new file mode 100644 index 00000000..a173c568 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/EmptySourceConfig.scala @@ -0,0 +1,42 @@ +package io.epiphanous.flinkrunner.model.source + +import io.epiphanous.flinkrunner.model._ +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} +import org.apache.flink.table.data.{GenericRowData, RowData} + +case class EmptySourceConfig[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig) + extends SourceConfig[ADT] { + override def connector: FlinkConnectorName = FlinkConnectorName.Empty + + def _emptySource[E: TypeInformation]( + env: StreamExecutionEnvironment): DataStream[E] = { + val x = env.fromCollection(Seq.empty[E]) + x + } + + override def getSourceStream[E <: ADT: TypeInformation]( + env: StreamExecutionEnvironment): DataStream[E] = + _emptySource[E](env) + + override def getAvroSourceStream[ + E <: ADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation]( + env: StreamExecutionEnvironment)(implicit + fromKV: EmbeddedAvroRecordInfo[A] => E): DataStream[E] = + _emptySource[E](env) + + override def getRowSourceStream[ + E <: ADT with EmbeddedRowType: TypeInformation]( + env: StreamExecutionEnvironment)(implicit + fromRowData: RowData => E): DataStream[E] = + _emptySource[GenericRowData](env).map(fromRowData) + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfig.scala index f507896c..5e9d5a83 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfig.scala @@ -13,11 +13,20 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.connector.source.{Source, SourceSplit} import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.connector.file.src.FileSource -import org.apache.flink.connector.file.src.reader.{StreamFormat, TextLineInputFormat} +import org.apache.flink.connector.file.src.reader.{ + StreamFormat, + TextLineInputFormat +} import org.apache.flink.core.fs.Path import org.apache.flink.formats.avro.AvroInputFormat -import org.apache.flink.streaming.api.functions.source.{FileProcessingMode, SourceFunction} -import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} +import org.apache.flink.streaming.api.functions.source.{ + FileProcessingMode, + SourceFunction +} +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} import org.apache.flink.util.Collector import java.time.Duration diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfig.scala new file mode 100644 index 00000000..e2e3ec9b --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfig.scala @@ -0,0 +1,120 @@ +package io.epiphanous.flinkrunner.model.source + +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkConnectorName, + FlinkEvent, + IcebergCommonConfig +} +import org.apache.flink.api.common.eventtime.WatermarkStrategy +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} +import org.apache.flink.table.data.RowData +import org.apache.iceberg.flink.TableLoader +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory +import org.apache.iceberg.flink.source.{ + IcebergSource, + StreamingStartingStrategy +} + +import java.time.Duration + +case class IcebergSourceConfig[ADT <: FlinkEvent]( + name: String, + config: FlinkConfig) + extends SourceConfig[ADT] { + + override val connector: FlinkConnectorName = FlinkConnectorName.Iceberg + + val common: IcebergCommonConfig = IcebergCommonConfig(this) + + val streamingOpt: Option[Boolean] = config.getBooleanOpt("streaming") + val batchOpt: Option[Boolean] = config.getBooleanOpt(pfx("batch")) + val batch: Boolean = + streamingOpt.contains(false) || batchOpt.contains(true) + + val streamingStartingStrategy: Option[StreamingStartingStrategy] = config + .getStringOpt(pfx("start.strategy")) + .map(_.toLowerCase.replaceAll("[^a-zA-Z]+", "_")) + .map { + case "full" | "table_scan_then_incremental" => + StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL + case "latest" | "incremental_from_latest_snapshot" => + StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT + case "earliest" | "incremental_from_earliest_snapshot" => + StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT + case "id" | "snapshot_id" | "incremental_from_snapshot_id" => + StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID + case "ts" | "snapshot_ts" | "incremental_from_snapshot_timestamp" => + StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP + case ss => + throw new RuntimeException( + s"Unknown start.strategy $ss for iceberg source $name" + ) + } + + val startSnapshotId: Option[Long] = + config.getLongOpt(pfx("start.snapshot.id")) + + val startSnapshotTs: Option[Long] = + config.getLongOpt(pfx("start.snapshot.timestamp")) + + val monitoringInterval: Option[Duration] = + config.getDurationOpt(pfx("monitor.interval")) + + if ( + streamingStartingStrategy.contains( + StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID + ) + && startSnapshotId.isEmpty + ) + throw new RuntimeException( + s"Missing configuration for start.snapshot.id for iceberg source $name" + ) + + if ( + streamingStartingStrategy.contains( + StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP + ) + && startSnapshotTs.isEmpty + ) + throw new RuntimeException( + s"Missing configuration for start.snapshot.timestamp for iceberg source $name" + ) + + override def getRowSource( + env: StreamExecutionEnvironment): DataStream[RowData] = { + val sourceBuilder = IcebergSource + .forRowData() + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(!batch) + .tableLoader( + TableLoader + .fromCatalog(common.catalogLoader, common.tableIdentifier) + ) + logger.debug(s"setting properties: $propertiesMap") + sourceBuilder.setAll(propertiesMap) + if (!batch) { + streamingStartingStrategy.foreach(ss => + sourceBuilder.streamingStartingStrategy(ss) + ) + startSnapshotId.foreach(id => sourceBuilder.startSnapshotId(id)) + startSnapshotTs.foreach(ts => + sourceBuilder.startSnapshotTimestamp(ts) + ) + monitoringInterval.foreach(mi => sourceBuilder.monitorInterval(mi)) + } + env + .fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + name + ) + .uid(uid) + .setParallelism(parallelism) + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfig.scala index 01ff5526..da6f1081 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfig.scala @@ -16,6 +16,9 @@ import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants._ import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema +import java.text.SimpleDateFormat +import java.time.Instant +import scala.collection.JavaConverters._ import scala.util.Try /** A source config for kinesis streams. For example, the following config @@ -35,19 +38,26 @@ import scala.util.Try * Configuration options: * - `connector`: `kafka` (required only if it can't be inferred from * the source name) - * - `stream`: the name of the kinesis stream + * - `stream` (`streams`): the name of the kinesis stream or streams to + * consume. If you want to read from multiple streams, either use the + * `stream` property and separate stream names with commas (`stream = + * a,b,c`), or use the `streams` property and configure an array + * (`streams = [ a, b, c ]`). * - `starting.position`: the starting position of the stream; one of: * - `TRIM_HORIZON`: the position of the earliest data in a shard * - `LATEST`: the position after the most recent data in a shard * - `AT_TIMESTAMP`: on or after the `starting.timestamp` - * - `starting.timestamp`: a timestamp as fractional epoch seconds - * (format: `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`) + * - `starting.timestamp`: a timestamp as fractional epoch seconds, + * formatted according to `timestamp.format` + * - `timestamp.format`: a valid DateTimeFormatter pattern (default + * `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`) * - `use.efo`: if true, turn on enhanced fan-out to read the stream * faster (defaults to true, may cost more money) * - `efo.consumer`: name of the efo consumer (defaults to * `jobName`.`sourceName`) * - `aws.region`: AWS region of your kinesis endpoint - * - `config`: optional config to pass to kinesis client + * - `config`: optional config to pass to kinesis client (see + * [[org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants]]) * * @param name * name of the source @@ -69,14 +79,35 @@ case class KinesisSourceConfig[ADT <: FlinkEvent]( ) properties.setProperty(AWS_REGION, awsRegion) - val stream: String = Try(config.getString(pfx("stream"))).fold( - t => + val streams: List[String] = { + + val streamOpt: Option[String] = Try( + config + .getString(pfx("stream")) + ).toOption.orElse(Try(config.getString(pfx("streams"))).toOption) + + val streamsOpt: Option[List[String]] = + Try(config.getStringList(pfx("stream"))).toOption.orElse( + Try(config.getStringList(pfx("streams"))).toOption + ) + + if (streamOpt.isEmpty && streamsOpt.isEmpty) { throw new RuntimeException( - s"kinesis source $name is missing required 'stream' property", - t - ), - s => s - ) + s"Kinesis source $name is missing required 'stream' or 'streams' property" + ) + } + + if (streamOpt.nonEmpty && streamsOpt.nonEmpty) { + throw new RuntimeException( + s"Kinesis source $name has both 'stream' and 'streams' properties. Please specify one or the other." + ) + } + + (streamOpt.map( + _.split("\\s*[,;|]\\s*").toList + ) ++ streamsOpt).flatten.toList + + } val startPos: String = { val pos = getFromEither( @@ -101,29 +132,58 @@ case class KinesisSourceConfig[ADT <: FlinkEvent]( ) } - val startTimestampOpt: Option[String] = getFromEither( - pfx(), - Seq( - "starting.timestamp", - "starting.ts", - "start.timestamp", - "start.ts" - ), - config.getStringOpt - ) - properties.setProperty(STREAM_INITIAL_POSITION, startPos) - if ( - startPos.equalsIgnoreCase( - InitialPosition.AT_TIMESTAMP.name() - ) - ) { - startTimestampOpt.fold( - throw new RuntimeException( - s"kinesis sink $name set starting.position to AT_TIMESTAMP but provided no starting.timestamp" - ) - )(ts => properties.setProperty(STREAM_INITIAL_TIMESTAMP, ts)) - } + + val startTimestampOpt: Option[Instant] = + if (startPos.equalsIgnoreCase(InitialPosition.AT_TIMESTAMP.name())) { + getFromEither( + pfx(), + Seq( + "starting.timestamp", + "starting.ts", + "start.timestamp", + "start.ts" + ), + config.getStringOpt + ).map { ts => + val tsf = getFromEither( + pfx(), + Seq("timestamp.format", "ts.format"), + config.getStringOpt + ).getOrElse(DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT) + val startAt = Try(ts.toDouble) + .map { d => + Try(Instant.ofEpochMilli(Math.floor(d * 1000).toLong)) + } + .getOrElse { + for { + sdf <- Try(new SimpleDateFormat(tsf)) + instant <- Try(sdf.parse(ts).toInstant) + } yield instant + } + startAt.fold( + t => + throw new RuntimeException( + s"Kinesis source $name has invalid starting timestamp value '$ts' or format '$tsf'", + t + ), + instant => { + val epochSeconds = instant.toEpochMilli / 1000d + if (epochSeconds < 0) + throw new RuntimeException( + s"Kinesis source $name has negative starting timestamp value '$epochSeconds'" + ) + properties + .setProperty(STREAM_INITIAL_TIMESTAMP, f"$epochSeconds%.3f") + instant + } + ) + }.orElse { + throw new RuntimeException( + s"Kinesis source $name set starting.position to AT_TIMESTAMP but provided no starting.timestamp" + ) + } + } else None val useEfo: Boolean = getFromEither( @@ -162,7 +222,7 @@ case class KinesisSourceConfig[ADT <: FlinkEvent]( : Either[SourceFunction[E], Source[E, _ <: SourceSplit, _]] = Left( new FlinkKinesisConsumer[E]( - stream, + streams.asJava, getDeserializationSchema, properties ) diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/RabbitMQSourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/RabbitMQSourceConfig.scala index b8c5d9d4..44625743 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/source/RabbitMQSourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/RabbitMQSourceConfig.scala @@ -1,11 +1,19 @@ package io.epiphanous.flinkrunner.model.source -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkConnectorName, FlinkEvent, RabbitMQConnectionInfo} +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkConnectorName, + FlinkEvent, + RabbitMQConnectionInfo +} import io.epiphanous.flinkrunner.serde.JsonRMQDeserializationSchema import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.connector.source.{Source, SourceSplit} import org.apache.flink.streaming.api.functions.source.SourceFunction -import org.apache.flink.streaming.connectors.rabbitmq.{RMQDeserializationSchema, RMQSource} +import org.apache.flink.streaming.connectors.rabbitmq.{ + RMQDeserializationSchema, + RMQSource +} /** Source configuration for Rabbit MQ. * @param name diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/SocketSourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/SocketSourceConfig.scala index 2fa7300e..59144555 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/source/SocketSourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/SocketSourceConfig.scala @@ -1,9 +1,22 @@ package io.epiphanous.flinkrunner.model.source -import io.epiphanous.flinkrunner.model.{FlinkConfig, FlinkConnectorName, FlinkEvent, StreamFormatName} -import io.epiphanous.flinkrunner.serde.{DelimitedConfig, DelimitedRowDecoder, JsonRowDecoder, RowDecoder} +import io.epiphanous.flinkrunner.model.{ + FlinkConfig, + FlinkConnectorName, + FlinkEvent, + StreamFormatName +} +import io.epiphanous.flinkrunner.serde.{ + DelimitedConfig, + DelimitedRowDecoder, + JsonRowDecoder, + RowDecoder +} import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} /** A socket source configuration. * @param name diff --git a/src/main/scala/io/epiphanous/flinkrunner/model/source/SourceConfig.scala b/src/main/scala/io/epiphanous/flinkrunner/model/source/SourceConfig.scala index 22cb7829..ec1f5884 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/model/source/SourceConfig.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/model/source/SourceConfig.scala @@ -1,22 +1,17 @@ package io.epiphanous.flinkrunner.model.source -import com.google.common.hash.Hashing -import com.typesafe.scalalogging.LazyLogging import io.epiphanous.flinkrunner.model.FlinkConnectorName._ import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.util.BoundedLatenessWatermarkStrategy -import io.epiphanous.flinkrunner.util.StreamUtils._ import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.eventtime.WatermarkStrategy import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.connector.source.{Source, SourceSplit} import org.apache.flink.streaming.api.functions.source.SourceFunction import org.apache.flink.streaming.api.scala._ +import org.apache.flink.table.data.RowData -import java.nio.charset.StandardCharsets import java.time.Duration -import java.util -import java.util.Properties import scala.util.Try /** A flinkrunner source configuration trait. All source configs have a few @@ -51,40 +46,9 @@ import scala.util.Try * @tparam ADT * flinkrunner algebraic data type */ -trait SourceConfig[ADT <: FlinkEvent] extends LazyLogging { - def name: String +trait SourceConfig[ADT <: FlinkEvent] extends SourceOrSinkConfig[ADT] { - def config: FlinkConfig - - def connector: FlinkConnectorName - - lazy val label: String = - s"${config.jobName.toLowerCase}/${connector.entryName.toLowerCase}/$name" - - lazy val stdUid: String = Hashing - .sha256() - .hashString( - label, - StandardCharsets.UTF_8 - ) - .toString - - lazy val uid: String = config.getStringOpt(pfx("uid")).getOrElse(stdUid) - - lazy val parallelism: Int = config - .getIntOpt(pfx("parallelism")) - .getOrElse(config.globalParallelism) - - def pfx(path: String = ""): String = Seq( - Some("sources"), - Some(name), - if (path.isEmpty) None else Some(path) - ).flatten.mkString(".") - - val properties: Properties = config.getProperties(pfx("config")) - - lazy val propertiesMap: util.HashMap[String, String] = - properties.asJavaMap + override val _sourceOrSink = "source" val watermarkStrategy: String = Try(config.getString(pfx("watermark.strategy"))) @@ -268,6 +232,25 @@ trait SourceConfig[ADT <: FlinkEvent] extends LazyLogging { env: StreamExecutionEnvironment)(implicit fromKV: EmbeddedAvroRecordInfo[A] => E): DataStream[E] = getAvroSourceStreamDefault[E, A](env) + + def getRowSource(env: StreamExecutionEnvironment): DataStream[RowData] = + ??? + + def getRowSourceStreamDefault[ + E <: ADT with EmbeddedRowType: TypeInformation]( + env: StreamExecutionEnvironment)(implicit + fromRowData: RowData => E): DataStream[E] = + getRowSource(env) + .map(fromRowData) + .assignTimestampsAndWatermarks(getWatermarkStrategy[E]) + .name(label) + .uid(label) + .setParallelism(parallelism) + + def getRowSourceStream[E <: ADT with EmbeddedRowType: TypeInformation]( + env: StreamExecutionEnvironment)(implicit + fromRowData: RowData => E): DataStream[E] = + getRowSourceStreamDefault[E](env) } object SourceConfig { @@ -282,12 +265,14 @@ object SourceConfig { config.jobName, config.getStringOpt(s"sources.$name.connector") ) match { + case Empty => EmptySourceConfig[ADT](name, config) case File => FileSourceConfig[ADT](name, config) case Hybrid => HybridSourceConfig[ADT](name, config) case Kafka => KafkaSourceConfig[ADT](name, config) case Kinesis => KinesisSourceConfig[ADT](name, config) case RabbitMQ => RabbitMQSourceConfig[ADT](name, config) case Socket => SocketSourceConfig[ADT](name, config) + case Iceberg => IcebergSourceConfig(name, config) case Generator => generatorFactoryOpt .map(factory => diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/AvroJsonSerializer.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/AvroJsonSerializer.scala index 32be1d1a..c1d49dba 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/AvroJsonSerializer.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/AvroJsonSerializer.scala @@ -6,7 +6,11 @@ import com.fasterxml.jackson.databind.{JsonSerializer, SerializerProvider} import com.typesafe.scalalogging.LazyLogging import org.apache.avro.Schema import org.apache.avro.Schema.Type._ -import org.apache.avro.generic.{GenericEnumSymbol, GenericFixed, GenericRecord} +import org.apache.avro.generic.{ + GenericEnumSymbol, + GenericFixed, + GenericRecord +} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.createTypeInformation diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala index 56d6807a..ba6ea1eb 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchema.scala @@ -1,6 +1,14 @@ package io.epiphanous.flinkrunner.serde import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.{ + CachedSchemaRegistryClient, + SchemaRegistryClient +} +import io.confluent.kafka.serializers.{ + KafkaAvroDeserializer, + KafkaAvroDeserializerConfig +} import io.epiphanous.flinkrunner.model.KafkaInfoHeader._ import io.epiphanous.flinkrunner.model.source.KafkaSourceConfig import io.epiphanous.flinkrunner.model.{ @@ -8,21 +16,17 @@ import io.epiphanous.flinkrunner.model.{ EmbeddedAvroRecordInfo, FlinkEvent } -import io.epiphanous.flinkrunner.util.AvroUtils.{ - isSpecific, - schemaOf, - toEmbeddedAvroInstance -} +import io.epiphanous.flinkrunner.util.AvroUtils.isSpecific import org.apache.avro.generic.GenericRecord 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.RegistryAvroDeserializationSchema -import org.apache.flink.formats.avro.registry.confluent.ConfluentRegistryAvroDeserializationSchema import org.apache.flink.util.Collector import org.apache.kafka.clients.consumer.ConsumerRecord import java.nio.charset.StandardCharsets +import java.util import scala.collection.JavaConverters._ +import scala.util.Try /** A deserialization schema that uses a confluent schema registry to * deserialize a kafka key/value pair into instances of a flink runner ADT @@ -31,6 +35,8 @@ import scala.collection.JavaConverters._ * config for the kafka source * @param schemaOpt * optional avro schema string, which is required if A is GenericRecord + * @param schemaRegistryClientOpt + * optional schema registry client (useful for testing) * @tparam E * event type being deserialized, with an embedded avro record * @tparam A @@ -44,31 +50,80 @@ class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ ADT <: FlinkEvent ]( sourceConfig: KafkaSourceConfig[ADT], - schemaOpt: Option[String] = None + schemaOpt: Option[String] = None, + schemaRegistryClientOpt: Option[SchemaRegistryClient] = None )(implicit fromKV: EmbeddedAvroRecordInfo[A] => E) extends KafkaRecordDeserializationSchema[E] with LazyLogging { - val avroClass: Class[A] = implicitly[TypeInformation[A]].getTypeClass + val avroClass: Class[A] = implicitly[TypeInformation[A]].getTypeClass + val avroClassIsSpecific: Boolean = isSpecific(avroClass) require( - isSpecific(avroClass) || schemaOpt.nonEmpty, + avroClassIsSpecific || schemaOpt.nonEmpty, s"You must provide an avro record schema in the configuration of source `${sourceConfig.name}`" + " if you want to deserialize into a generic record type" ) - @transient lazy val deserializer - : RegistryAvroDeserializationSchema[GenericRecord] = - ConfluentRegistryAvroDeserializationSchema.forGeneric( - schemaOf(avroClass, schemaOpt), - sourceConfig.schemaRegistryConfig.url, + @transient + lazy val schemaRegistryClient: SchemaRegistryClient = + schemaRegistryClientOpt.getOrElse( + new CachedSchemaRegistryClient( + sourceConfig.schemaRegistryConfig.url, + sourceConfig.schemaRegistryConfig.cacheCapacity, + sourceConfig.schemaRegistryConfig.props, + sourceConfig.schemaRegistryConfig.headers + ) + ) + + @transient + lazy val deserializerProps: util.Map[String, String] = + new KafkaAvroDeserializerConfig( sourceConfig.schemaRegistryConfig.props + ).originalsStrings() + + /** The deserializer used for keys. + * + * Note: This forces keys to be deserialized as generic records and then + * will call their toString() method to convert them to a string, since + * we require all keys to be strings. + */ + @transient + lazy val keyDeserializer: KafkaAvroDeserializer = { + val kkad = new KafkaAvroDeserializer(schemaRegistryClient) + val p = new util.HashMap[String, String]() + p.putAll(deserializerProps) + // NOTE: we deserialize keys as generic records and then toString the result + p.put(KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG, "false") + kkad.configure(p, true) + kkad + } + + /** The deserializer used for values. + * + * Note: This forces values to be deserialized as specific records if A + * is a specific record class (and forces deserialization as generic + * records if A == GenericRecord) + */ + @transient + lazy val valueDeserializer: KafkaAvroDeserializer = { + val vkad = new KafkaAvroDeserializer(schemaRegistryClient) + val p = new util.HashMap[String, String]() + p.putAll(deserializerProps) + // ensure we use the right setting for specific + p.put( + KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG, + if (avroClassIsSpecific) "true" else "false" ) + vkad.configure(p, false) + vkad + } override def deserialize( record: ConsumerRecord[Array[Byte], Array[Byte]], out: Collector[E]): Unit = { + /** extract headers */ val headers = Option(record.headers()) .map(_.asScala.map { h => (h.key(), new String(h.value(), StandardCharsets.UTF_8)) @@ -89,27 +144,56 @@ class ConfluentAvroRegistryKafkaRecordDeserializationSchema[ headerName(Topic) -> record.topic() ) - val key = Option(record.key()).map(keyBytes => - new String(keyBytes, StandardCharsets.UTF_8) - ) - - deserializer.deserialize(record.value()) match { - case a: GenericRecord => - out.collect( - toEmbeddedAvroInstance[E, A, ADT]( - a, - avroClass, - sourceConfig.config, - key, - headers - ) + // deserialize the key + val keyOpt = + Try( + keyDeserializer.deserialize( + record.topic(), + record.key() ) - case c if Option(c).nonEmpty => - throw new RuntimeException( - s"deserialized value is an unexpected type of object: $c" + ) + .fold( + error => { + logger.error( + s"failed to deserialize kafka message key (${record + .serializedKeySize()} bytes) from topic ${record.topic()}", + error + ) + None + }, + k => + Some( + k.toString + ) // if result not already a string, convert it to a string ) - case _ => logger.trace(s"ignoring tombstone record $record") - } + + Try( + valueDeserializer.deserialize(record.topic(), record.value()) + ) + .fold( + error => + logger.error( + s"failed to deserialize kafka message value (${record + .serializedValueSize()} bytes) to a ${avroClass.getCanonicalName}", + error + ), + { + case rec: A => + val event = fromKV( + EmbeddedAvroRecordInfo( + rec, + sourceConfig.config, + keyOpt, + headers + ) + ) + out.collect(event) + case unexpected => + logger.error( + s"Expected deserialized kafka message value of type ${avroClass.getCanonicalName}, but got [$unexpected]" + ) + } + ) } override def getProducedType: TypeInformation[E] = diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala index e7476c6f..d0cebaff 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchema.scala @@ -1,14 +1,16 @@ package io.epiphanous.flinkrunner.serde -import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.typesafe.scalalogging.LazyLogging +import io.confluent.kafka.schemaregistry.client.{ + CachedSchemaRegistryClient, + SchemaRegistryClient +} +import io.confluent.kafka.serializers.KafkaAvroSerializer import io.epiphanous.flinkrunner.model.sink.KafkaSinkConfig import io.epiphanous.flinkrunner.model.{EmbeddedAvroRecord, FlinkEvent} import io.epiphanous.flinkrunner.util.SinkDestinationNameUtils.RichSinkDestinationName -import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord 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 org.apache.kafka.common.header.internals.RecordHeaders @@ -26,43 +28,27 @@ case class ConfluentAvroRegistryKafkaRecordSerializationSchema[ A <: GenericRecord, ADT <: FlinkEvent ]( - sinkConfig: KafkaSinkConfig[ADT] + sinkConfig: KafkaSinkConfig[ADT], + schemaRegistryClientOpt: Option[SchemaRegistryClient] = None ) extends KafkaRecordSerializationSchema[E] with LazyLogging { - @transient lazy val serializerCacheLoader - : CacheLoader[Schema, ConfluentRegistryAvroSerializationSchema[ - GenericRecord - ]] = - new CacheLoader[Schema, ConfluentRegistryAvroSerializationSchema[ - GenericRecord - ]] { - override def load(schema: Schema) - : ConfluentRegistryAvroSerializationSchema[GenericRecord] = - ConfluentRegistryAvroSerializationSchema - .forGeneric( - s"${schema.getFullName}-value", - schema, - sinkConfig.schemaRegistryConfig.url, - sinkConfig.schemaRegistryConfig.props - ) - - } + @transient + lazy val schemaRegistryClient: SchemaRegistryClient = + schemaRegistryClientOpt.getOrElse( + new CachedSchemaRegistryClient( + sinkConfig.schemaRegistryConfig.url, + sinkConfig.schemaRegistryConfig.cacheCapacity, + sinkConfig.schemaRegistryConfig.props, + sinkConfig.schemaRegistryConfig.headers + ) + ) - @transient lazy val serializerCache - : LoadingCache[Schema, ConfluentRegistryAvroSerializationSchema[ - GenericRecord - ]] = { - val cacheBuilder = CacheBuilder - .newBuilder() - .concurrencyLevel(sinkConfig.cacheConcurrencyLevel) - .maximumSize(sinkConfig.cacheMaxSize) - .expireAfterWrite(sinkConfig.cacheExpireAfter) - if (sinkConfig.cacheRecordStats) cacheBuilder.recordStats() - cacheBuilder.build[Schema, ConfluentRegistryAvroSerializationSchema[ - GenericRecord - ]](serializerCacheLoader) - } + @transient + lazy val serializer: KafkaAvroSerializer = new KafkaAvroSerializer( + schemaRegistryClient, + sinkConfig.schemaRegistryConfig.props + ) override def serialize( element: E, @@ -80,14 +66,14 @@ case class ConfluentAvroRegistryKafkaRecordSerializationSchema[ val topic = sinkConfig.expandTemplate(info.record) - val key = info.keyOpt.map(_.getBytes(StandardCharsets.UTF_8)) + val key = info.keyOpt.map(k => serializer.serialize(topic, k)) + logger.trace( - s"serializing ${info.record.getSchema.getFullName} record ${element.$id} to $topic ${if (sinkConfig.isKeyed) "with key" + s"serializing ${info.record.getSchema.getFullName} record ${element.$id} to topic <$topic> ${if (key.nonEmpty) "with key" else "without key"}, headers=${info.headers}" ) - val value = - serializerCache.get(info.record.getSchema).serialize(info.record) + val value = serializer.serialize(topic, info.record) new ProducerRecord( topic, diff --git a/src/main/scala/io/epiphanous/flinkrunner/serde/JsonKinesisSerializationSchema.scala b/src/main/scala/io/epiphanous/flinkrunner/serde/JsonKinesisSerializationSchema.scala index 7bbac216..1ff7135c 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/serde/JsonKinesisSerializationSchema.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/serde/JsonKinesisSerializationSchema.scala @@ -19,5 +19,5 @@ class JsonKinesisSerializationSchema[ ByteBuffer.wrap(jsonSerializationSchema.serialize(element)) override def getTargetStream(element: E): String = - kinesisSinkConfig.stream + kinesisSinkConfig.props.stream } diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/AttributesOf.scala b/src/main/scala/io/epiphanous/flinkrunner/util/AttributesOf.scala new file mode 100644 index 00000000..84a5fc8c --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/AttributesOf.scala @@ -0,0 +1,33 @@ +package io.epiphanous.flinkrunner.util + +import shapeless._ +import shapeless.ops.hlist +import shapeless.ops.record._ + +trait AttributesOf[T] { + def names: List[String] +} + +object AttributesOf { + implicit def toAttributes[T, Repr <: HList, KeysRepr <: HList](implicit + gen: LabelledGeneric.Aux[T, Repr], + keys: Keys.Aux[Repr, KeysRepr], + traversable: hlist.ToTraversable.Aux[KeysRepr, List, Symbol]) + : AttributesOf[T] = + new AttributesOf[T] { + override def names: List[String] = keys().toList.map(_.name) + } + + def apply[T](implicit attributesOf: AttributesOf[T]): AttributesOf[T] = + attributesOf + +} + +object Fields { + def namesOf[A <: Product](a: A)(implicit + attributesOf: AttributesOf[A]): List[String] = attributesOf.names + + def of[A <: Product](a: A)(implicit + attributesOf: AttributesOf[A]): List[(String, Any)] = + namesOf(a) zip a.productIterator.toList +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/AvroUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/AvroUtils.scala index ff9b512f..e84dbd95 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/AvroUtils.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/AvroUtils.scala @@ -1,7 +1,12 @@ package io.epiphanous.flinkrunner.util import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.model.{EmbeddedAvroRecord, EmbeddedAvroRecordInfo, FlinkConfig, FlinkEvent} +import io.epiphanous.flinkrunner.model.{ + EmbeddedAvroRecord, + EmbeddedAvroRecordInfo, + FlinkConfig, + FlinkEvent +} import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificRecordBase @@ -27,7 +32,7 @@ object AvroUtils extends LazyLogging { def schemaOf[A <: GenericRecord]( typeClass: Class[A], - schemaStringOpt: Option[String]): Schema = + schemaStringOpt: Option[String] = None): Schema = if (isSpecific(typeClass)) instanceOf(typeClass).getSchema else schemaStringOpt diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/InstantUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/InstantUtils.scala index 080dcae6..3b08cd04 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/InstantUtils.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/InstantUtils.scala @@ -1,7 +1,7 @@ package io.epiphanous.flinkrunner.util -import java.time.{Instant, ZoneOffset} import java.time.format.DateTimeFormatter +import java.time.{Instant, ZoneOffset} object InstantUtils { diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/MetricUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/MetricUtils.scala new file mode 100644 index 00000000..a9d9cccc --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/MetricUtils.scala @@ -0,0 +1,152 @@ +package io.epiphanous.flinkrunner.util + +import com.codahale.metrics.ExponentiallyDecayingReservoir +import com.typesafe.scalalogging.LazyLogging +import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper +import org.apache.flink.metrics._ + +object MetricUtils { + + def unregisteredMetricMessage( + metricType: String, + key: String, + value: String): String = + s"failed to update unregistered $metricType metric (key=$key,value=$value)" + + implicit class RichVariableCounters( + counters: Map[String, Map[String, Counter]]) + extends LazyLogging { + def inc(key: String, value: String, num: Long = 1): Unit = + if (counters.contains(key) && counters(key).contains(value)) + counters(key)(value).inc(num) + else logger.warn(unregisteredMetricMessage("counter", key, value)) + + def dec(key: String, value: String, num: Long = 1): Unit = + if (counters.contains(key) && counters(key).contains(value)) + counters(key)(value).dec(num) + else logger.warn(unregisteredMetricMessage("counter", key, value)) + } + + implicit class RichVariableMeters( + meters: Map[String, Map[String, Meter]]) + extends LazyLogging { + def markEvent(key: String, value: String, num: Long = 1): Unit = + if (meters.contains(key) && meters(key).contains(value)) + meters(key)(value).markEvent(num) + else logger.warn(unregisteredMetricMessage("meter", key, value)) + } + + implicit class RichVariableHistograms( + histograms: Map[String, Map[String, DropwizardHistogramWrapper]]) + extends LazyLogging { + def update(key: String, value: String, updatedValue: Long): Unit = + if (histograms.contains(key) && histograms(key).contains(value)) + histograms(key)(value).update(updatedValue) + else + logger.warn(unregisteredMetricMessage("histogram", key, value)) + } + + def defaultHistogramFactory() = new com.codahale.metrics.Histogram( + new ExponentiallyDecayingReservoir() + ) + + implicit class RichMetricGroup(mg: MetricGroup) { + + def registerMetricByVariables[M]( + name: String, + variables: Map[String, List[String]], + func: MetricGroup => M): Map[String, Map[String, M]] = + variables.map { case (key, values) => + key -> values + .map(value => value -> func(mg.addGroup(key, value))) + .toMap + } + + def registerCounter(counterName: String): Counter = + mg.counter(counterName) + + def registerCounterByVariables( + counterName: String, + variables: Map[String, List[String]]) + : Map[String, Map[String, Counter]] = + registerMetricByVariables( + counterName, + variables, + _.registerCounter(counterName) + ) + + /** Registers a meter and an associated counter in this metric group. + * The name of the meter is derived from the provided counter name. + * + * @param counterName + * name of the counter + * @param span + * the number of seconds over which counts are aggregated to compute + * the rate (defaults to 60 seconds) + * @return + * Meter + */ + def registerMeter(counterName: String, span: Int = 60): Meter = { + val rateName = s"$counterName-per-second" + mg.meter( + rateName, + new MeterView(mg.counter(counterName), span) + ) + } + + /** Register a set of meters, one for each pair of label and associated + * values provided in the labels parameter. + * @param counterName + * name of the counter within the meter + * @param variables + * map of labels, with a list of values for each + * @param span + * number of seconds over which counts are aggregated to compute the + * rate in the meters (defaults to 60 seconds) + * @return + * Map[String, Map[String, Meter`]``]` + */ + def registerMeterByVariables( + counterName: String, + variables: Map[String, List[String]], + span: Int = 60): Map[String, Map[String, Meter]] = + registerMetricByVariables( + counterName, + variables, + _.registerMeter(counterName, span) + ) + + def registerGauge[T](name: String, gauge: Gauge[T]): Gauge[T] = + mg.gauge[T, Gauge[T]](name, gauge) + + def registerGaugeByVariables[T]( + name: String, + gauge: Gauge[T], + variables: Map[String, List[String]]) + : Map[String, Map[String, Gauge[T]]] = + registerMetricByVariables( + name, + variables, + _.registerGauge[T](name, gauge) + ) + + def registerHistogram( + name: String, + histogram: com.codahale.metrics.Histogram = + defaultHistogramFactory()): DropwizardHistogramWrapper = + mg.histogram(name, new DropwizardHistogramWrapper(histogram)) + + def registerHistogramByVariables( + name: String, + variables: Map[String, List[String]], + histogramFactory: () => com.codahale.metrics.Histogram = + defaultHistogramFactory) + : Map[String, Map[String, DropwizardHistogramWrapper]] = + registerMetricByVariables( + name, + variables, + _.registerHistogram(name, histogramFactory()) + ) + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/RichString.scala b/src/main/scala/io/epiphanous/flinkrunner/util/RichString.scala new file mode 100644 index 00000000..2c24fc90 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/RichString.scala @@ -0,0 +1,38 @@ +package com.mdsol.streaming.util + +import java.util.regex.Pattern + +object RichString { + + private val camelCasePattern1: Pattern = + Pattern.compile("([A-Z]+)([A-Z][a-z])") + private val camelCasePattern2: Pattern = + Pattern.compile("([a-z\\d])([A-Z])") + private val camelCaseReplacement: String = "$1_$2" + + def splitCamel(string: String): Array[String] = { + val first = + camelCasePattern1.matcher(string).replaceAll(camelCaseReplacement) + camelCasePattern2 + .matcher(first) + .replaceAll(camelCaseReplacement) + .split("_") + } + + def camel2XCase(string: String, sep: String): String = + splitCamel(string).mkString(sep).toLowerCase + + implicit class RichString(string: String) { + + def dotCase: String = camel2XCase(string, ".").toLowerCase + def upperDotCase: String = dotCase.toUpperCase + + def snakeCase: String = camel2XCase(string, "_").toLowerCase + def upperSnakeCase: String = snakeCase.toUpperCase + + def kebabCase: String = camel2XCase(string, "-").toLowerCase + def upperKebabCase: String = kebabCase.toUpperCase + + } + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/RowUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/RowUtils.scala new file mode 100644 index 00000000..8be7b42d --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/RowUtils.scala @@ -0,0 +1,215 @@ +package io.epiphanous.flinkrunner.util + +import com.typesafe.scalalogging.LazyLogging +import io.epiphanous.flinkrunner.model.{DataTypeConfig, FlinkEvent} +import org.apache.avro.Schema +import org.apache.avro.generic.GenericRecord +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter +import org.apache.flink.table.annotation.DataTypeHint +import org.apache.flink.table.api.DataTypes +import org.apache.flink.table.types.DataType +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.types.logical.utils.LogicalTypeParser + +import scala.collection.JavaConverters._ +import scala.reflect.runtime.{universe => ru} +import scala.util.{Failure, Success, Try} + +object RowUtils extends LazyLogging { + + /** Infer a RowType for a FlinkEvent of type E. The algorithm here + * leverages scala runtime reflection, so you need to make sure a + * TypeTag is available implicitly. Usually, you simply need to add a + * TypeTag bound on E where you declare your job. + * + * If the event type is an EmbeddedAvroRecord, it will extract the avro + * schema from the type definition and create a RowType from the avro + * schema. Otherwise, it will reflect on the class' primary constructor + * and define a schema for each field in its first param list. + * @tparam E + * the flink event type + * @return + * RowType + */ + def rowTypeOf[E <: FlinkEvent: ru.TypeTag]: RowType = { + val t = ru.typeOf[E] + t.decls + .find(d => + d.isMethod && d.asMethod.name.toString.contentEquals("$record") + ) + .flatMap { k => + schemaOf(k.asMethod.returnType.typeSymbol.asClass) match { + case Success(schema) => Some(schema) + case Failure(ex) => + logger.error("failed to get schema", ex) + None + } + } + .map(schema => + AvroSchemaConverter + .convertToDataType(schema.toString) + ) + .getOrElse( + getRowTypeOf(t).notNull() + ) + .getLogicalType + .asInstanceOf[RowType] + } + + private def schemaOf(klass: ru.ClassSymbol): Try[Schema] = Try { + val cm = scala.reflect.runtime.currentMirror.reflectClass(klass) + // find zero-arg constructor + val constructor = + cm.symbol.info.decls + .find(p => + p.isMethod && { + val m = p.asMethod + m.isConstructor && m.paramLists.flatten.isEmpty + } + ) + .get + .asMethod + val constructorMethod = cm.reflectConstructor(constructor) + constructorMethod().asInstanceOf[GenericRecord].getSchema + } + + private def primaryConstructor(scalaType: ru.Type): Seq[ru.Symbol] = { + scalaType.decls + .find(d => d.isMethod && d.asMethod.isPrimaryConstructor) + .get + .typeSignature + .paramLists + .head + } + + private def getRowTypeOf(scalaType: ru.Type): DataType = { + val fields = primaryConstructor(scalaType) + .map(f => + DataTypes.FIELD( + f.name.toString, + fieldType( + f.typeSignature, + f.annotations + .find(a => a.tree.tpe <:< ru.typeOf[DataTypeHint]) + .map(parseDataTypeConfig) + .getOrElse(DataTypeConfig()) + ) + ) + ) + .asJava + DataTypes.ROW(fields) + } + + private def fieldType( + scalaType: ru.Type, + dataTypeConfig: DataTypeConfig): DataType = { + dataTypeConfig.value + .map(t => + DataTypes.of(LogicalTypeParser.parse(t, getClass.getClassLoader)) + ) + .getOrElse { + scalaType.typeSymbol.fullName match { + case "java.lang.String" => DataTypes.STRING().notNull() + case "scala.Boolean" | "java.lang.Boolean" => + DataTypes.BOOLEAN().notNull() + case "scala.Byte" | "java.lang.Byte" => + DataTypes.TINYINT().notNull() + case "scala.Short" | "java.lang.Short" => + DataTypes.SMALLINT().notNull() + case "scala.Int" | "java.lang.Integer" => + DataTypes.INT().notNull() + case "scala.Long" | "java.lang.Long" => + DataTypes.BIGINT().notNull() + case "scala.BigInt" | "java.math.BigInteger" => + DataTypes.DECIMAL( + dataTypeConfig.defaultDecimalPrecision.getOrElse(10), + dataTypeConfig.defaultDecimalScale.getOrElse(0) + ) + case "scala.Float" | "java.lang.Float" => + DataTypes.FLOAT().notNull() + case "scala.Double" => DataTypes.DOUBLE().notNull() + case "scala.BigDecimal" | "java.math.BigDecimal" => + DataTypes.DECIMAL( + dataTypeConfig.defaultDecimalPrecision.getOrElse(10), + dataTypeConfig.defaultDecimalScale.getOrElse(0) + ) + case "java.time.Instant" | "java.sql.Timestamp" => + dataTypeConfig.defaultSecondPrecision + .map(p => DataTypes.TIMESTAMP_LTZ(p)) + .getOrElse(DataTypes.TIMESTAMP_LTZ()) + .notNull() + case "java.time.LocalDateTime" => + dataTypeConfig.defaultSecondPrecision + .map(p => DataTypes.TIMESTAMP(p)) + .getOrElse(DataTypes.TIMESTAMP()) + .notNull() + case "java.time.OffsetDateTime" | "java.time.ZonedDateTime" => + dataTypeConfig.defaultSecondPrecision + .map(p => DataTypes.TIMESTAMP_WITH_TIME_ZONE(p)) + .getOrElse(DataTypes.TIMESTAMP_WITH_TIME_ZONE()) + .notNull() + case "java.time.LocalDate" | "java.sql.Date" => + DataTypes.DATE().notNull() + case "java.time.LocalTime" | "java.sql.Time" => + dataTypeConfig.defaultSecondPrecision + .map(p => DataTypes.TIME(p)) + .getOrElse(DataTypes.TIME()) + .notNull() + case "scala.Option" => + fieldType(scalaType.typeArgs.head, dataTypeConfig).nullable() + case "java.nio.ByteBuffer" => DataTypes.BYTES().notNull() + case "scala.Array" + if scalaType.typeArgs.head.typeSymbol.fullName.contentEquals( + "scala.Byte" + ) => + DataTypes.BYTES().notNull() + case _ if isArray(scalaType) => + DataTypes.ARRAY( + fieldType(scalaType.typeArgs.head, dataTypeConfig) + ) + case _ if isMap(scalaType) => + DataTypes.MAP( + fieldType(scalaType.typeArgs.head, dataTypeConfig), + fieldType(scalaType.typeArgs.last, dataTypeConfig) + ) + case _ if isRow(scalaType) => + getRowTypeOf(scalaType) + } + } + } + + private def parseDataTypeConfig(a: ru.Annotation): DataTypeConfig = + a.tree.children.tail.foldLeft(DataTypeConfig()) { + case ( + dtc, + arg @ ru.AssignOrNamedArg(ru.Ident(name), ru.Literal(lit)) + ) => + val prop = name.toString + (prop, lit) match { + case ("value", ru.Constant(v: String)) => + dtc.copy(value = Some(v)) + case ("defaultDecimalPrecision", ru.Constant(dp: Int)) => + dtc.copy(defaultDecimalPrecision = Some(dp)) + case ("defaultDecimalScale", ru.Constant(ds: Int)) => + dtc.copy(defaultDecimalPrecision = Some(ds)) + case ("defaultSecondPrecision", ru.Constant(sp: Int)) => + dtc.copy(defaultSecondPrecision = Some(sp)) + case ("defaultYearPrecision", ru.Constant(yp: Int)) => + dtc.copy(defaultSecondPrecision = Some(yp)) + case ("bridgedTo", ru.Constant(bt: Class[_])) => + dtc.copy(bridgedTo = Some(bt)) + case _ => + throw new RuntimeException(s"unsupported literal type $lit") + } + } + + private def isArray(scalaType: ru.Type): Boolean = ??? + + private def isMap(scalaType: ru.Type): Boolean = ??? + + private def isRow(scalaType: ru.Type): Boolean = ??? + + private def isa(scalaType: ru.Type, parentClass: String): Boolean = + scalaType.baseClasses.map(_.fullName).contains(parentClass) + +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/SinkDestinationNameUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/SinkDestinationNameUtils.scala index becac587..b8773249 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/SinkDestinationNameUtils.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/SinkDestinationNameUtils.scala @@ -67,7 +67,7 @@ object SinkDestinationNameUtils extends Serializable { ) normalized // TODO: get rules to normalize these names too - case s: KinesisSinkConfig[ADT] => s.stream + case s: KinesisSinkConfig[ADT] => s.props.stream case s: FileSinkConfig[ADT] => s.path case s: ElasticsearchSinkConfig[ADT] => s.index case s => s.name diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala b/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala index 066ee570..284e26ca 100644 --- a/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala +++ b/src/main/scala/io/epiphanous/flinkrunner/util/StreamUtils.scala @@ -1,7 +1,6 @@ package io.epiphanous.flinkrunner.util import com.google.common.collect.Maps -import com.typesafe.config.Config import com.typesafe.scalalogging.LazyLogging import java.util @@ -41,8 +40,6 @@ object StreamUtils extends LazyLogging { } } - implicit class RichConfig(val c: Config) {} - implicit class RichProps(val p: Properties) { def asJavaMap: util.HashMap[String, String] = Maps.newHashMap(Maps.fromProperties(p)) diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/test/FlinkRunnerSpec.scala b/src/main/scala/io/epiphanous/flinkrunner/util/test/FlinkRunnerSpec.scala new file mode 100644 index 00000000..26f4a5d3 --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/test/FlinkRunnerSpec.scala @@ -0,0 +1,164 @@ +package io.epiphanous.flinkrunner.util.test + +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.flink.StreamJob +import io.epiphanous.flinkrunner.model._ +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.data.RowData + +import scala.reflect.runtime.{universe => ru} + +trait FlinkRunnerSpec { + val DEFAULT_CONFIG_STR: String = + """ + |sources{empty-source{}} + |sinks{print-sink{}} + |""".stripMargin + + def getRunner[ + IN <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + JF <: StreamJob[OUT, ADT], + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + jobFactory: JobFactory[JF, IN, OUT, ADT], + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true): FlinkRunner[ADT] = { + val config = new FlinkConfig(args, Some(configStr)) + new FlinkRunner[ADT]( + config, + checkResultsOpt = checkResultsOpt, + executeJob = executeJob + ) { + override def invoke(jobName: String): Unit = + jobFactory.getJob(this).run() + } + } + + def getStreamJobRunner[ + IN <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true): FlinkRunner[ADT] = { + getRunner( + configStr, + new StreamJobFactory[IN, OUT, ADT](transformer, input, sourceName), + checkResultsOpt, + args, + executeJob + ) + } + + def getIdentityStreamJobRunner[ + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true): FlinkRunner[ADT] = + getRunner( + configStr, + new IdentityStreamJobFactory(input, sourceName), + checkResultsOpt, + args, + executeJob + ) + + def getAvroStreamJobRunner[ + IN <: ADT with EmbeddedAvroRecord[INA]: TypeInformation, + INA <: GenericRecord: TypeInformation, + OUT <: ADT with EmbeddedAvroRecord[OUTA]: TypeInformation, + OUTA <: GenericRecord: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true)(implicit + fromKV: EmbeddedAvroRecordInfo[INA] => IN): FlinkRunner[ADT] = + getRunner( + configStr, + new AvroStreamJobFactory[IN, INA, OUT, OUTA, ADT]( + transformer, + input, + sourceName + ), + checkResultsOpt, + args, + executeJob + ) + + def getIdentityAvroStreamJobRunner[ + OUT <: ADT with EmbeddedAvroRecord[OUTA]: TypeInformation, + OUTA <: GenericRecord: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true)(implicit + fromKV: EmbeddedAvroRecordInfo[OUTA] => OUT): FlinkRunner[ADT] = + getRunner( + configStr, + new IdentityAvroStreamJobFactory[OUT, OUTA, ADT](input, sourceName), + checkResultsOpt, + args, + executeJob + ) + + def getTableStreamJobRunner[ + IN <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + OUT <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true)(implicit + fromRowData: RowData => IN): FlinkRunner[ADT] = + getRunner( + configStr, + new TableStreamJobFactory[IN, OUT, ADT]( + transformer, + input, + sourceName + ), + checkResultsOpt, + args, + executeJob + ) + + def getIdentityTableStreamJobRunner[ + OUT <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + ADT <: FlinkEvent: TypeInformation]( + configStr: String = DEFAULT_CONFIG_STR, + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None, + checkResultsOpt: Option[CheckResults[ADT]] = None, + args: Array[String] = Array("testJob"), + executeJob: Boolean = true)(implicit + fromRowData: RowData => OUT): FlinkRunner[ADT] = + getRunner( + configStr, + new IdentityTableStreamJobFactory[OUT, ADT](input, sourceName), + checkResultsOpt, + args, + executeJob + ) +} diff --git a/src/main/scala/io/epiphanous/flinkrunner/util/test/JobFactory.scala b/src/main/scala/io/epiphanous/flinkrunner/util/test/JobFactory.scala new file mode 100644 index 00000000..5f6feb6f --- /dev/null +++ b/src/main/scala/io/epiphanous/flinkrunner/util/test/JobFactory.scala @@ -0,0 +1,142 @@ +package io.epiphanous.flinkrunner.util.test + +import io.epiphanous.flinkrunner.FlinkRunner +import io.epiphanous.flinkrunner.flink.{ + AvroStreamJob, + StreamJob, + TableStreamJob +} +import io.epiphanous.flinkrunner.model.{ + EmbeddedAvroRecord, + EmbeddedAvroRecordInfo, + EmbeddedRowType, + FlinkEvent +} +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.data.RowData + +import scala.reflect.runtime.{universe => ru} + +abstract class JobFactory[ + JF <: StreamJob[OUT, ADT], + IN <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None) + extends Serializable { + def getJob(runner: FlinkRunner[ADT]): JF +} + +@SerialVersionUID(1L) +class StreamJobFactory[ + IN <: ADT: TypeInformation, + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None) + extends JobFactory[StreamJob[OUT, ADT], IN, OUT, ADT]( + transformer, + input, + sourceName + ) { + def getJob(runner: FlinkRunner[ADT]): StreamJob[OUT, ADT] = + new StreamJob[OUT, ADT](runner) { + override def transform: DataStream[OUT] = + seqOrSingleSource(input, sourceName).map(transformer) + } +} + +@SerialVersionUID(1L) +class IdentityStreamJobFactory[ + OUT <: ADT: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None) + extends StreamJobFactory[OUT, OUT, ADT]( + new IdentityMap[OUT], + input, + sourceName + ) + +@SerialVersionUID(1L) +class AvroStreamJobFactory[ + IN <: ADT with EmbeddedAvroRecord[INA]: TypeInformation, + INA <: GenericRecord: TypeInformation, + OUT <: ADT with EmbeddedAvroRecord[OUTA]: TypeInformation, + OUTA <: GenericRecord: TypeInformation, + ADT <: FlinkEvent: TypeInformation +]( + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None)(implicit + fromKV: EmbeddedAvroRecordInfo[INA] => IN) + extends JobFactory[AvroStreamJob[OUT, OUTA, ADT], IN, OUT, ADT]( + transformer, + input, + sourceName + ) { + def getJob(runner: FlinkRunner[ADT]): AvroStreamJob[OUT, OUTA, ADT] = + new AvroStreamJob[OUT, OUTA, ADT](runner) { + override def transform: DataStream[OUT] = + seqOrSingleAvroSource[IN, INA](input, sourceName).map(transformer) + } +} + +@SerialVersionUID(1L) +class IdentityAvroStreamJobFactory[ + OUT <: ADT with EmbeddedAvroRecord[OUTA]: TypeInformation, + OUTA <: GenericRecord: TypeInformation, + ADT <: FlinkEvent: TypeInformation]( + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None)(implicit + fromKV: EmbeddedAvroRecordInfo[OUTA] => OUT) + extends AvroStreamJobFactory[OUT, OUTA, OUT, OUTA, ADT]( + new IdentityMap[OUT], + input, + sourceName + ) + +@SerialVersionUID(1L) +class TableStreamJobFactory[ + IN <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + OUT <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + ADT <: FlinkEvent: TypeInformation]( + transformer: MapFunction[IN, OUT], + input: Seq[IN] = Seq.empty, + sourceName: Option[String] = None)(implicit fromRowData: RowData => IN) + extends JobFactory[TableStreamJob[OUT, ADT], IN, OUT, ADT]( + transformer, + input, + sourceName + ) { + def getJob(runner: FlinkRunner[ADT]): TableStreamJob[OUT, ADT] = + new TableStreamJob[OUT, ADT](runner) { + override def transform: DataStream[OUT] = + seqOrSingleRowSource[IN](input, sourceName).map(transformer) + } +} + +@SerialVersionUID(1L) +class IdentityTableStreamJobFactory[ + OUT <: ADT with EmbeddedRowType: TypeInformation: ru.TypeTag, + ADT <: FlinkEvent: TypeInformation]( + input: Seq[OUT] = Seq.empty, + sourceName: Option[String] = None)(implicit + fromRowData: RowData => OUT) + extends TableStreamJobFactory[OUT, OUT, ADT]( + new IdentityMap[OUT], + input, + sourceName + ) + +@SerialVersionUID(1L) +class IdentityMap[A] extends MapFunction[A, A] { + override def map(value: A): A = + value +} diff --git a/src/test/resources/logback.xml b/src/test/resources/logback.xml index 9ab0f472..ae3b2b40 100644 --- a/src/test/resources/logback.xml +++ b/src/test/resources/logback.xml @@ -35,5 +35,10 @@ + + + + + diff --git a/src/test/scala/io/epiphanous/flinkrunner/BaseSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/BaseSpec.scala index 3c93ea17..30ba2e5d 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/BaseSpec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/BaseSpec.scala @@ -1,11 +1,7 @@ package io.epiphanous.flinkrunner import com.typesafe.scalalogging.LazyLogging -import io.epiphanous.flinkrunner.flink.{AvroStreamJob, StreamJob} -import io.epiphanous.flinkrunner.model._ -import org.apache.avro.generic.GenericRecord -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.createTypeInformation +import io.epiphanous.flinkrunner.util.test.FlinkRunnerSpec import org.scalatest.matchers.should.Matchers import org.scalatest.{EitherValues, Inside, OptionValues, TryValues} @@ -15,166 +11,5 @@ trait BaseSpec with EitherValues with TryValues with Inside - with LazyLogging { - - def getRunner[ADT <: FlinkEvent: TypeInformation]( - args: Array[String] = Array.empty, - optConfig: Option[String] = None): FlinkRunner[ADT] = { - val config = new FlinkConfig(args, optConfig) - new FlinkRunner[ADT](config) { - override def invoke(jobName: String): Unit = ??? - } - } - - /** Create a test job using the [[StreamJob]] base class. The code - * automatically arranges for the configuration's mock.edges - * setting to be set to true, which is required to route the output of - * the job to the test function mockSink. - * @param args - * Simulated command line arguments for invoking the job. - * @param configStr - * Simulated configuration, which together with args will be - * used to construct the flink runner's [[FlinkConfig]]. - * @param mockSources - * A map of collection based input arrays of type ADT. - * @param mockSink - * The output stream of the job will be directed to this function, - * which takes a [[List]] [ADT] as input and has a [[Unit]] return - * type. You should include your tests of the job output in this - * function. - * @param getJob - * A method to construct the job instance to be tested. This takes a - * string jobName as well as the constructed [[FlinkRunner]] object as - * arguments and must return an instance of the job to be tested. - * @tparam JOB - * The type of the job instance, which in this case is a subclass of - * [[StreamJob]]. - * @tparam OUT - * The output type of the job, which must extend the ADT. - * @tparam ADT - * The algebraic data type of the job. - * @return - * An instance of [[FlinkRunner][[[ADT]] ]. You can run the job by - * invoking this instance's run method. - */ - def getJobRunner[ - JOB <: StreamJob[OUT, ADT], - OUT <: ADT, - ADT <: FlinkEvent: TypeInformation]( - args: Array[String], - configStr: String, - checkResults: CheckResults[ADT], - getJob: (String, FlinkRunner[ADT]) => JOB): FlinkRunner[ADT] = { - val config = new FlinkConfig( - args, - Some(configStr + "\nenvironment=dev\nmock.edges=true\n") - ) - new FlinkRunner[ADT](config, Some(checkResults)) { - override def invoke(jobName: String): Unit = - getJob(jobName, this).run() - } - } - - /** Create a test job using the [[AvroStreamJob]] base class. The code - * automatically arranges for the configuration's mock.edges - * setting to be set to true, which is required to route the output of - * the job to the test function mockSink. - * - * Here's an example test job: - * {{{ - * - * property("singleAvroSource property") { - * val cfg = """ - * |sources { - * | test-single { - * | connector = collector - * |} - * |jobs { - * | SingleAvroSourceJob { - * | source = test - * | } - * |} - * |""".stripMargin - * val src = Map("test-single" -> genPop[AWrapper](3)) - * val getJob = (_: String, r: FlinkRunner[MyAvroADT]) => new - * SingleAvroSourceJob(r) - * - * // this contains the tests - * - * val mockSink = { r: List[MyAvroADT] => - * val result = r.map(_.asInstanceOf[AWrapper]) val twoDays = 2 * 86400000 - * val orig = src("test-single") result.head.$record.a0 shouldEqual - * orig.head.$record.a0 * 2 result.head.$record.a3 shouldEqual - * Instant.ofEpochMilli( orig.head.$record.a3.toEpochMilli + twoDays ) - * println(orig.head)* - * println(result.head) - * } - * - * // this creates and runs the job - * getAvroJobRunner[SingleAvroSourceJob, AWrapper, ARecord, MyAvroADT]( - * Array("SingleAvroSourceJob"), - * cfg, - * src, - * mockSink, - * getJob - * ).process() - * } - * - * }}} - * - * @param args - * Simulated command line arguments for invoking the job. - * @param configStr - * Simulated configuration, which together with args will be - * used to construct the flink runner's [[FlinkConfig]]. - * @param mockSources - * A map of collection based input arrays of type ADT. - * @param mockSink - * The output stream of the job will be directed to this function, - * which takes a [[List]] [ADT] as input and has a [[Unit]] return - * type. You should include your tests of the job output in this - * function. - * @param getJob - * A method to construct the job instance to be tested. This takes a - * string jobName as well as the constructed [[FlinkRunner]] object as - * arguments and must return an instance of the job to be tested. - * @tparam JOB - * The type of the job instance, which in this case is a subclass of - * [[AvroStreamJob]]. - * @tparam OUT - * The output type of the job, which must extend the [[ADT]] with - * [[EmbeddedAvroRecord]] of type [[A]]. - * @tparam A - * The avro record type for the job. This must be a subclass of an - * avro GenericRecord or SpecificRecord. - * @tparam ADT - * The algebraic data type of the job. - * @return - * An instance of FlinkRunner[ADT]. You can run the job by - * invoking this instance's run method. - */ - def getAvroJobRunner[ - JOB <: AvroStreamJob[OUT, A, ADT], - OUT <: ADT with EmbeddedAvroRecord[A], - A <: GenericRecord, - ADT <: FlinkEvent: TypeInformation]( - args: Array[String], - configStr: String, - checkResults: CheckResults[ADT], - getJob: (String, FlinkRunner[ADT]) => JOB): FlinkRunner[ADT] = { - val config = new FlinkConfig( - args, - Some(configStr + "\nenvironment=dev\nmock.edges=true\n") - ) - new FlinkRunner[ADT](config, Some(checkResults)) { - override def invoke(jobName: String): Unit = - getJob(jobName, this).run() - } - } - - lazy val nothingFlinkRunner: FlinkRunner[NothingADT] = - getRunner[NothingADT]() - - lazy val mySimpleFlinkRunner: FlinkRunner[MySimpleADT] = - getRunner[MySimpleADT]() -} + with FlinkRunnerSpec + with LazyLogging {} diff --git a/src/test/scala/io/epiphanous/flinkrunner/FlinkRunnerSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/FlinkRunnerSpec.scala deleted file mode 100644 index ea860ff7..00000000 --- a/src/test/scala/io/epiphanous/flinkrunner/FlinkRunnerSpec.scala +++ /dev/null @@ -1,65 +0,0 @@ -package io.epiphanous.flinkrunner - -import io.epiphanous.flinkrunner.flink.{AvroStreamJob, StreamJob} -import io.epiphanous.flinkrunner.model.{ - CheckResults, - EmbeddedAvroRecord, - FlinkConfig, - FlinkEvent -} -import org.apache.avro.generic.GenericRecord -import org.apache.flink.api.common.typeinfo.TypeInformation - -class FlinkRunnerSpec extends PropSpec { - - def getStreamJobRunner[ - OUT <: ADT: TypeInformation, - ADT <: FlinkEvent: TypeInformation]( - configStr: String, - jobFactory: FlinkRunner[ADT] => StreamJob[OUT, ADT], - checkResultsOpt: Option[CheckResults[ADT]] = None) - : FlinkRunner[ADT] = { - val config = new FlinkConfig(Array("testJob"), Some(configStr)) - new FlinkRunner[ADT](config, checkResultsOpt) { - override def invoke(jobName: String): Unit = jobName match { - case "testJob" => - logger.debug("invoking job") - jobFactory(this).run() - case _ => throw new RuntimeException(s"unknown job $jobName") - } - } - } - - def testStreamJob[ - OUT <: ADT: TypeInformation, - ADT <: FlinkEvent: TypeInformation]( - configStr: String, - jobFactory: FlinkRunner[ADT] => StreamJob[OUT, ADT], - checkResultsOpt: Option[CheckResults[ADT]] = None): Unit = - getStreamJobRunner(configStr, jobFactory, checkResultsOpt).process() - - def getAvroStreamJobRunner[ - OUT <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation, - ADT <: FlinkEvent: TypeInformation]( - configStr: String, - jobFactory: FlinkRunner[ADT] => AvroStreamJob[OUT, A, ADT]) - : FlinkRunner[ADT] = { - val config = new FlinkConfig(Array("testJob"), Some(configStr)) - new FlinkRunner[ADT](config, None) { - override def invoke(jobName: String): Unit = jobName match { - case "testJob" => jobFactory(this).run() - case _ => throw new RuntimeException(s"unknown job $jobName") - } - } - } - - def testAvroStreamJob[ - OUT <: ADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation, - ADT <: FlinkEvent: TypeInformation]( - configStr: String, - jobFactory: FlinkRunner[ADT] => AvroStreamJob[OUT, A, ADT]): Unit = - getAvroStreamJobRunner(configStr, jobFactory).process() - -} diff --git a/src/test/scala/io/epiphanous/flinkrunner/flink/AvroStreamJobSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/flink/AvroStreamJobSpec.scala index cf4e1630..248be95b 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/flink/AvroStreamJobSpec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/flink/AvroStreamJobSpec.scala @@ -1,12 +1,12 @@ package io.epiphanous.flinkrunner.flink import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} -import org.apache.flink.connector.file.src.reader.StreamFormat -import org.apache.flink.formats.parquet.avro.AvroParquetReaders +import io.epiphanous.flinkrunner.PropSpec +import io.epiphanous.flinkrunner.util.test.IdentityMap import org.apache.flink.streaming.api.scala._ import java.time.Instant +import java.time.temporal.ChronoUnit class AvroStreamJobSpec extends PropSpec { @@ -58,7 +58,7 @@ class AvroStreamJobSpec extends PropSpec { } property("singleAvroSource property") { - val cfg = + val cfg = """ |sources { | kafka_source { @@ -68,7 +68,9 @@ class AvroStreamJobSpec extends PropSpec { | } |} |sinks { - | mock_sink { + | kafka_sink { + | topic = bogus2 + | bootstrap.servers = "localhost:9092" | } |} |jobs { @@ -77,16 +79,18 @@ class AvroStreamJobSpec extends PropSpec { |} |execution.runtime-mode = batch |""".stripMargin - val getJob = - (_: String, r: FlinkRunner[MyAvroADT]) => new SingleAvroSourceJob(r) // this creates and runs the job - getAvroJobRunner[SingleAvroSourceJob, AWrapper, ARecord, MyAvroADT]( - Array("SingleAvroSourceJob"), - cfg, - new MyAvroCheckResults(), - getJob - ).process() + val checkResults = new MyAvroCheckResults() + val times2 = new TimesTwo() + getAvroStreamJobRunner[ + AWrapper, + ARecord, + AWrapper, + ARecord, + MyAvroADT + ](cfg, times2, checkResults.inA, checkResultsOpt = Some(checkResults)) + .process() } property("connectedAvroSource property") {} @@ -97,22 +101,14 @@ class AvroStreamJobSpec extends PropSpec { } -class SingleAvroSourceJob(runner: FlinkRunner[MyAvroADT])(implicit - fromKV: EmbeddedAvroRecordInfo[ARecord] => AWrapper) - extends AvroStreamJob[AWrapper, ARecord, MyAvroADT](runner) { - implicit val avroParquetRecordFormat: StreamFormat[ARecord] = - AvroParquetReaders.forSpecificRecord(classOf[ARecord]) - override def transform: DataStream[AWrapper] = - singleAvroSource[AWrapper, ARecord]().map { a => - val (a0, a1, a2, a3) = - (a.$record.a0, a.$record.a1, a.$record.a2, a.$record.a3) - AWrapper( - a.$record.copy( - a0 + a0, - 2 * a1, - 2 * a2, - Instant.ofEpochMilli(a3.toEpochMilli + 2 * 86400000) - ) +class TimesTwo() extends IdentityMap[AWrapper] { + override def map(a: AWrapper): AWrapper = + a.copy( + a.$record.copy( + a.$record.a0 * 2, + a.$record.a1 * 2, + a.$record.a2 * 2, + a.$record.a3.plus(2, ChronoUnit.DAYS) ) - } + ) } diff --git a/src/test/scala/io/epiphanous/flinkrunner/flink/StreamJobTest.scala b/src/test/scala/io/epiphanous/flinkrunner/flink/StreamJobTest.scala deleted file mode 100644 index 7ae07f58..00000000 --- a/src/test/scala/io/epiphanous/flinkrunner/flink/StreamJobTest.scala +++ /dev/null @@ -1,91 +0,0 @@ -//package io.epiphanous.flinkrunner.flink -// -//import io.epiphanous.flinkrunner.model._ -//import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} -//import org.apache.flink.api.common.JobExecutionResult -//import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction -//import org.apache.flink.streaming.api.scala._ -//import org.apache.flink.util.Collector -// -//class StreamJobTest extends PropSpec { -// -// property("singleSource") { -// val in = genPop[BWrapper](10) -// val outLen = in.count(_.value.b2.nonEmpty) -// val config = new FlinkConfig( -// Array("singleSource") -// ) -// new FlinkRunner[MyAvroADT]( -// config, -// Map("in" -> in), -// out => { -// println("========================================") -// out.foreach(println) -// out.length shouldEqual outLen -// out.forall(_.isInstanceOf[AWrapper]) shouldBe true -// } -// ) { -// override def invoke( -// jobName: String): Either[List[_], JobExecutionResult] = { -// jobName match { -// case "singleSource" => new SingleSourceTestJob(this).run() -// } -// } -// }.process() -// } -//} -// -//class SingleSourceTestJob(runner: FlinkRunner[MyAvroADT]) -// extends StreamJob[AWrapper, MyAvroADT](runner) { -// override def transform: DataStream[AWrapper] = -// singleSource[BWrapper]("test").flatMap { bw => -// val b = bw.$record -// b -// .map(d => -// List(AWrapper(ARecord(b.b0, b.b1.getOrElse(1), d, b.b3))) -// ) -// .getOrElse(List.empty) -// } -//} -// -//class BroadcastConnectedTestJob(runner: FlinkRunner[MySimpleADT]) -// extends StreamJob[SimpleC, MySimpleADT](runner) { -// override def transform: DataStream[SimpleC] = -// broadcastConnectedSource[SimpleA, SimpleB, String]("a", "b", _.$key) -// .process( -// new KeyedBroadcastProcessFunction[ -// String, -// SimpleA, -// SimpleB, -// SimpleC] { -// override def processElement( -// value: SimpleA, -// ctx: KeyedBroadcastProcessFunction[ -// String, -// SimpleA, -// SimpleB, -// SimpleC]#ReadOnlyContext, -// out: Collector[SimpleC]): Unit = ??? -// -// override def processBroadcastElement( -// value: SimpleB, -// ctx: KeyedBroadcastProcessFunction[ -// String, -// SimpleA, -// SimpleB, -// SimpleC]#Context, -// out: Collector[SimpleC]): Unit = ??? -// } -// ) -//} -// -//class FilterByControlTestJob(runner: FlinkRunner[MySimpleADT]) -// extends StreamJob[SimpleB, MySimpleADT](runner) { -// override def transform: DataStream[SimpleB] = -// filterByControlSource[SimpleA, SimpleB, String]( -// "a", -// "b", -// _.$key, -// _.$key -// ) -//} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/D64Spec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/D64Spec.scala index a2be0ac4..b2dabc0f 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/D64Spec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/D64Spec.scala @@ -3,7 +3,6 @@ package io.epiphanous.flinkrunner.model import io.epiphanous.flinkrunner.PropSpec import java.nio.ByteBuffer -import java.nio.charset.StandardCharsets import scala.collection.mutable import scala.util.Random diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/FlinkConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/FlinkConfigSpec.scala index 9387b43b..644556d5 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/FlinkConfigSpec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/FlinkConfigSpec.scala @@ -1,8 +1,7 @@ package io.epiphanous.flinkrunner.model -import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} +import io.epiphanous.flinkrunner.PropSpec import org.apache.flink.api.common.RuntimeExecutionMode -import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.streaming.api.environment.CheckpointConfig import java.time.Duration @@ -10,11 +9,10 @@ import scala.collection.JavaConverters.mapAsScalaMapConverter import scala.util.Try class FlinkConfigSpec extends PropSpec { - val cmdLineArgs: Array[String] = + val cmdLineArgs: Array[String] = Array("someJob", "-a", "--arg1", "fish", "--arg2", "dog") - val runner: FlinkRunner[NothingADT] = getRunner[NothingADT](cmdLineArgs) - val config: FlinkConfig = runner.config - val runner2: FlinkRunner[NothingADT] = getRunner[NothingADT]( + val config: FlinkConfig = new FlinkConfig(cmdLineArgs) + val config2 = new FlinkConfig( cmdLineArgs, Some(""" |system.help = "system help" @@ -45,7 +43,6 @@ class FlinkConfigSpec extends PropSpec { |environment = dev |""".stripMargin) ) - val config2: FlinkConfig = runner2.config property("jobName") { config.jobName shouldEqual cmdLineArgs.head @@ -67,9 +64,9 @@ class FlinkConfigSpec extends PropSpec { } property("config precedence with file") { - val configx = getRunner[NothingADT]( + val configx = new FlinkConfig( cmdLineArgs :+ "--config" :+ "resource://test-precedence.conf" - ).config + ) configx.getString( "test.precedence" ) shouldEqual "test-precedence.conf" @@ -195,44 +192,44 @@ class FlinkConfigSpec extends PropSpec { config.environment shouldEqual "dev" } property("isDev") { - getRunner[NothingADT]( + new FlinkConfig( Array.empty[String], Some("environment = dev") - ).config.isDev shouldEqual true - getRunner[NothingADT]( + ).isDev shouldEqual true + new FlinkConfig( Array.empty[String], Some("environment = development") - ).config.isDev shouldEqual true + ).isDev shouldEqual true config2.isDev shouldEqual true } property("isStage") { - getRunner[NothingADT]( + new FlinkConfig( Array.empty[String], Some("environment = staging") - ).config.isStage shouldEqual true - getRunner[NothingADT]( + ).isStage shouldEqual true + new FlinkConfig( Array.empty[String], Some("environment = stage") - ).config.isStage shouldEqual true + ).isStage shouldEqual true config.isStage shouldEqual false } property("isProd") { - getRunner[NothingADT]( + new FlinkConfig( Array.empty[String], Some("environment = production") - ).config.isProd shouldEqual true - getRunner[NothingADT]( + ).isProd shouldEqual true + new FlinkConfig( Array.empty[String], Some("environment = prod") - ).config.isProd shouldEqual true + ).isProd shouldEqual true config.isProd shouldEqual false config2.isProd shouldEqual false } property("watermarkStrategy") { - getRunner[NothingADT]( + new FlinkConfig( Array.empty[String], Some("watermark.strategy = NONE") - ).config.watermarkStrategy shouldEqual "none" + ).watermarkStrategy shouldEqual "none" config.watermarkStrategy shouldEqual "bounded out of orderness" } property("systemHelp") { diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/IcebergConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/IcebergConfigSpec.scala new file mode 100644 index 00000000..dc59403d --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/IcebergConfigSpec.scala @@ -0,0 +1,282 @@ +package io.epiphanous.flinkrunner.model + +import com.dimafeng.testcontainers.lifecycle.and +import com.dimafeng.testcontainers.scalatest.TestContainersForAll +import com.dimafeng.testcontainers.{ + GenericContainer, + LocalStackV2Container +} +import io.epiphanous.flinkrunner.PropSpec +import org.apache.avro.generic.GenericRecord +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.table.data.RowData +import org.apache.hadoop.conf.Configuration +import org.apache.iceberg.aws.AwsProperties +import org.apache.iceberg.catalog.{Namespace, TableIdentifier} +import org.apache.iceberg.data.parquet.GenericParquetWriter +import org.apache.iceberg.data.{IcebergGenerics, Record} +import org.apache.iceberg.parquet.Parquet +import org.apache.iceberg.rest.RESTCatalog +import org.apache.iceberg.{CatalogProperties, PartitionSpec, Schema, Table} +import org.testcontainers.containers.Network +import org.testcontainers.containers.localstack.LocalStackContainer.Service +import org.testcontainers.utility.Base58 +import requests.Response +import software.amazon.awssdk.auth.credentials.AwsCredentials +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient +import software.amazon.awssdk.services.s3.S3Client +import software.amazon.awssdk.services.s3.model.CreateBucketRequest + +import java.util.UUID +import scala.collection.JavaConverters._ +import scala.reflect.runtime.{universe => ru} +import scala.util.Try + +class IcebergConfigSpec extends PropSpec with TestContainersForAll { + val bucketName = "warehouse" + val icebergRESTPort = 8181 + val icebergHost = s"iceberg-${Base58.randomString(6)}" + val localstackHost = s"localstack-${Base58.randomString(6)}" + + val icebergImage: GenericContainer.DockerImage = + GenericContainer.stringToDockerImage("tabulario/iceberg-rest:0.2.0") + + override type Containers = LocalStackV2Container and GenericContainer + + def awsCreds(ls: LocalStackV2Container): AwsCredentials = + ls.staticCredentialsProvider.resolveCredentials() + + def awsRegion(ls: LocalStackV2Container): String = ls.region.toString + + override def startContainers(): Containers = { + val network = Network.newNetwork() + val localstack = LocalStackV2Container( + tag = "1.4.0", + services = Seq(Service.S3) + ).configure(_.withNetwork(network).withNetworkAliases(localstackHost)) + localstack.start() + + val creds = awsCreds(localstack) + val env = Map( + "CATALOG_S3_ACCESS__KEY__ID" -> creds.accessKeyId(), + "CATALOG_S3_SECRET__ACCESS__KEY" -> creds.secretAccessKey(), + "AWS_REGION" -> localstack.region.toString, + "CATALOG_WAREHOUSE" -> s"s3://$bucketName/", + "CATALOG_IO__IMPL" -> "org.apache.iceberg.aws.s3.S3FileIO", + "CATALOG_S3_ENDPOINT" -> s3Endpoint(localstack, outside = false), + "CATALOG_S3_PATH__STYLE__ACCESS" -> "true" + ) + val icebergCatalog = GenericContainer( + dockerImage = icebergImage, + exposedPorts = Seq(icebergRESTPort), + env = env + ).configure { c => + c.withNetwork(network) + c.withNetworkAliases(icebergHost) + } + icebergCatalog.start() + localstack and icebergCatalog + } + + override def afterContainersStart(containers: Containers): Unit = { + super.afterContainersStart(containers) + containers match { + case ls and _ => + val s3 = getS3Client(ls) + s3.createBucket( + CreateBucketRequest.builder().bucket(bucketName).build() + ) + } + } + + def getS3Client(ls: LocalStackV2Container): S3Client = { + S3Client + .builder() + .region(ls.region) + .endpointOverride(ls.endpointOverride(Service.S3)) + .httpClient(UrlConnectionHttpClient.builder().build()) + .credentialsProvider(ls.staticCredentialsProvider) + .forcePathStyle(true) + .build() + } + + def s3Endpoint( + ls: LocalStackV2Container, + outside: Boolean = true): String = { + if (outside) ls.endpointOverride(Service.S3).toString + else + s"http://$localstackHost:4566" + } + + def icebergEndpoint( + ib: GenericContainer, + outside: Boolean = true): String = { + val mappedPort = ib.container.getMappedPort(icebergRESTPort) + if (outside) s"http://localhost:$mappedPort" + else s"http://$icebergHost:$mappedPort" + } + + def icebergRest( + ib: GenericContainer, + path: String, + params: Iterable[(String, String)] = Nil): Response = { + val endpoint = + s"${icebergEndpoint(ib)}${if (path.startsWith("/")) "" + else "/"}$path" + requests.get(endpoint, params = params) + } + + def getIcebergConfig[E]( + ls: LocalStackV2Container, + ib: GenericContainer, + tableName: String, + isSink: Boolean, + otherConfig: String = ""): String = { + val sourceOrSink: String = if (isSink) "sink" else "source" + val creds = awsCreds(ls) + s"""|${sourceOrSink}s { + | iceberg-$sourceOrSink { + | $otherConfig + | catalog { + | name = iceberg + | uri = "${icebergEndpoint(ib)}" + | io-impl = "org.apache.iceberg.aws.s3.S3FileIO" + | s3.endpoint = "${s3Endpoint(ls)}" + | s3.access-key-id = "${creds.accessKeyId()}" + | s3.secret-access-key = "${creds.secretAccessKey()}" + | warehouse = "s3://$bucketName" + | } + | namespace = "testing.tables" + | table = "$tableName" + | } + |} + |""".stripMargin + } + + def writeRowsAsJob[E <: MySimpleADT: TypeInformation: ru.TypeTag]( + data: Seq[E], + tableName: String, + ls: LocalStackV2Container, + ib: GenericContainer)(implicit fromRowData: RowData => E): Unit = { + val configStr = + s""" + |jobs { testJob {} } + |${getIcebergConfig( + ls, + ib, + tableName, + isSink = true + )} + |""".stripMargin + getIdentityTableStreamJobRunner[E, MySimpleADT](configStr, data) + .process() + } + + def writeRowsDirectly[E <: MySimpleADT]( + seq: Seq[E], + table: Table, + schema: Schema): Try[Unit] = + Try { + val filepath = table.location() + "/" + UUID.randomUUID().toString + val file = table.io().newOutputFile(filepath) + val dataWriter = Parquet + .writeData(file) + .schema(schema) + .createWriterFunc(GenericParquetWriter.buildWriter) + .overwrite() + .withSpec(PartitionSpec.unpartitioned()) + .build[org.apache.iceberg.data.GenericRecord]() + seq.map(_.toIcebergRecord).foreach(dataWriter.write) + dataWriter.close() + table.newAppend().appendFile(dataWriter.toDataFile).commit() + println(s"wrote ${seq.size} rows to $table") + } + + def getCatalog( + ls: LocalStackV2Container, + ib: GenericContainer): RESTCatalog = { + val creds = awsCreds(ls) + val props = Map( + AwsProperties.S3FILEIO_ACCESS_KEY_ID -> creds.accessKeyId(), + AwsProperties.S3FILEIO_SECRET_ACCESS_KEY -> creds.secretAccessKey(), + AwsProperties.S3FILEIO_PATH_STYLE_ACCESS -> "true", + "client.region" -> ls.region.toString, + CatalogProperties.CATALOG_IMPL -> "org.apache.iceberg.rest.RESTCatalog", + CatalogProperties.URI -> icebergEndpoint(ib), + CatalogProperties.WAREHOUSE_LOCATION -> s"s3://$bucketName", + CatalogProperties.FILE_IO_IMPL -> "org.apache.iceberg.aws.s3.S3FileIO", + AwsProperties.S3FILEIO_ENDPOINT -> s3Endpoint(ls) + ).asJava + val catalog = new RESTCatalog() + catalog.setConf(new Configuration()) + catalog.initialize("test", props) + catalog + } + + def createTable( + catalog: RESTCatalog, + schema: Schema, + name: String): Table = { + val tblId = TableIdentifier.of(Namespace.of("testing", "tables"), name) + catalog.createTable(tblId, schema, PartitionSpec.unpartitioned()) + } + + def loadTable(catalog: RESTCatalog, name: String): Table = { + val tblId = TableIdentifier.of(Namespace.of("testing", "tables"), name) + catalog.loadTable(tblId) + } + + def writeAvroRowsAsJob[ + E <: MyAvroADT with EmbeddedAvroRecord[A]: TypeInformation, + A <: GenericRecord: TypeInformation]( + rows: Seq[E], + tableName: String, + ls: LocalStackV2Container, + ib: GenericContainer)(implicit + fromKV: EmbeddedAvroRecordInfo[A] => E): Unit = { + val configStr = + s""" + |jobs { testJob {} } + |${getIcebergConfig(ls, ib, tableName, isSink = true)} + |""".stripMargin + getIdentityAvroStreamJobRunner[E, A, MyAvroADT](configStr, rows) + .process() + } + + def readRowsAsJob[E <: MySimpleADT: TypeInformation: ru.TypeTag]( + tableName: String, + checkResults: CheckResults[MySimpleADT], + ls: LocalStackV2Container, + ib: GenericContainer)(implicit fromRowData: RowData => E): Unit = { + val configStr = + s""" + |runtime.mode = batch + |jobs { testJob {} } + |${getIcebergConfig( + ls, + ib, + tableName, + isSink = false, + "batch = true" + )} + |sinks { print-sink {} } + |""".stripMargin + println(s"CONFIG: $configStr") + getIdentityTableStreamJobRunner[E, MySimpleADT]( + configStr, + checkResultsOpt = Some(checkResults) + ) + .process() + } + + def readRowsDirectly[E <: FlinkEvent]( + tableName: String, + ls: LocalStackV2Container, + ib: GenericContainer)(implicit + fromIcebergRecord: Record => E): Try[Iterable[E]] = Try { + val catalog = getCatalog(ls, ib) + val table = loadTable(catalog, tableName) + IcebergGenerics.read(table).build().asScala.map(fromIcebergRecord) + } +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/Id64Spec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/Id64Spec.scala index 66803aed..42aa768e 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/Id64Spec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/Id64Spec.scala @@ -36,11 +36,13 @@ class Id64Spec extends PropSpec { val i = Id64.instantOf(id) val g = Id64.GREGORIAN_OFFSET val e = t - g - mc shouldEqual Math.floor(e / 10) - ms shouldEqual Math.floor(mc / 1000) - i.getEpochSecond shouldEqual Math.floor( - e / 10000000 - ) + mc shouldEqual Math.floor(e / 10.0).toLong + ms shouldEqual Math.floor(mc / 1000.0).toLong + i.getEpochSecond shouldEqual Math + .floor( + e / 10000000.0 + ) + .toLong i.getNano shouldEqual (e - 10000000 * i.getEpochSecond) * 100 } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala index 8fbdd742..924f516a 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/MyAvroADT.scala @@ -2,6 +2,8 @@ package io.epiphanous.flinkrunner.model import io.epiphanous.flinkrunner.serde.{DelimitedConfig, JsonConfig} import org.apache.avro.generic.GenericRecord +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter +import org.apache.flink.table.types.logical.RowType import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -12,6 +14,10 @@ sealed trait MyAvroADT extends FlinkEvent { record: Option[GenericRecord] = None): String } +trait HasRowType { + def getRowType: RowType +} + trait TestSerializers[A <: GenericRecord] { def $record: A @@ -70,11 +76,18 @@ case class AWrapper(value: ARecord) override val $record: ARecord = value } -object AWrapper extends EmbeddedAvroRecordFactory[AWrapper, ARecord] { +object AWrapper + extends EmbeddedAvroRecordFactory[AWrapper, ARecord] + with HasRowType { override implicit def fromKV( info: EmbeddedAvroRecordInfo[ARecord]): AWrapper = AWrapper( info.record ) + + override def getRowType: RowType = AvroSchemaConverter + .convertToDataType(ARecord.SCHEMA$.toString) + .getLogicalType + .asInstanceOf[RowType] } case class BWrapper(value: BRecord) @@ -88,11 +101,19 @@ case class BWrapper(value: BRecord) override val $record: BRecord = value } -object BWrapper extends EmbeddedAvroRecordFactory[BWrapper, BRecord] { +object BWrapper + extends EmbeddedAvroRecordFactory[BWrapper, BRecord] + with HasRowType { override implicit def fromKV( info: EmbeddedAvroRecordInfo[BRecord]): BWrapper = BWrapper( info.record ) + + override def getRowType: RowType = AvroSchemaConverter + .convertToDataType(BRecord.SCHEMA$.toString) + .getLogicalType + .asInstanceOf[RowType] + } case class CWrapper(value: CRecord) @@ -110,9 +131,17 @@ case class CWrapper(value: CRecord) override def $timestamp: Long = value.ts.toEpochMilli } -object CWrapper extends EmbeddedAvroRecordFactory[CWrapper, CRecord] { +object CWrapper + extends EmbeddedAvroRecordFactory[CWrapper, CRecord] + with HasRowType { override implicit def fromKV( info: EmbeddedAvroRecordInfo[CRecord]): CWrapper = CWrapper( info.record ) + + override def getRowType: RowType = AvroSchemaConverter + .convertToDataType(CRecord.SCHEMA$.toString) + .getLogicalType + .asInstanceOf[RowType] + } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/MySimpleADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/MySimpleADT.scala index 229afb2e..3cb9c9e8 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/MySimpleADT.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/MySimpleADT.scala @@ -1,18 +1,66 @@ package io.epiphanous.flinkrunner.model import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import org.apache.flink.table.annotation.DataTypeHint +import org.apache.flink.table.data.RowData +import org.apache.iceberg.Schema +import org.apache.iceberg.data.{GenericRecord, Record} +import org.apache.iceberg.types.Types -import java.time.Instant +import java.time.{Instant, LocalDateTime, OffsetDateTime, ZoneOffset} +import scala.language.implicitConversions -sealed trait MySimpleADT extends FlinkEvent +trait ToIceberg { + def toIcebergRecord: GenericRecord +} + +trait FromIcebergRecord[E <: FlinkEvent] { + def ICEBERG_SCHEMA: Schema + implicit def fromIcebergRecord(r: Record): E +} + +sealed trait MySimpleADT + extends FlinkEvent + with EmbeddedRowType + with ToIceberg case class SimpleA(id: String, a0: String, a1: Int, ts: Instant) - extends MySimpleADT { + extends MySimpleADT + with EmbeddedRowType { override def $id: String = id override def $key: String = a0 override def $timestamp: Long = ts.toEpochMilli + + override def toIcebergRecord: GenericRecord = { + val record = GenericRecord.create(SimpleA.ICEBERG_SCHEMA) + record.setField("id", id) + record.setField("a0", a0) + record.setField("a1", a1) + record.setField("ts", ts) + record + } +} + +object SimpleA + extends EmbeddedRowTypeFactory[SimpleA] + with FromIcebergRecord[SimpleA] { + override implicit def fromRowData(rowData: RowData): SimpleA = SimpleA( + rowData.getString(0).toString, + rowData.getString(1).toString, + rowData.getInt(2), + rowData.getTimestamp(3, 3).toInstant + ) + + override val ICEBERG_SCHEMA: Schema = new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "a0", Types.StringType.get()), + Types.NestedField.required(3, "a1", Types.IntegerType.get()), + Types.NestedField.required(4, "ts", Types.TimestampType.withoutZone()) + ) + + override implicit def fromIcebergRecord(r: Record): SimpleA = ??? } /** Simple Class. Note this has a field (b2) that is an Option[Int]. If we @@ -38,16 +86,61 @@ case class SimpleB( id: String, b0: String, b1: Double, - @JsonDeserialize(contentAs = classOf[java.lang.Integer]) b2: Option[ + @JsonDeserialize(contentAs = classOf[java.lang.Integer]) + @DataTypeHint("INT") + b2: Option[ Int ], ts: Instant) - extends MySimpleADT { + extends MySimpleADT + with EmbeddedRowType { override def $id: String = id override def $key: String = b0 override def $timestamp: Long = ts.toEpochMilli + + override def toIcebergRecord: GenericRecord = { + val record = GenericRecord.create(SimpleB.ICEBERG_SCHEMA) + record.setField("id", id) + record.setField("b0", b0) + record.setField("b1", b1) + record.setField("b2", b2.orNull) + record.setField("ts", LocalDateTime.ofInstant(ts, ZoneOffset.UTC)) + record + } +} + +object SimpleB + extends EmbeddedRowTypeFactory[SimpleB] + with FromIcebergRecord[SimpleB] { + + override implicit def fromRowData(rowData: RowData): SimpleB = { + SimpleB( + rowData.getString(0).toString, + rowData.getString(1).toString, + rowData.getDouble(2), + if (rowData.isNullAt(3)) None else Some(rowData.getInt(3)), + rowData.getTimestamp(4, 3).toInstant + ) + } + + implicit def fromIcebergRecord(record: Record): SimpleB = + SimpleB( + record.getField("id").asInstanceOf[String], + record.getField("b0").asInstanceOf[String], + record.getField("b1").asInstanceOf[Double], + Option(record.getField("b2")).map(_.asInstanceOf[Int]), + record.getField("ts").asInstanceOf[OffsetDateTime].toInstant + ) + + override val ICEBERG_SCHEMA: Schema = new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "b0", Types.StringType.get()), + Types.NestedField.required(3, "b1", Types.DoubleType.get()), + Types.NestedField.optional(4, "b2", Types.IntegerType.get()), + Types.NestedField.required(5, "ts", Types.TimestampType.withoutZone()) + ) } case class SimpleC( @@ -56,10 +149,35 @@ case class SimpleC( c2: Double, c3: Int, ts: Instant) - extends MySimpleADT { + extends MySimpleADT + with EmbeddedRowType { override def $id: String = id override def $key: String = c1 override def $timestamp: Long = ts.toEpochMilli + + override def toIcebergRecord: GenericRecord = ??? +} + +object SimpleC + extends EmbeddedRowTypeFactory[SimpleC] + with FromIcebergRecord[SimpleC] { + override implicit def fromRowData(rowData: RowData): SimpleC = SimpleC( + rowData.getString(0).toString, + rowData.getString(1).toString, + rowData.getDouble(2), + rowData.getInt(3), + rowData.getTimestamp(4, 3).toInstant + ) + + override val ICEBERG_SCHEMA: Schema = new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "c1", Types.StringType.get()), + Types.NestedField.required(3, "c2", Types.DoubleType.get()), + Types.NestedField.required(4, "c3", Types.IntegerType.get()), + Types.NestedField.required(5, "ts", Types.TimestampType.withoutZone()) + ) + + override implicit def fromIcebergRecord(r: Record): SimpleC = ??? } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/NothingADT.scala b/src/test/scala/io/epiphanous/flinkrunner/model/NothingADT.scala deleted file mode 100644 index ddb86401..00000000 --- a/src/test/scala/io/epiphanous/flinkrunner/model/NothingADT.scala +++ /dev/null @@ -1,3 +0,0 @@ -package io.epiphanous.flinkrunner.model - -sealed trait NothingADT extends FlinkEvent diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/StreamJobSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/StreamJobSpec.scala new file mode 100644 index 00000000..2cb75cf7 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/StreamJobSpec.scala @@ -0,0 +1,126 @@ +package io.epiphanous.flinkrunner.model + +import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} +import org.apache.flink.api.scala.createTypeInformation +import org.scalatest.Assertion + +class StreamJobSpec extends PropSpec { + + def isSingleSourceOf( + runner: FlinkRunner[_], + outClass: String): Assertion = { + val sources = runner.getStreamNodesInfo.filter(_.isSource) + sources.size shouldBe 1 + sources.flatMap(_.simpleOutClass) shouldEqual List(outClass) + } + + def isSingleSinkOf( + runner: FlinkRunner[_], + inClass: String): Assertion = { + val sinks = runner.getStreamNodesInfo.filter(_.isSink) + sinks.size shouldBe 1 + sinks.flatMap(_.simpleInClasses) shouldEqual List(inClass) + } + + def testSingleSource(seq: Seq[SimpleA] = Seq.empty): Assertion = { + val runner = getIdentityStreamJobRunner[SimpleA, MySimpleADT]( + executeJob = false + ) + runner.process() + isSingleSourceOf(runner, "SimpleA") + } + + def testSingleSink(seq: Seq[SimpleA] = Seq.empty): Assertion = { + val runner = getIdentityStreamJobRunner[SimpleA, MySimpleADT]( + executeJob = false + ) + runner.process() + isSingleSinkOf(runner, "SimpleA") + } + + def testSingleAvroSource(seq: Seq[BWrapper] = Seq.empty): Assertion = { + val runner = + getIdentityAvroStreamJobRunner[BWrapper, BRecord, MyAvroADT]( + input = seq, + executeJob = false + ) + runner.process() + isSingleSourceOf(runner, "BWrapper") + } + + def testSingleAvroSink(seq: Seq[BWrapper] = Seq.empty): Assertion = { + val runner = + getIdentityAvroStreamJobRunner[BWrapper, BRecord, MyAvroADT]( + input = seq, + executeJob = false + ) + runner.process() + isSingleSinkOf(runner, "BWrapper") + } + + def testSingleRowSource(seq: Seq[SimpleA] = Seq.empty): Assertion = { + val runner = getIdentityTableStreamJobRunner[SimpleA, MySimpleADT]( + input = seq, + executeJob = false + ) + runner.process() + isSingleSourceOf(runner, "GenericRowData") + } + + def testSingleRowSink(seq: Seq[SimpleA] = Seq.empty): Assertion = { + val runner = getIdentityTableStreamJobRunner[SimpleA, MySimpleADT]( + input = seq, + executeJob = false + ) + runner.process() + isSingleSinkOf(runner, "Row") + } + + property("singleAvroSource property") { + testSingleAvroSource() + } + + property("seqOrSingleAvroSource property") { + testSingleAvroSource(genPop[BWrapper]()) + } + + property("sink property") { + testSingleSink() + testSingleSink(genPop[SimpleA]()) + testSingleAvroSink() + testSingleAvroSink(genPop[BWrapper]()) + testSingleRowSink() + testSingleRowSink(genPop[SimpleA]()) + } + + property("singleSource property") { + testSingleSource() + } + + property("singleRowSource property") { + testSingleRowSource() + } + + property("filterByControlSource property") {} + + property("seqOrSingleSource property") { + testSingleSource(genPop[SimpleA]()) + } + + property("seqOrSingleRowSource property") { + testSingleRowSource(genPop[SimpleA]()) + } + + property("run property") {} + + property("windowedAggregation property") {} + + property("broadcastConnectedSource property") {} + + property("transform property") {} + + property("maybeSink property") {} + + property("connectedSource property") {} + +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/FileSinkJobTest.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/FileSinkJobTest.scala index fb3783f2..45df12d2 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/FileSinkJobTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/FileSinkJobTest.scala @@ -1,28 +1,13 @@ package io.epiphanous.flinkrunner.model.sink -import io.epiphanous.flinkrunner.flink.{AvroStreamJob, StreamJob} +import io.epiphanous.flinkrunner.PropSpec import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.{FlinkRunner, FlinkRunnerSpec} import org.apache.flink.api.scala.createTypeInformation -import org.apache.flink.streaming.api.scala.DataStream -class FileSinkJobTest extends FlinkRunnerSpec { - - class IdentityJob(runner: FlinkRunner[MySimpleADT], input: Seq[SimpleB]) - extends StreamJob[SimpleB, MySimpleADT](runner) { - override def transform: DataStream[SimpleB] = - runner.env.fromCollection(input) - } - - class IdentityAvroJob( - runner: FlinkRunner[MyAvroADT], - input: Seq[BWrapper]) - extends AvroStreamJob[BWrapper, BRecord, MyAvroADT](runner) { - override def transform: DataStream[BWrapper] = - runner.env.fromCollection(input) - } +class FileSinkJobTest extends PropSpec { property("write avro json results to sink") { + val input = genPop[BWrapper]() val configStr = """ |sinks { @@ -38,12 +23,10 @@ class FileSinkJobTest extends FlinkRunnerSpec { |} |execution.runtime-mode = batch |""".stripMargin - val input = genPop[BWrapper]() - val factory = - (runner: FlinkRunner[MyAvroADT]) => - new IdentityAvroJob(runner, input) - testAvroStreamJob(configStr, factory) - + getIdentityAvroStreamJobRunner[BWrapper, BRecord, MyAvroADT]( + configStr, + input + ).process() } property("write avro delimited results to sink") { @@ -62,11 +45,10 @@ class FileSinkJobTest extends FlinkRunnerSpec { |} |execution.runtime-mode = batch |""".stripMargin - val input = genPop[BWrapper]() - val factory = - (runner: FlinkRunner[MyAvroADT]) => - new IdentityAvroJob(runner, input) - testAvroStreamJob(configStr, factory) + getIdentityAvroStreamJobRunner[BWrapper, BRecord, MyAvroADT]( + configStr, + genPop[BWrapper]() + ).process() } property("write simple json results to sink") { @@ -85,10 +67,10 @@ class FileSinkJobTest extends FlinkRunnerSpec { |} |execution.runtime-mode = batch |""".stripMargin - val input = genPop[SimpleB]() - val factory = - (runner: FlinkRunner[MySimpleADT]) => new IdentityJob(runner, input) - testStreamJob(configStr, factory) + getIdentityStreamJobRunner[SimpleA, MySimpleADT]( + configStr, + genPop[SimpleA]() + ).process() } property("write simple delimited results to sink") { @@ -107,10 +89,10 @@ class FileSinkJobTest extends FlinkRunnerSpec { |} |execution.runtime-mode = batch |""".stripMargin - val input = genPop[SimpleB]() - val factory = - (runner: FlinkRunner[MySimpleADT]) => new IdentityJob(runner, input) - testStreamJob(configStr, factory) + getIdentityStreamJobRunner[SimpleA, MySimpleADT]( + configStr, + genPop[SimpleA]() + ).process() } } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfigSpec.scala new file mode 100644 index 00000000..20da81bc --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/FirehoseSinkConfigSpec.scala @@ -0,0 +1,202 @@ +package io.epiphanous.flinkrunner.model.sink + +import com.amazonaws.regions.Regions +import io.epiphanous.flinkrunner.model.{ + KinesisProperties, + MySimpleADT, + SimpleA +} +import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} +import org.apache.flink.api.scala.createTypeInformation + +class FirehoseSinkConfigSpec extends PropSpec { + + def testConfig(config: String): FlinkRunner[MySimpleADT] = + getIdentityStreamJobRunner[SimpleA, MySimpleADT]( + s""" + |jobs { + | testJob { + | sinks { + | firehose-sink { + | $config + | } + | } + | } + |} + |""".stripMargin + ) + + property("unconfigured firehose") { + val runner = testConfig("") + the[RuntimeException] thrownBy runner + .getSinkConfig() should have message "kinesis stream name required but missing in sink of job " + } + + property("minimal configuration") { + val runner = testConfig("stream = stream") + noException should be thrownBy runner.getSinkConfig() + } + + property("stream.name") { + val runner = testConfig("stream.name = stream") + noException should be thrownBy runner.getSinkConfig() + } + + property("delivery.stream") { + val runner = testConfig("delivery.stream = stream") + noException should be thrownBy runner.getSinkConfig() + } + + property("delivery.stream.name") { + val runner = testConfig("delivery.stream.name = stream") + noException should be thrownBy runner.getSinkConfig() + } + + def getProps(config: String = ""): KinesisProperties = + testConfig("stream = stream\n" + config) + .getSinkConfig() + .asInstanceOf[FirehoseSinkConfig[MySimpleADT]] + .props + + def testProp[T]( + propList: Seq[String], + prop: KinesisProperties => T, + value: T): Unit = + (propList ++ propList.map(p => s"config.$p")).foreach(c => + prop(getProps(s"$c = $value")) shouldEqual value + ) + + property("stream name") { + getProps().stream shouldEqual "stream" + } + + property("default aws.region") { + getProps().clientProperties.getProperty( + "aws.region" + ) shouldEqual KinesisProperties.DEFAULT_REGION + } + + property("aws.region") { + testProp( + Seq( + "region", + "aws.region" + ), + _.clientProperties.getProperty("aws.region"), + Regions + .values() + .filterNot(_.getName != KinesisProperties.DEFAULT_REGION) + .head + .getName + ) + } + + property("default aws.endpoint") { + getProps().clientProperties + .getProperty("aws.endpoint", "none") shouldEqual "none" + } + + property("aws.endpoint") { + testProp( + Seq( + "endpoint", + "aws.endpoint" + ), + _.clientProperties.getProperty("aws.endpoint"), + "other" + ) + } + + property("default failOnError") { + getProps().failOnError shouldBe KinesisProperties.DEFAULT_FAIL_ON_ERROR + } + + property("failOnError") { + testProp( + Seq( + "fail.on.error", + "failOnError" + ), + _.failOnError, + !KinesisProperties.DEFAULT_FAIL_ON_ERROR + ) + } + + property("default maxInFlightRequests") { + getProps().maxInFlightRequests shouldEqual KinesisProperties.DEFAULT_MAX_IN_FLIGHT_REQUESTS + } + + property("maxInFlightRequests") { + testProp( + Seq( + "maxInFlightRequests", + "max.in.flight.requests" + ), + _.maxInFlightRequests, + 2 * KinesisProperties.DEFAULT_MAX_IN_FLIGHT_REQUESTS + ) + } + + property("default maxBufferedRequests") { + getProps().maxBufferedRequests shouldEqual KinesisProperties.DEFAULT_MAX_BUFFERED_REQUESTS + } + + property("maxBufferedRequests") { + testProp( + Seq( + "maxBufferedRequests", + "max.buffered.requests" + ), + _.maxBufferedRequests, + 2 * KinesisProperties.DEFAULT_MAX_BUFFERED_REQUESTS + ) + } + + property("default maxBatchSizeInNumber") { + getProps().maxBatchSizeInNumber shouldEqual KinesisProperties.DEFAULT_MAX_BATCH_SIZE_IN_NUMBER + } + + property("maxBatchSizeInNumber") { + testProp( + Seq( + "maxBatchSizeInNumber", + "max.batch.size.in.number", + "max.batch.size.number" + ), + _.maxBatchSizeInNumber, + 2 * KinesisProperties.DEFAULT_MAX_BATCH_SIZE_IN_NUMBER + ) + } + + property("default maxBatchSizeInBytes") { + getProps().maxBatchSizeInBytes shouldEqual KinesisProperties.DEFAULT_MAX_BATCH_SIZE_IN_BYTES + } + + property("maxBatchSizeInBytes") { + testProp( + Seq( + "maxBatchSizeInBytes", + "max.batch.size.in.bytes", + "max.batch.size.bytes" + ), + _.maxBatchSizeInBytes, + 2 * KinesisProperties.DEFAULT_MAX_BATCH_SIZE_IN_BYTES + ) + } + + property("default maxBufferTime") { + getProps().maxBufferTime shouldEqual KinesisProperties.DEFAULT_MAX_BUFFER_TIME + } + + property("maxBufferTime") { + testProp( + Seq( + "maxBufferTime", + "max.buffer.time" + ), + _.maxBufferTime, + 2 * KinesisProperties.DEFAULT_MAX_BUFFER_TIME + ) + } + +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfigSpec.scala new file mode 100644 index 00000000..b8c6b848 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/IcebergSinkConfigSpec.scala @@ -0,0 +1,92 @@ +package io.epiphanous.flinkrunner.model.sink + +import com.dimafeng.testcontainers.lifecycle.and +import com.dimafeng.testcontainers.{ + GenericContainer, + LocalStackV2Container +} +import io.epiphanous.flinkrunner.model._ +import io.epiphanous.flinkrunner.util.RowUtils +import org.apache.flink.api.scala.createTypeInformation +import org.apache.flink.table.types.logical.RowType +import requests.Response +import software.amazon.awssdk.services.s3.model.{ + GetObjectRequest, + ListObjectsV2Request +} + +import java.nio.charset.StandardCharsets +import scala.collection.JavaConverters._ + +class IcebergSinkConfigSpec extends IcebergConfigSpec { + + def maybeCreateTableTest( + ls: LocalStackV2Container, + ib: GenericContainer, + rowType: RowType): Unit = { + val config = new FlinkConfig( + Array.empty[String], + Some(getIcebergConfig(ls, ib, "brecord", isSink = true)) + ) + val sinkConfig = + new IcebergSinkConfig[MyAvroADT]("iceberg-sink", config) + val table = + sinkConfig.maybeCreateTable(sinkConfig.getFlinkTableSchema(rowType)) + table should be a 'Success + logger.debug(table.get.toString) + } + + // ****************** TESTS START HERE ************************ + + property("has warehouse s3 bucket") { + withContainers { case ls and _ => + val s3 = getS3Client(ls) + val buckets = s3.listBuckets().buckets().asScala.toList + logger.debug(buckets.toString) + buckets should have size 1 + buckets.head.name() shouldEqual bucketName + } + } + + property("has iceberg catalog endpoint") { + withContainers { case _ and ib => + val r: Response = icebergRest(ib, "/v1/config") + logger.debug(r.text()) + r.statusCode shouldEqual 200 + } + } + + property("create iceberg table") { + withContainers { case ls and ib => + val rowType = RowUtils.rowTypeOf[BWrapper] + logger.debug(rowType.toString) + maybeCreateTableTest(ls, ib, rowType) + val s3 = getS3Client(ls) + val r = s3.listObjectsV2( + ListObjectsV2Request.builder().bucket(bucketName).build() + ) + r.contents().asScala.foreach { o => + println(o.key()) + println( + s3.getObjectAsBytes( + GetObjectRequest + .builder() + .bucket(bucketName) + .key(o.key()) + .build() + ).asString(StandardCharsets.UTF_8) + ) + } + } + } + + property("can write some rows") { + withContainers { case ls and ib => + val data = genPop[SimpleB]() + writeRowsAsJob(data, "simple_b", ls, ib) + val readRows = readRowsDirectly[SimpleB]("simple_b", ls, ib) + readRows.success.value.toList.sortBy(_.id) shouldBe data.sortBy(_.id) + } + } + +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkCreateTableTest.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkCreateTableTest.scala index 7c000b37..2f1629e5 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkCreateTableTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkCreateTableTest.scala @@ -6,8 +6,7 @@ import com.dimafeng.testcontainers.{ PostgreSQLContainer } import io.epiphanous.flinkrunner.UnitSpec -import io.epiphanous.flinkrunner.model.MyAvroADT -import org.apache.flink.api.scala.createTypeInformation +import io.epiphanous.flinkrunner.model.{FlinkConfig, MyAvroADT} class JdbcSinkCreateTableTest extends UnitSpec { @@ -23,7 +22,7 @@ class JdbcSinkCreateTableTest extends UnitSpec { jdbcUrl: String, username: String, password: String) = { - val runner = getRunner[MyAvroADT]( + val config = new FlinkConfig( Array.empty[String], Some(s""" |sinks { @@ -68,7 +67,7 @@ class JdbcSinkCreateTableTest extends UnitSpec { |""".stripMargin) ) val sinkConfig = - new JdbcSinkConfig[MyAvroADT]("jdbc-test", runner.config) + new JdbcSinkConfig[MyAvroADT]("jdbc-test", config) sinkConfig.maybeCreateTable() } @@ -78,7 +77,7 @@ class JdbcSinkCreateTableTest extends UnitSpec { jdbcUrl: String, username: String, password: String): Unit = { - val runner = getRunner[MyAvroADT]( + val config = new FlinkConfig( Array.empty[String], Some(s""" |sinks { @@ -121,7 +120,7 @@ class JdbcSinkCreateTableTest extends UnitSpec { |""".stripMargin) ) val sinkConfig = - new JdbcSinkConfig[MyAvroADT]("jdbc-test", runner.config) + new JdbcSinkConfig[MyAvroADT]("jdbc-test", config) sinkConfig.maybeCreateTable() } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkJobTest.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkJobTest.scala index 29fa3225..f9344428 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkJobTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/JdbcSinkJobTest.scala @@ -64,7 +64,6 @@ class JdbcSinkJobTest extends SinkSpec { "resource://SampleB.csv", otherJobConfig = "show.plan = true" ) - val props = new Properties() props.put("user", pgContainer.username) props.put("password", pgContainer.password) diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleAvroIdentityJob.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleAvroIdentityJob.scala deleted file mode 100644 index 0c3c3ae5..00000000 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleAvroIdentityJob.scala +++ /dev/null @@ -1,27 +0,0 @@ -package io.epiphanous.flinkrunner.model.sink - -import io.epiphanous.flinkrunner.FlinkRunner -import io.epiphanous.flinkrunner.flink.AvroStreamJob -import io.epiphanous.flinkrunner.model.{ - EmbeddedAvroRecord, - EmbeddedAvroRecordInfo, - MyAvroADT -} -import org.apache.avro.generic.GenericRecord -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.createTypeInformation -import org.apache.flink.streaming.api.scala.DataStream - -class SimpleAvroIdentityJob[ - E <: MyAvroADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation](runner: FlinkRunner[MyAvroADT])( - implicit fromKV: EmbeddedAvroRecordInfo[A] => E) - extends AvroStreamJob[E, A, MyAvroADT](runner) { - - override def transform: DataStream[E] = { - singleAvroSource[E, A]().map { e: E => - println(e.$record.toString) - e - } - } -} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleIdentityJob.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleIdentityJob.scala deleted file mode 100644 index 95a56502..00000000 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SimpleIdentityJob.scala +++ /dev/null @@ -1,20 +0,0 @@ -package io.epiphanous.flinkrunner.model.sink - -import io.epiphanous.flinkrunner.FlinkRunner -import io.epiphanous.flinkrunner.flink.StreamJob -import io.epiphanous.flinkrunner.model.MySimpleADT -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala.createTypeInformation -import org.apache.flink.streaming.api.scala.DataStream - -class SimpleIdentityJob[E <: MySimpleADT: TypeInformation]( - runner: FlinkRunner[MySimpleADT]) - extends StreamJob[E, MySimpleADT](runner) { - - override def transform: DataStream[E] = { - singleSource[E]().map { e: E => - println(e.toString) - e - } - } -} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SinkSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/sink/SinkSpec.scala index 89f7ec33..4e90a5c6 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/sink/SinkSpec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/sink/SinkSpec.scala @@ -1,26 +1,12 @@ package io.epiphanous.flinkrunner.model.sink +import io.epiphanous.flinkrunner.PropSpec import io.epiphanous.flinkrunner.model._ -import io.epiphanous.flinkrunner.{FlinkRunner, FlinkRunnerSpec} import org.apache.avro.generic.GenericRecord import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.createTypeInformation -class SinkSpec extends FlinkRunnerSpec with AvroFileTestUtils { - - def getFactory[E <: MySimpleADT: TypeInformation] - : FlinkRunner[MySimpleADT] => SimpleIdentityJob[E] = - (runner: FlinkRunner[MySimpleADT]) => new SimpleIdentityJob[E](runner) - - def getAvroFactory[ - E <: MyAvroADT with EmbeddedAvroRecord[A]: TypeInformation, - A <: GenericRecord: TypeInformation](implicit - fromKV: EmbeddedAvroRecordInfo[A] => E) - : FlinkRunner[MyAvroADT] => SimpleAvroIdentityJob[ - E, - A - ] = (runner: FlinkRunner[MyAvroADT]) => - new SimpleAvroIdentityJob[E, A](runner) +class SinkSpec extends PropSpec with AvroFileTestUtils { def getJobConfig( sinkConfigStr: String, @@ -51,17 +37,17 @@ class SinkSpec extends FlinkRunnerSpec with AvroFileTestUtils { sourceFile: String, sourceFormat: String = "csv", batchMode: Boolean = true, - otherJobConfig: String = "runtime.execution-mode = batch"): Unit = - testStreamJob( + otherJobConfig: String = "runtime.execution-mode = batch"): Unit = { + getIdentityStreamJobRunner[E, MySimpleADT]( getJobConfig( sinkConfigStr, sourceFile, sourceFormat, batchMode, otherJobConfig - ), - getFactory[E] - ) + ) + ).process() + } def testAvroJob[ E <: MyAvroADT with EmbeddedAvroRecord[A]: TypeInformation, @@ -72,15 +58,14 @@ class SinkSpec extends FlinkRunnerSpec with AvroFileTestUtils { batchMode: Boolean = true, otherJobConfig: String = "")(implicit fromKV: EmbeddedAvroRecordInfo[A] => E): Unit = - testAvroStreamJob( + getIdentityAvroStreamJobRunner[E, A, MyAvroADT]( getJobConfig( sinkConfigStr, sourceFile, sourceFormat, batchMode, otherJobConfig - ), - getAvroFactory[E, A] - ) + ) + ).process() } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfigTest.scala b/src/test/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfigTest.scala index 6e726a13..50e01ef3 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfigTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/source/FileSourceConfigTest.scala @@ -37,7 +37,7 @@ class FileSourceConfigTest extends PropSpec with AvroFileTestUtils { getTempFile(format).map { path => val file = path.toString writeFile(file, format, in) - val optConfig = + val configStr = s""" |execution.runtime-mode = batch |jobs { @@ -51,12 +51,13 @@ class FileSourceConfigTest extends PropSpec with AvroFileTestUtils { | } |} |""".stripMargin - getAvroJobRunner[TestIdentityJob, BWrapper, BRecord, MyAvroADT]( - Array(s"$fmtName-test-job"), - optConfig, - new TestCheckResults(in, path), - (_, runner) => new TestIdentityJob(runner) - ).process() + getIdentityAvroStreamJobRunner[BWrapper, BRecord, MyAvroADT]( + configStr, + in, + checkResultsOpt = Some(new TestCheckResults(in, path)), + args = Array(s"$fmtName-test-job") + ) + .process() } } diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfigSpec.scala new file mode 100644 index 00000000..8b513ca0 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/model/source/IcebergSourceConfigSpec.scala @@ -0,0 +1,47 @@ +package io.epiphanous.flinkrunner.model.source + +import com.dimafeng.testcontainers.lifecycle.and +import io.epiphanous.flinkrunner.model.{ + CheckResults, + IcebergConfigSpec, + MySimpleADT, + SimpleB +} +import org.apache.flink.api.scala.createTypeInformation + +class IcebergSourceConfigSpec extends IcebergConfigSpec { + + override def afterContainersStart(containers: Containers): Unit = { + super.afterContainersStart(containers) + containers match { + case ls and ib => + } + } + + property("can read some rows") { + withContainers { case ls and ib => + val data = genPop[SimpleB]().sortBy(_.id) +// println("INPUT:") +// data.foreach(println) + val catalog = getCatalog(ls, ib) + val schema = SimpleB.ICEBERG_SCHEMA + val table = createTable(catalog, schema, "simple_b") + val checkResults: CheckResults[MySimpleADT] = + new CheckResults[MySimpleADT] { + val name = "check-iceberg-read" + + override def checkOutputEvents[OUT <: MySimpleADT]( + out: List[OUT]): Unit = { + val results = out.map(_.asInstanceOf[SimpleB]).sortBy(_.id) +// println("OUTPUT:") +// results.foreach(println) + results shouldEqual data + } + } + writeRowsDirectly[SimpleB](data, table, schema).fold( + t => throw t, + _ => readRowsAsJob[SimpleB]("simple_b", checkResults, ls, ib) + ) + } + } +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfigSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfigSpec.scala index e1c095cf..4f624e0b 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfigSpec.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/model/source/KinesisSourceConfigSpec.scala @@ -31,21 +31,39 @@ class KinesisSourceConfigSpec extends PropSpec { |stream = test |""".stripMargin - def defaultConfig: KinesisSourceConfig[MySimpleADT] = getConfig( - requiredProps - ) + val requiredProps1: String = + """ + |streams = test + |""".stripMargin + + val requiredPropsMulti1: String = + """ + |stream = "test1, test2" + |""".stripMargin + + val requiredPropsMulti2: String = + """ + |streams = [test1, test2] + |""".stripMargin + + def defaultConfig( + reqProps: String = requiredProps): KinesisSourceConfig[MySimpleADT] = + getConfig( + reqProps + ) def defaultConfigPlus( str: String, - job: String = "test"): KinesisSourceConfig[MySimpleADT] = - getConfig(requiredProps + str, job) + job: String = "test", + reqProps: String = requiredProps): KinesisSourceConfig[MySimpleADT] = + getConfig(reqProps + str, job) def noProvidedConfig: KinesisSourceConfig[MySimpleADT] = getConfig( "config={}" ) property("default startPos property") { - defaultConfig.startPos shouldEqual "LATEST" + defaultConfig().startPos shouldEqual "LATEST" } property("bad startPos property") { @@ -66,12 +84,40 @@ class KinesisSourceConfigSpec extends PropSpec { ).startPos shouldEqual "TRIM_HORIZON" } - property("start.pos=at_timestamp property") { + property("start.pos=at_timestamp no timestamp property") { the[Exception] thrownBy { defaultConfigPlus(""" |start.pos = AT_TIMESTAMP |""".stripMargin) - } should have message "kinesis sink kinesis-test set starting.position to AT_TIMESTAMP but provided no starting.timestamp" + } should have message "Kinesis source kinesis-test set starting.position to AT_TIMESTAMP but provided no starting.timestamp" + } + + property("start.pos=at_timestamp bad timestamp format property") { + intercept[Exception] { + defaultConfigPlus(""" + |start.pos = AT_TIMESTAMP + |start.ts = "2023-02-23T12:00:00" + |timestamp.format = bad-timestamp-format + |""".stripMargin) + }.getCause should have message "Illegal pattern character 'b'" + } + + property("start.pos=at_timestamp bad timestamp property") { + intercept[Exception] { + defaultConfigPlus(""" + |start.pos = AT_TIMESTAMP + |start.ts = bad-timestamp + |""".stripMargin) + }.getCause should have message "Unparseable date: \"bad-timestamp\"" + } + + property("start.pos=at_timestamp negative timestamp property") { + intercept[Exception] { + defaultConfigPlus(""" + |start.pos = AT_TIMESTAMP + |start.ts = -100 + |""".stripMargin) + }.getCause should have message "Kinesis source kinesis-test has negative starting timestamp value '-100.0'" } property("efoConsumer property") { @@ -85,7 +131,7 @@ class KinesisSourceConfigSpec extends PropSpec { } property("useEfo true by default property") { - defaultConfig.useEfo shouldBe true + defaultConfig().useEfo shouldBe true } property("useEfo property") { @@ -94,8 +140,39 @@ class KinesisSourceConfigSpec extends PropSpec { defaultConfigPlus("efo.enabled = false").useEfo shouldBe false } + property("parallelism property") { + defaultConfigPlus("parallelism = 10").parallelism shouldBe 10 + defaultConfigPlus("parallelism = 10.5").parallelism shouldBe 10 + } + + property("multi streams via stream property") { + defaultConfig(requiredPropsMulti1).streams shouldBe Seq( + "test1", + "test2" + ) + } + + property("multi streams via streams property") { + defaultConfig(requiredPropsMulti2).streams shouldBe List( + "test1", + "test2" + ) + } + + property("single stream via streams property") { + defaultConfig(requiredProps1).streams shouldBe List( + "test" + ) + } + + property("single stream via stream property") { + defaultConfig(requiredProps).streams shouldBe List( + "test" + ) + } + property("missing stream property") { - the[Exception] thrownBy noProvidedConfig should have message "kinesis source kinesis-test is missing required 'stream' property" + the[Exception] thrownBy noProvidedConfig should have message "Kinesis source kinesis-test is missing required 'stream' or 'streams' property" } } diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest.scala index de65bc1f..ecf99fb9 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest.scala @@ -14,8 +14,7 @@ import scala.collection.mutable class ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest extends SerdeTestFixtures { - // ignore until set up testcontainers schema registry - ignore("deserialize works for bwrapper") { + property("deserialize works for bwrapper") { val serde = getDeserializerFor[BWrapper, BRecord] val collected = mutable.ArrayBuffer.empty[BWrapper] val collector = new Collector[BWrapper] { @@ -29,7 +28,7 @@ class ConfluentAvroRegistryKafkaRecordDeserializationSchemaTest collected.head shouldEqual bWrapper } - ignore("deserialize works for awrapper") { + property("deserialize works for awrapper") { val serde = getDeserializerFor[AWrapper, ARecord] val collected = mutable.ArrayBuffer.empty[AWrapper] val collector = new Collector[AWrapper] { diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala index e0c2688a..e3137fb8 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/ConfluentAvroRegistryKafkaRecordSerializationSchemaTest.scala @@ -18,13 +18,15 @@ class ConfluentAvroRegistryKafkaRecordSerializationSchemaTest } // ignore this until we set up testcontainers schema registry testing - ignore("serialize a MyAvroADT instance to a producer record") { + property("serialize a MyAvroADT instance to a producer record") { val serializer = getSerializerFor[BWrapper, BRecord] val serialized = serializer.serialize( bWrapper, null, Instant.now().toEpochMilli ) +// showBytes("serialized key:", serialized.key()) +// showBytes("serialized value:", serialized.value()) serialized.key() shouldEqual bKeyBytes serialized.value() shouldEqual bValueBytes serialized.timestamp() shouldEqual bWrapper.$timestamp diff --git a/src/test/scala/io/epiphanous/flinkrunner/serde/SerdeTestFixtures.scala b/src/test/scala/io/epiphanous/flinkrunner/serde/SerdeTestFixtures.scala index ab385577..2aaf6d10 100644 --- a/src/test/scala/io/epiphanous/flinkrunner/serde/SerdeTestFixtures.scala +++ b/src/test/scala/io/epiphanous/flinkrunner/serde/SerdeTestFixtures.scala @@ -6,10 +6,10 @@ import io.confluent.kafka.schemaregistry.client.{ SchemaMetadata, SchemaRegistryClient } -import io.epiphanous.flinkrunner.PropSpec import io.epiphanous.flinkrunner.model._ import io.epiphanous.flinkrunner.model.sink.KafkaSinkConfig import io.epiphanous.flinkrunner.model.source.KafkaSourceConfig +import io.epiphanous.flinkrunner.{FlinkRunner, PropSpec} import org.apache.avro.Schema import org.apache.avro.generic.{ GenericContainer, @@ -25,7 +25,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord import java.io.{ByteArrayOutputStream, DataOutputStream} trait SerdeTestFixtures extends PropSpec { - val optConfig: String = + val optConfig: String = s""" |jobs { | DeduplicationJob { @@ -60,8 +60,11 @@ trait SerdeTestFixtures extends PropSpec { | } |} |""".stripMargin - val runner = - getRunner[MyAvroADT](Array("confluent-serde-test"), Some(optConfig)) + val runner: FlinkRunner[MyAvroADT] = + getIdentityAvroStreamJobRunner[AWrapper, ARecord, MyAvroADT]( + configStr = optConfig, + args = Array("confluent-serde-test") + ) val kafkaSinkConfig: KafkaSinkConfig[MyAvroADT] = runner.getSinkConfig("test").asInstanceOf[KafkaSinkConfig[MyAvroADT]] @@ -111,7 +114,8 @@ trait SerdeTestFixtures extends PropSpec { A, MyAvroADT ]( - kafkaSinkConfig + kafkaSinkConfig, + Some(schemaRegistryClient) ) } ss.open(null, null) @@ -132,7 +136,8 @@ trait SerdeTestFixtures extends PropSpec { A, MyAvroADT ]( - kafkaSourceConfig + kafkaSourceConfig, + schemaRegistryClientOpt = Some(schemaRegistryClient) ) ds.open(null) ds diff --git a/src/test/scala/io/epiphanous/flinkrunner/util/AttributesOfSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/util/AttributesOfSpec.scala new file mode 100644 index 00000000..b829d379 --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/util/AttributesOfSpec.scala @@ -0,0 +1,20 @@ +package io.epiphanous.flinkrunner.util + +import io.epiphanous.flinkrunner.PropSpec +import io.epiphanous.flinkrunner.model.SimpleA + +class AttributesOfSpec extends PropSpec { + + property("works") { + val a = genOne[SimpleA] + val f = Fields.of( + a + ) // <-- ignore this implicit error in intellij, compiler is ok + f shouldEqual List( + ("id", a.id), + ("a0", a.a0), + ("a1", a.a1), + ("ts", a.ts) + ) + } +} diff --git a/src/test/scala/io/epiphanous/flinkrunner/util/RowUtilsSpec.scala b/src/test/scala/io/epiphanous/flinkrunner/util/RowUtilsSpec.scala new file mode 100644 index 00000000..40e0013b --- /dev/null +++ b/src/test/scala/io/epiphanous/flinkrunner/util/RowUtilsSpec.scala @@ -0,0 +1,32 @@ +package io.epiphanous.flinkrunner.util + +import io.epiphanous.flinkrunner.PropSpec +import io.epiphanous.flinkrunner.model.{BWrapper, SimpleB} +import org.apache.iceberg.flink.FlinkSchemaUtil + +import scala.collection.JavaConverters._ + +class RowUtilsSpec extends PropSpec { + + property("rowType works for avro") { + val x = RowUtils.rowTypeOf[BWrapper] + x shouldEqual BWrapper.getRowType + } + + property("rowType works for non-avro") { + val z = RowUtils.rowTypeOf[SimpleB] + val zz = FlinkSchemaUtil.convert(SimpleB.ICEBERG_SCHEMA) + // Ugggh. We need this complicated comparison because FlinkSchemaUtil + // from iceberg doesn't convert timestamps consistently + z.getFieldNames.asScala.zipWithIndex.foreach { case (f, i) => + val actual = z.getTypeAt(i) + val expected = zz.getTypeAt(i) + if (f == "ts") { + actual.asSummaryString() shouldEqual "TIMESTAMP_LTZ(6) NOT NULL" + expected.asSummaryString() shouldEqual "TIMESTAMP(6) NOT NULL" + } else { + actual shouldEqual expected + } + } + } +}