├── .gitignore ├── .scalafmt.conf ├── README.md ├── build.sbt ├── project └── build.properties └── src └── main └── scala └── com └── ververica ├── Example_01_DataStream_Motivation.scala ├── Example_02_Table_Motivation.scala ├── Example_03_DataStream_ETL.scala ├── Example_04_Table_ETL.scala ├── Example_05_DataStream_Deduplicate.scala ├── Example_06_DataStream_Join.scala ├── Example_07_Table_Deduplicate_Join.scala ├── Example_08_Table_Updating_Join.scala ├── Example_09_Table_Temporal_Join.scala ├── Example_10_Table_Temporal_Join_Pure.scala ├── FillKafkaWithCustomers.scala ├── FillKafkaWithTransactions.scala ├── data └── ExampleData.scala └── models ├── Customer.scala └── Transaction.scala /.gitignore: -------------------------------------------------------------------------------- 1 | ### SBT ### 2 | # Simple Build Tool 3 | # http://www.scala-sbt.org/release/docs/Getting-Started/Directories.html#configuring-version-control 4 | 5 | dist/* 6 | target/ 7 | lib_managed/ 8 | src_managed/ 9 | project/boot/ 10 | project/plugins/project/ 11 | .history 12 | .cache 13 | .lib/ 14 | .bsp/ 15 | 16 | ### Scala ### 17 | *.class 18 | *.log 19 | 20 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml 21 | hs_err_pid* 22 | 23 | target/ 24 | pom.xml.tag 25 | pom.xml.releaseBackup 26 | pom.xml.versionsBackup 27 | pom.xml.next 28 | release.properties 29 | dependency-reduced-pom.xml 30 | buildNumber.properties 31 | .mvn/timing.properties 32 | # https://github.com/takari/maven-wrapper#usage-without-binary-jar 33 | .mvn/wrapper/maven-wrapper.jar 34 | .idea 35 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | version = 3.3.0 2 | runner.dialect = scala3 3 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Flink API Examples for DataStream API and Table API in Scala 3 2 | 3 | Flink is now Scala Free. In the upcoming 1.15 release, Flink will not expose any specific Scala version. 4 | Users can now choose whatever Scala version they need in their user code, including Scala 3. 5 | 6 | This repository is a reimplementation of Timo Walther's [Flink API Examples for DataStream API and Table API](https://github.com/twalthr/flink-api-examples) 7 | examples in Scala 3. 8 | You can watch his talk [Flink's Table & DataStream API: A Perfect Symbiosis](https://youtu.be/vLLn5PxF2Lw) on YouTube which walks through the Java version of this code. 9 | 10 | # How to Use This Repository 11 | 12 | 1. Import this repository into your IDE (preferably IntelliJ IDEA). The project uses the latest Flink 1.15 nightly version. 13 | 14 | 2. All examples are runnable from the IDE or SBT. You simply need to execute the `main()` method of every example class. 15 | 16 | 3. In order to make the examples run within IntelliJ IDEA, it is necessary to tick 17 | the `Add dependencies with "provided" scope to classpath` option in the run configuration under `Modify options`. 18 | 19 | 4. For the Apache Kafka examples, download and unzip [Apache Kafka](https://kafka.apache.org/downloads). 20 | 21 | 5. Start up Kafka and Zookeeper: 22 | 23 | ``` 24 | ./bin/zookeeper-server-start.sh config/zookeeper.properties & 25 | 26 | ./bin/kafka-server-start.sh config/server.properties & 27 | ``` 28 | 29 | 6. Run `FillKafkaWithCustomers` and `FillKafkaWithTransactions` to create and fill the Kafka topics with Flink. -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | name := "flink-scala-3" 2 | 3 | version := "0.1" 4 | 5 | scalaVersion := "3.0.2" 6 | 7 | resolvers += Resolver.mavenLocal 8 | 9 | javacOptions ++= Seq("-source", "11", "-target", "11") 10 | 11 | libraryDependencies += "org.apache.flink" % "flink-streaming-java" % "1.15-SNAPSHOT" 12 | libraryDependencies += "org.apache.flink" % "flink-clients" % "1.15-SNAPSHOT" 13 | libraryDependencies += "org.apache.flink" % "flink-table-planner-loader" % "1.15-SNAPSHOT" 14 | libraryDependencies += "org.apache.flink" % "flink-table-common" % "1.15-SNAPSHOT" 15 | libraryDependencies += "org.apache.flink" % "flink-table-api-java" % "1.15-SNAPSHOT" 16 | libraryDependencies += "org.apache.flink" % "flink-table-api-java-bridge" % "1.15-SNAPSHOT" 17 | libraryDependencies += "org.apache.flink" % "flink-table-runtime" % "1.15-SNAPSHOT" 18 | libraryDependencies += "org.apache.flink" % "flink-connector-kafka" % "1.15-SNAPSHOT" 19 | 20 | libraryDependencies += "com.github.losizm" %% "little-json" % "9.0.0" 21 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 1.7.1 2 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_01_DataStream_Motivation.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 4 | import com.ververica.data.ExampleData 5 | 6 | /** Basic example of generating data and printing it. */ 7 | @main def example1 = 8 | val env = StreamExecutionEnvironment.getExecutionEnvironment 9 | 10 | env 11 | .fromElements(ExampleData.customers: _*) 12 | .executeAndCollect 13 | .forEachRemaining(println) 14 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_02_Table_Motivation.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment} 4 | 5 | import java.time.LocalDate 6 | import org.apache.flink.table.api.Expressions.row 7 | 8 | /** Basic example of generating data and printing it. */ 9 | @main def example2 = 10 | val env = TableEnvironment.create(EnvironmentSettings.inStreamingMode()) 11 | 12 | env 13 | .fromValues( 14 | row(12L, "Alice", LocalDate.of(1984, 2, 12)), 15 | row(32L, "Bob", LocalDate.of(1990, 10, 14)), 16 | row(7L, "Kyle", LocalDate.of(1979, 2, 23)) 17 | ) 18 | .as("c_id", "c_name", "c_birthday") 19 | .execute() 20 | .print() 21 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_03_DataStream_ETL.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 5 | 6 | import java.time.{LocalDate, Period} 7 | 8 | /** Use arbitrary libraries to perform streaming ETL i.e. convert records into 9 | * JSON. 10 | */ 11 | @main def example3 = 12 | val env = StreamExecutionEnvironment.getExecutionEnvironment 13 | 14 | env 15 | .fromElements(ExampleData.customers: _*) 16 | .map(customer => 17 | val age = Period 18 | .between(customer.c_birthday, LocalDate.now()) 19 | .getYears 20 | 21 | s""" 22 | |{ 23 | | "name": ${customer.c_name}, 24 | | "age": $age 25 | |} 26 | |""".stripMargin 27 | ) 28 | .executeAndCollect() 29 | .forEachRemaining(println); 30 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_04_Table_ETL.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import org.apache.flink.table.api.{ 4 | EnvironmentSettings, 5 | JsonOnNull, 6 | TableEnvironment 7 | } 8 | import org.apache.flink.table.api.Expressions.* 9 | import org.apache.flink.table.expressions.TimePointUnit 10 | 11 | import java.time.LocalDate 12 | 13 | /** Use built-in functions to perform streaming ETL i.e. convert records into 14 | * JSON. 15 | */ 16 | @main def example4 = 17 | val env = TableEnvironment.create(EnvironmentSettings.inStreamingMode) 18 | env 19 | .fromValues( 20 | row(12L, "Alice", LocalDate.of(1984, 3, 12)), 21 | row(32L, "Bob", LocalDate.of(1990, 10, 14)), 22 | row(7L, "Kyle", LocalDate.of(1979, 2, 23)) 23 | ) 24 | .as("c_id", "c_name", "c_birthday") 25 | .select( 26 | jsonObject( 27 | JsonOnNull.NULL, 28 | "name", 29 | $("c_name"), 30 | "age", 31 | timestampDiff(TimePointUnit.YEAR, $("c_birthday"), currentDate()) 32 | ) 33 | ) 34 | .execute 35 | .print() 36 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_05_DataStream_Deduplicate.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.models.{Transaction, TransactionDeserializer} 4 | import org.apache.flink.api.common.eventtime.WatermarkStrategy 5 | import org.apache.flink.api.common.state.ValueState 6 | import org.apache.flink.api.common.state.ValueStateDescriptor 7 | import org.apache.flink.configuration.Configuration 8 | import org.apache.flink.connector.kafka.source.KafkaSource 9 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer 10 | import org.apache.flink.streaming.api.datastream.DataStream 11 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 12 | import org.apache.flink.streaming.api.functions.KeyedProcessFunction 13 | import org.apache.flink.util.Collector 14 | 15 | import java.time.Duration 16 | 17 | /** Use Flink's state to perform efficient record deduplication. */ 18 | @main def example5 = 19 | val env = StreamExecutionEnvironment.getExecutionEnvironment 20 | 21 | // set up a Kafka source 22 | val transactionSource = KafkaSource 23 | .builder[Transaction] 24 | .setBootstrapServers("localhost:9092") 25 | .setTopics("transactions") 26 | .setStartingOffsets(OffsetsInitializer.earliest) 27 | .setValueOnlyDeserializer(new TransactionDeserializer) 28 | .build 29 | 30 | val transactionStream = env.fromSource( 31 | transactionSource, 32 | WatermarkStrategy.noWatermarks, 33 | "Transactions" 34 | ) 35 | 36 | transactionStream 37 | .keyBy((t: Transaction) => t.t_id) 38 | .process(new DataStreamDeduplicate) 39 | .executeAndCollect 40 | .forEachRemaining(println) 41 | 42 | class DataStreamDeduplicate 43 | extends KeyedProcessFunction[Long, Transaction, Transaction]: 44 | // use Flink's managed keyed state 45 | var seen: ValueState[Transaction] = _ 46 | 47 | override def open(parameters: Configuration): Unit = 48 | seen = getRuntimeContext.getState( 49 | new ValueStateDescriptor("seen", classOf[Transaction]) 50 | ) 51 | 52 | @throws[Exception] 53 | override def processElement( 54 | transaction: Transaction, 55 | context: KeyedProcessFunction[Long, Transaction, Transaction]#Context, 56 | out: Collector[Transaction] 57 | ): Unit = 58 | if (seen.value == null) { 59 | seen.update(transaction) 60 | // use timers to clean up state 61 | context.timerService.registerProcessingTimeTimer( 62 | context.timerService.currentProcessingTime + Duration 63 | .ofHours(1) 64 | .toMillis 65 | ) 66 | out.collect(transaction) 67 | } 68 | 69 | override def onTimer( 70 | timestamp: Long, 71 | ctx: KeyedProcessFunction[Long, Transaction, Transaction]#OnTimerContext, 72 | out: Collector[Transaction] 73 | ): Unit = 74 | seen.clear() 75 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_06_DataStream_Join.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import com.ververica.models.{Customer, Transaction, TransactionDeserializer} 5 | import org.apache.flink.api.common.eventtime.WatermarkStrategy 6 | import org.apache.flink.api.common.state.{ 7 | ListState, 8 | ListStateDescriptor, 9 | ValueState, 10 | ValueStateDescriptor 11 | } 12 | import org.apache.flink.configuration.Configuration 13 | import org.apache.flink.connector.kafka.source.KafkaSource 14 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer 15 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 16 | import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction 17 | import org.apache.flink.util.Collector 18 | import scala.jdk.CollectionConverters._ 19 | 20 | /** Use Flink's state to perform efficient record joining based on business 21 | * requirements. 22 | */ 23 | @main def example6 = 24 | val env = StreamExecutionEnvironment.getExecutionEnvironment() 25 | 26 | // switch to batch mode on demand 27 | // env.setRuntimeMode(RuntimeExecutionMode.BATCH) 28 | 29 | // read transactions 30 | val transactionSource = KafkaSource 31 | .builder[Transaction] 32 | .setBootstrapServers("localhost:9092") 33 | .setTopics("transactions") 34 | .setStartingOffsets(OffsetsInitializer.earliest()) 35 | .setValueOnlyDeserializer(new TransactionDeserializer) 36 | // .setBounded(OffsetsInitializer.latest()) 37 | .build() 38 | 39 | val transactionStream = 40 | env.fromSource( 41 | transactionSource, 42 | WatermarkStrategy.noWatermarks(), 43 | "Transactions" 44 | ) 45 | 46 | // Deduplicate using the function 47 | // defined in example 5 48 | val deduplicatedStream = 49 | transactionStream 50 | .keyBy((t: Transaction) => t.t_id) 51 | .process(new DataStreamDeduplicate) 52 | 53 | // join transactions and customers 54 | env 55 | .fromElements(ExampleData.customers: _*) 56 | .connect(deduplicatedStream) 57 | .keyBy((c: Customer) => c.c_id, (t: Transaction) => t.t_customer_id) 58 | .process(new JoinCustomersWithTransaction) 59 | .executeAndCollect 60 | .forEachRemaining(println) 61 | 62 | class JoinCustomersWithTransaction 63 | extends KeyedCoProcessFunction[Long, Customer, Transaction, String]: 64 | 65 | var customer: ValueState[Customer] = _ 66 | var transactions: ListState[Transaction] = _ 67 | 68 | override def open(parameters: Configuration): Unit = 69 | customer = getRuntimeContext.getState( 70 | new ValueStateDescriptor("customer", classOf[Customer]) 71 | ) 72 | transactions = getRuntimeContext.getListState( 73 | new ListStateDescriptor("transactions", classOf[Transaction]) 74 | ) 75 | 76 | override def processElement1( 77 | in1: Customer, 78 | context: KeyedCoProcessFunction[ 79 | Long, 80 | Customer, 81 | Transaction, 82 | String 83 | ]#Context, 84 | collector: Collector[String] 85 | ): Unit = 86 | customer.update(in1) 87 | val txs = transactions.get().asScala.to(LazyList) 88 | 89 | if !txs.isEmpty then join(collector, in1, txs) 90 | 91 | override def processElement2( 92 | in2: Transaction, 93 | context: KeyedCoProcessFunction[ 94 | Long, 95 | Customer, 96 | Transaction, 97 | String 98 | ]#Context, 99 | collector: Collector[String] 100 | ): Unit = 101 | transactions.add(in2) 102 | val c = customer.value 103 | 104 | if c != null then 105 | join(collector, c, transactions.get().asScala.to(LazyList)) 106 | 107 | private def join( 108 | out: Collector[String], 109 | c: Customer, 110 | txs: LazyList[Transaction] 111 | ) = 112 | txs.foreach(t => out.collect(s"${c.c_name} ${t.t_amount}")) 113 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_07_Table_Deduplicate_Join.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import com.ververica.models.{Customer, Transaction, TransactionDeserializer} 5 | import org.apache.flink.api.common.eventtime.WatermarkStrategy 6 | import org.apache.flink.connector.kafka.source.KafkaSource 7 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer 8 | import org.apache.flink.streaming.api.datastream.DataStream 9 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 10 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment 11 | 12 | /** Use DataStream API connectors but deduplicate and join in SQL. */ 13 | @main def example7 = 14 | val env = StreamExecutionEnvironment.getExecutionEnvironment 15 | // switch to batch mode on demand 16 | // env.setRuntimeMode(RuntimeExecutionMode.BATCH) 17 | val tableEnv = StreamTableEnvironment.create(env) 18 | 19 | val customerStream = env.fromElements(ExampleData.customers: _*) 20 | tableEnv.createTemporaryView("Customers", customerStream) 21 | 22 | // read transactions 23 | val transactionSource = KafkaSource 24 | .builder[Transaction] 25 | .setBootstrapServers("localhost:9092") 26 | .setTopics("transactions") 27 | .setStartingOffsets(OffsetsInitializer.earliest) 28 | .setValueOnlyDeserializer(new TransactionDeserializer) 29 | .build 30 | 31 | val transactionStream = env.fromSource( 32 | transactionSource, 33 | WatermarkStrategy.noWatermarks, 34 | "Transactions" 35 | ) 36 | // seamlessly switch from DataStream to Table API 37 | tableEnv.createTemporaryView("Transactions", transactionStream) 38 | 39 | // use Flink SQL to do the heavy lifting 40 | tableEnv.executeSql( 41 | """ 42 | |SELECT c_name, CAST(t_amount AS DECIMAL(5, 2)) 43 | |FROM Customers 44 | |JOIN (SELECT DISTINCT * FROM Transactions) ON c_id = t_customer_id 45 | |""".stripMargin 46 | ) 47 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_08_Table_Updating_Join.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import com.ververica.models.{Transaction, TransactionDeserializer} 5 | import org.apache.flink.api.common.eventtime.WatermarkStrategy 6 | import org.apache.flink.api.common.typeinfo.Types 7 | import org.apache.flink.connector.kafka.source.KafkaSource 8 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer 9 | import org.apache.flink.streaming.api.datastream.DataStream 10 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 11 | import org.apache.flink.table.api.Schema 12 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment 13 | import org.apache.flink.table.connector.ChangelogMode 14 | import org.apache.flink.types.Row 15 | 16 | /** Maintain a materialized view. */ 17 | @main def example8 = 18 | val env = StreamExecutionEnvironment.getExecutionEnvironment 19 | val tableEnv = StreamTableEnvironment.create(env) 20 | // read transactions 21 | val transactionSource = KafkaSource 22 | .builder[Transaction] 23 | .setBootstrapServers("localhost:9092") 24 | .setTopics("transactions") 25 | .setStartingOffsets(OffsetsInitializer.earliest) 26 | .setValueOnlyDeserializer(new TransactionDeserializer) 27 | .build 28 | 29 | val transactionStream = env.fromSource( 30 | transactionSource, 31 | WatermarkStrategy.noWatermarks, 32 | "Transactions" 33 | ) 34 | tableEnv.createTemporaryView("Transactions", transactionStream) 35 | // use a customer changelog 36 | val customerStream = env 37 | .fromElements(ExampleData.customers_with_updates: _*) 38 | .returns( 39 | Types.ROW_NAMED( 40 | Array("c_id", "c_name", "c_birthday"), 41 | Types.LONG, 42 | Types.STRING, 43 | Types.LOCAL_DATE 44 | ) 45 | ) 46 | // make it an updating view 47 | tableEnv.createTemporaryView( 48 | "Customers", 49 | tableEnv.fromChangelogStream( 50 | customerStream, 51 | Schema.newBuilder.primaryKey("c_id").build, 52 | ChangelogMode.upsert 53 | ) 54 | ) 55 | // query the changelog backed view 56 | // and thus perform materialized view maintenance 57 | tableEnv 58 | .executeSql( 59 | """ 60 | |SELECT c_name, CAST(t_amount AS DECIMAL(5, 2)) 61 | |FROM Customers 62 | |JOIN (SELECT DISTINCT * FROM Transactions) ON c_id = t_customer_id 63 | |""".stripMargin 64 | ) 65 | .print() 66 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_09_Table_Temporal_Join.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import com.ververica.models.{Transaction, TransactionDeserializer} 5 | import org.apache.flink.api.common.eventtime.WatermarkStrategy 6 | import org.apache.flink.api.common.typeinfo.Types 7 | import org.apache.flink.connector.kafka.source.KafkaSource 8 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer 9 | import org.apache.flink.streaming.api.datastream.DataStream 10 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 11 | import org.apache.flink.table.api.Schema 12 | import org.apache.flink.table.api.Table 13 | import org.apache.flink.table.api.TableConfig 14 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment 15 | import org.apache.flink.table.connector.ChangelogMode 16 | import org.apache.flink.types.Row 17 | 18 | import java.time.ZoneId 19 | 20 | /** Perform the materialized view maintenance smarter by using time-versioned 21 | * joins. 22 | */ 23 | @main def example9 = 24 | val env = StreamExecutionEnvironment.getExecutionEnvironment 25 | env.setParallelism(1) // due to little data 26 | 27 | val tableEnv = StreamTableEnvironment.create(env) 28 | val config = tableEnv.getConfig 29 | config.setLocalTimeZone(ZoneId.of("UTC")) 30 | // read transactions 31 | val transactionSource = KafkaSource 32 | .builder[Transaction] 33 | .setBootstrapServers("localhost:9092") 34 | .setTopics("transactions") 35 | .setStartingOffsets(OffsetsInitializer.earliest) 36 | .setValueOnlyDeserializer(new TransactionDeserializer) 37 | .build 38 | 39 | val transactionStream = env.fromSource( 40 | transactionSource, 41 | WatermarkStrategy.noWatermarks, 42 | "Transactions" 43 | ) 44 | tableEnv.createTemporaryView( 45 | "Transactions", 46 | transactionStream, 47 | Schema.newBuilder 48 | .columnByExpression("t_rowtime", "CAST(t_time AS TIMESTAMP_LTZ(3))") 49 | .watermark("t_rowtime", "t_rowtime - INTERVAL '10' SECONDS") 50 | .build 51 | ) 52 | 53 | val deduplicateTransactions = tableEnv.sqlQuery( 54 | """ 55 | |SELECT t_id, t_rowtime, t_customer_id, t_amount 56 | |FROM ( 57 | | SELECT * 58 | | ROW_NUMBER() OVER (PARTITION BY t_id ORDER BY t_rowtime) AS row_num 59 | | FROM Transactions) 60 | |WHERE row_num = 1 61 | |""".stripMargin 62 | ) 63 | tableEnv.createTemporaryView( 64 | "DeduplicateTransactions", 65 | deduplicateTransactions 66 | ) 67 | // use a customer changelog with timestamps 68 | val customerStream = env 69 | .fromElements(ExampleData.customers_with_temporal_updates: _*) 70 | .returns( 71 | Types.ROW_NAMED( 72 | Array("c_update_time", "c_id", "c_name", "c_birthday"), 73 | Types.INSTANT, 74 | Types.LONG, 75 | Types.STRING, 76 | Types.LOCAL_DATE 77 | ) 78 | ) 79 | // make it a temporal view 80 | tableEnv.createTemporaryView( 81 | "Customers", 82 | tableEnv.fromChangelogStream( 83 | customerStream, 84 | Schema.newBuilder 85 | .columnByExpression( 86 | "c_rowtime", 87 | "CAST(c_update_time AS TIMESTAMP_LTZ(3))" 88 | ) 89 | .primaryKey("c_id") 90 | .watermark("c_rowtime", "c_rowtime - INTERVAL '10' SECONDS") 91 | .build, 92 | ChangelogMode.upsert 93 | ) 94 | ) 95 | tableEnv 96 | .executeSql(""" 97 | |SELECT t_rowtime, c_rowtime, t_id, c_name, t_amount 98 | |FROM DeduplicateTransactions 99 | |LEFT JOIN Customers FOR SYSTEM_TIME AS OF t_rowtime 100 | | ON c_id = t_customer_id 101 | |""".stripMargin) 102 | .print() 103 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/Example_10_Table_Temporal_Join_Pure.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import org.apache.flink.configuration.CoreOptions 4 | import org.apache.flink.table.api.{ 5 | DataTypes, 6 | EnvironmentSettings, 7 | Schema, 8 | Table, 9 | TableConfig, 10 | TableDescriptor, 11 | TableEnvironment 12 | } 13 | 14 | import java.time.ZoneId 15 | 16 | /** Table API end-to-end example with time-versioned joins. */ 17 | @main def example10 = 18 | val tableEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode) 19 | val config = tableEnv.getConfig 20 | config.getConfiguration.set( 21 | CoreOptions.DEFAULT_PARALLELISM, 22 | 1 23 | ) // due to little data 24 | 25 | config.setLocalTimeZone(ZoneId.of("UTC")) 26 | 27 | // use descriptor API to use dedicated table connectors 28 | tableEnv.createTemporaryTable( 29 | "Customers", 30 | TableDescriptor 31 | .forConnector("upsert-kafka") 32 | .schema( 33 | Schema 34 | .newBuilder() 35 | .column("c_rowtime", DataTypes.TIMESTAMP_LTZ(3)) 36 | .column("c_id", DataTypes.BIGINT().notNull()) 37 | .column("c_name", DataTypes.STRING()) 38 | .column("c_birthday", DataTypes.DATE()) 39 | .primaryKey("c_id") 40 | .watermark("c_rowtime", "c_rowtime - INTERVAL '10' SECONDS") 41 | .build() 42 | ) 43 | .option("key.format", "json") 44 | .option("value.format", "json") 45 | .option("topic", "customers") 46 | .option("properties.bootstrap.servers", "localhost:9092") 47 | .build() 48 | ) 49 | 50 | tableEnv.createTemporaryTable( 51 | "Transactions", 52 | TableDescriptor 53 | .forConnector("kafka") 54 | .schema( 55 | Schema 56 | .newBuilder() 57 | .column("t_time", DataTypes.TIMESTAMP_LTZ(3)) 58 | .column("t_id", DataTypes.BIGINT().notNull()) 59 | .column("t_customer_id", DataTypes.BIGINT().notNull()) 60 | .column("t_amount", DataTypes.DECIMAL(5, 2)) 61 | .watermark("t_time", "t_time - INTERVAL '10' SECONDS") 62 | .build() 63 | ) 64 | .format("json") 65 | .option("json.timestamp-format.standard", "ISO-8601") 66 | .option("topic", "transactions") 67 | .option("scan.startup.mode", "earliest-offset") 68 | .option("properties.bootstrap.servers", "localhost:9092") 69 | .build() 70 | ) 71 | 72 | val deduplicateTransactions = tableEnv.sqlQuery( 73 | """ 74 | |SELECT t_id, t_rowtime, t_customer_id, t_amount 75 | |FROM ( 76 | | SELECT * 77 | | ROW_NUMBER() OVER (PARTITION BY t_id ORDER BY t_rowtime) AS row_num 78 | | FROM Transactions) 79 | |WHERE row_num = 1 80 | |""".stripMargin 81 | ) 82 | 83 | tableEnv.createTemporaryView( 84 | "DeduplicateTransactions", 85 | deduplicateTransactions 86 | ) 87 | tableEnv 88 | .executeSql(""" 89 | |SELECT t_rowtime, c_rowtime, t_id, c_name, t_amount 90 | |FROM DeduplicateTransactions 91 | |LEFT JOIN Customers FOR SYSTEM_TIME AS OF t_rowtime 92 | | ON c_id = t_customer_id 93 | |""".stripMargin) 94 | .print() 95 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/FillKafkaWithCustomers.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import org.apache.flink.api.common.typeinfo.Types 4 | import org.apache.flink.streaming.api.datastream.DataStream 5 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 6 | import org.apache.flink.table.api.{DataTypes, Schema, TableDescriptor} 7 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment 8 | import org.apache.flink.types.Row 9 | import org.apache.flink.types.RowKind 10 | 11 | import java.time.Instant 12 | import java.time.LocalDate 13 | 14 | /** Utility for writing exmple customer data into a Kafka topic 15 | */ 16 | @main def fillKafkaWithCustomers = 17 | val env = StreamExecutionEnvironment.getExecutionEnvironment 18 | val tableEnv = StreamTableEnvironment.create(env) 19 | val customerStream = env 20 | .fromElements( 21 | Row.ofKind( 22 | RowKind.INSERT, 23 | Instant.parse("2021-10-01T12:00:00.000Z"), 24 | 12L, 25 | "Alice", 26 | LocalDate.of(1984, 3, 12) 27 | ), 28 | Row.ofKind( 29 | RowKind.INSERT, 30 | Instant.parse("2021-10-01T12:00:00.000Z"), 31 | 32L, 32 | "Bob", 33 | LocalDate.of(1990, 10, 14) 34 | ), 35 | Row.ofKind( 36 | RowKind.INSERT, 37 | Instant.parse("2021-10-01T12:00:00.000Z"), 38 | 7L, 39 | "Kyle", 40 | LocalDate.of(1979, 2, 23) 41 | ), 42 | Row.ofKind( 43 | RowKind.UPDATE_AFTER, 44 | Instant.parse("2021-10-02T09:00:00.000Z"), 45 | 7L, 46 | "Kylie", 47 | LocalDate.of(1984, 3, 12) 48 | ), 49 | Row.ofKind( 50 | RowKind.UPDATE_AFTER, 51 | Instant.parse("2021-10-10T08:00:00.000Z"), 52 | 12L, 53 | "Aliceson", 54 | LocalDate.of(1984, 3, 12) 55 | ), 56 | Row.ofKind( 57 | RowKind.INSERT, 58 | Instant.parse("2021-10-20T12:00:00.000Z"), 59 | 77L, 60 | "Robert", 61 | LocalDate.of(2002, 7, 20) 62 | ) 63 | ) 64 | .returns( 65 | Types.ROW_NAMED( 66 | Array("c_update_time", "c_id", "c_name", "c_birthday"), 67 | Types.INSTANT, 68 | Types.LONG, 69 | Types.STRING, 70 | Types.LOCAL_DATE 71 | ) 72 | ) 73 | tableEnv 74 | .fromChangelogStream(customerStream) 75 | .executeInsert( 76 | TableDescriptor 77 | .forConnector("upsert-kafka") 78 | .schema( 79 | Schema 80 | .newBuilder() 81 | .column("c_rowtime", DataTypes.TIMESTAMP_LTZ(3)) 82 | .column("c_id", DataTypes.BIGINT().notNull()) 83 | .column("c_name", DataTypes.STRING()) 84 | .column("c_birthday", DataTypes.DATE()) 85 | .primaryKey("c_id") 86 | .watermark("c_rowtime", "c_rowtime - INTERVAL '10' SECONDS") 87 | .build() 88 | ) 89 | .option("key.format", "json") 90 | .option("value.format", "json") 91 | .option("topic", "customers") 92 | .option("properties.bootstrap.servers", "localhost:9092") 93 | .build() 94 | ) 95 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/FillKafkaWithTransactions.scala: -------------------------------------------------------------------------------- 1 | package com.ververica 2 | 3 | import com.ververica.data.ExampleData 4 | import org.apache.flink.streaming.api.datastream.DataStream 5 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment 6 | import org.apache.flink.table.api.{DataTypes, Schema, TableDescriptor} 7 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment 8 | 9 | /** Utility for writing exmple transaction data into a Kafka topic 10 | */ 11 | @main def fillKafkaWithTransactions = 12 | val env = StreamExecutionEnvironment.getExecutionEnvironment 13 | val tableEnv = StreamTableEnvironment.create(env) 14 | val transactionStream = env.fromElements(ExampleData.transaction: _*) 15 | tableEnv 16 | .fromDataStream(transactionStream) 17 | .executeInsert( 18 | TableDescriptor 19 | .forConnector("kafka") 20 | .schema( 21 | Schema 22 | .newBuilder() 23 | .column("t_time", DataTypes.TIMESTAMP_LTZ(3)) 24 | .column("t_id", DataTypes.BIGINT().notNull()) 25 | .column("t_customer_id", DataTypes.BIGINT().notNull()) 26 | .column("t_amount", DataTypes.DECIMAL(5, 2)) 27 | .watermark("t_time", "t_time - INTERVAL '10' SECONDS") 28 | .build() 29 | ) 30 | .format("json") 31 | .option("json.timestamp-format.standard", "ISO-8601") 32 | .option("topic", "transactions") 33 | .option("scan.startup.mode", "earliest-offset") 34 | .option("properties.bootstrap.servers", "localhost:9092") 35 | .build() 36 | ) 37 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/data/ExampleData.scala: -------------------------------------------------------------------------------- 1 | package com.ververica.data 2 | 3 | import com.ververica.models.{Transaction, Customer} 4 | import java.time.{LocalDate, Instant} 5 | import java.math.BigDecimal 6 | import org.apache.flink.types.Row 7 | import org.apache.flink.types.RowKind 8 | 9 | object ExampleData: 10 | val transaction = Array( 11 | new Transaction( 12 | Instant.parse("2021-10-08T12:33:12.000Z"), 13 | 1L, 14 | 12L, 15 | new BigDecimal("325.12") 16 | ), 17 | new Transaction( 18 | Instant.parse("2021-10-10T08:00:00.000Z"), 19 | 2L, 20 | 7L, 21 | new BigDecimal("13.99") 22 | ), 23 | new Transaction( 24 | Instant.parse("2021-10-10T08:00:00.000Z"), 25 | 2L, 26 | 7L, 27 | new BigDecimal("13.99") 28 | ), 29 | new Transaction( 30 | Instant.parse("2021-10-14T17:04:00.000Z"), 31 | 3L, 32 | 12L, 33 | new BigDecimal("52.48") 34 | ), 35 | new Transaction( 36 | Instant.parse("2021-10-14T17:06:00.000Z"), 37 | 4L, 38 | 32L, 39 | new BigDecimal("26.11") 40 | ), 41 | new Transaction( 42 | Instant.parse("2021-10-14T18:23:00.000Z"), 43 | 5L, 44 | 32L, 45 | new BigDecimal("22.03") 46 | ) 47 | ) 48 | 49 | val customers = Array( 50 | new Customer(12L, "Alice", LocalDate.of(1984, 3, 12)), 51 | new Customer(32L, "Bob", LocalDate.of(1990, 10, 14)), 52 | new Customer(7L, "Kyle", LocalDate.of(1979, 2, 23)) 53 | ) 54 | 55 | val customers_with_updates = Array( 56 | Row.ofKind(RowKind.INSERT, 12L, "Alice", LocalDate.of(1984, 3, 12)), 57 | Row.ofKind(RowKind.INSERT, 32L, "Bob", LocalDate.of(1990, 10, 14)), 58 | Row.ofKind(RowKind.INSERT, 7L, "Kyle", LocalDate.of(1979, 2, 23)), 59 | Row.ofKind(RowKind.UPDATE_AFTER, 7L, "Kylie", LocalDate.of(1984, 3, 12)), 60 | Row 61 | .ofKind(RowKind.UPDATE_AFTER, 12L, "Aliceson", LocalDate.of(1984, 3, 12)), 62 | Row.ofKind(RowKind.INSERT, 77L, "Robert", LocalDate.of(1984, 3, 12)) 63 | ) 64 | 65 | val customers_with_temporal_updates = Array( 66 | Row.ofKind( 67 | RowKind.INSERT, 68 | Instant.parse("2021-10-01T12:00:00.000Z"), 69 | 12L, 70 | "Alice", 71 | LocalDate.of(1984, 3, 12) 72 | ), 73 | Row.ofKind( 74 | RowKind.INSERT, 75 | Instant.parse("2021-10-01T12:00:00.000Z"), 76 | 32L, 77 | "Bob", 78 | LocalDate.of(1990, 10, 14) 79 | ), 80 | Row.ofKind( 81 | RowKind.INSERT, 82 | Instant.parse("2021-10-01T12:00:00.000Z"), 83 | 7L, 84 | "Kyle", 85 | LocalDate.of(1979, 2, 23) 86 | ), 87 | Row.ofKind( 88 | RowKind.UPDATE_AFTER, 89 | Instant.parse("2021-10-02T09:00:00.000Z"), 90 | 7L, 91 | "Kylie", 92 | LocalDate.of(1984, 3, 12) 93 | ), 94 | Row.ofKind( 95 | RowKind.UPDATE_AFTER, 96 | Instant.parse("2021-10-10T08:00:00.000Z"), 97 | 12L, 98 | "Aliceson", 99 | LocalDate.of(1984, 3, 12) 100 | ), 101 | Row.ofKind( 102 | RowKind.INSERT, 103 | Instant.parse("2021-10-20T12:00:00.000Z"), 104 | 77L, 105 | "Robert", 106 | LocalDate.of(2002, 7, 20) 107 | ) 108 | ) 109 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/models/Customer.scala: -------------------------------------------------------------------------------- 1 | package com.ververica.models 2 | 3 | import little.json.* 4 | import little.json.Implicits.{*, given} 5 | import org.apache.flink.api.common.serialization.DeserializationSchema 6 | import org.apache.flink.api.common.typeinfo.TypeInformation 7 | 8 | import java.time.LocalDate 9 | import java.util.Objects 10 | import scala.language.implicitConversions 11 | 12 | class Customer(var c_id: Long, var c_name: String, var c_birthday: LocalDate): 13 | def this() = 14 | this(0L, "", null) 15 | 16 | override def toString: String = s"Customer($c_id, $c_name, $c_birthday)" 17 | 18 | given jsonToLocalDate: JsonInput[LocalDate] with 19 | def apply(json: JsonValue) = LocalDate.parse(json.as[String]) 20 | 21 | given jsonToCustomer: JsonInput[Customer] with 22 | def apply(json: JsonValue) = 23 | val c = new Customer() 24 | c.c_id = json("c_id") 25 | c.c_name = json("c_name") 26 | c.c_birthday = json("c_birthday") 27 | c 28 | 29 | class CustomerDeserializer extends DeserializationSchema[Customer]: 30 | override def isEndOfStream(customer: Customer): Boolean = false 31 | 32 | override def getProducedType: TypeInformation[Customer] = 33 | TypeInformation.of(classOf[Customer]) 34 | 35 | override def deserialize(bytes: Array[Byte]): Customer = 36 | Json.parse(bytes).as[Customer] 37 | -------------------------------------------------------------------------------- /src/main/scala/com/ververica/models/Transaction.scala: -------------------------------------------------------------------------------- 1 | package com.ververica.models 2 | 3 | import little.json.* 4 | import little.json.Implicits.{*, given} 5 | import org.apache.flink.api.common.serialization.DeserializationSchema 6 | import org.apache.flink.api.common.typeinfo.TypeInformation 7 | 8 | import java.time.Instant 9 | import java.math.BigDecimal 10 | import scala.language.implicitConversions 11 | 12 | class Transaction( 13 | var t_time: Instant, 14 | var t_id: Long, 15 | var t_customer_id: Long, 16 | var t_amount: BigDecimal 17 | ): 18 | def this() = 19 | this(null, 0L, 0L, null) 20 | 21 | override def toString: String = 22 | s"Transaction($t_time, $t_id, $t_customer_id, $t_amount)" 23 | 24 | given jsonToInstant: JsonInput[Instant] with 25 | def apply(json: JsonValue) = Instant.parse(json.as[String]) 26 | 27 | given jsonToBigDecimal: JsonInput[BigDecimal] with 28 | def apply(json: JsonValue) = new BigDecimal(json.as[String]) 29 | 30 | given jsonToTransaction: JsonInput[Transaction] with 31 | def apply(json: JsonValue) = 32 | val t = new Transaction() 33 | t.t_time = json("t_time") 34 | t.t_id = json("t_id") 35 | t.t_customer_id = json("t_customer_id") 36 | t.t_amount = json("t_amount") 37 | t 38 | 39 | class TransactionDeserializer extends DeserializationSchema[Transaction]: 40 | override def isEndOfStream(customer: Transaction): Boolean = false 41 | 42 | override def getProducedType: TypeInformation[Transaction] = 43 | TypeInformation.of(classOf[Transaction]) 44 | 45 | override def deserialize(bytes: Array[Byte]): Transaction = 46 | Json.parse(bytes).as[Transaction] 47 | --------------------------------------------------------------------------------