├── .git-blame-ignore-revs
├── .gitattributes
├── .github
└── workflows
│ └── ci.yml
├── .gitignore
├── .scala-steward.conf
├── .scalafmt.conf
├── LICENSE.md
├── README.md
├── build.sbt
├── modules
├── examples
│ ├── notebooks
│ │ └── word_count_example.ipynb
│ ├── scripts
│ │ ├── PubSubConnectorWithJson.sc
│ │ ├── debug-sql.sc
│ │ ├── flink-amm.sc
│ │ ├── flink-scala-cli.scala
│ │ ├── gen-csv-file.sc
│ │ ├── gen-kafka-data.sc
│ │ ├── hybrid-source.sc
│ │ └── logback.xml
│ └── src
│ │ ├── main
│ │ ├── protobuf
│ │ │ └── simple.proto
│ │ ├── resources
│ │ │ ├── cities.csv
│ │ │ └── logback.xml
│ │ └── scala
│ │ │ └── org
│ │ │ └── example
│ │ │ ├── Job.scala
│ │ │ ├── SocketTextStreamWordCount.scala
│ │ │ ├── TransactonIOs.scala
│ │ │ ├── connectedStreams.scala
│ │ │ ├── fileFilter.scala
│ │ │ ├── fraud
│ │ │ ├── FraudDetectionJob.scala
│ │ │ ├── FraudDetector.scala
│ │ │ └── RunningAverage.scala
│ │ │ ├── runningSum.scala
│ │ │ ├── troubleshooting
│ │ │ ├── fakeKafkaSource.scala
│ │ │ ├── measurements.scala
│ │ │ └── troubleshootingExample.scala
│ │ │ └── wordCount.scala
│ │ └── test
│ │ └── scala
│ │ └── org
│ │ └── example
│ │ ├── ConnectedStreamsTest.scala
│ │ ├── CustomTriggerTests.scala
│ │ ├── MyKeyedProcessFunctionTest.scala
│ │ └── fraud
│ │ ├── FakeRuntimeContext.scala
│ │ └── FraudDetectorTest.scala
├── flink-1-api
│ └── src
│ │ ├── main
│ │ ├── scala-2
│ │ │ └── org
│ │ │ │ └── apache
│ │ │ │ └── flinkx
│ │ │ │ └── api
│ │ │ │ └── LowPrioImplicits.scala
│ │ ├── scala-3
│ │ │ └── org
│ │ │ │ └── apache
│ │ │ │ └── flinkx
│ │ │ │ └── api
│ │ │ │ └── LowPrioImplicits.scala
│ │ └── scala
│ │ │ └── org
│ │ │ └── apache
│ │ │ ├── flink
│ │ │ └── streaming
│ │ │ │ └── util
│ │ │ │ └── typeutils
│ │ │ │ └── DefaultScalaProductFieldAccessorFactory.scala
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ ├── AllWindowedStream.scala
│ │ │ ├── AsyncDataStream.scala
│ │ │ ├── BroadcastConnectedStream.scala
│ │ │ ├── CloseableIterator.scala
│ │ │ ├── ClosureCleaner.scala
│ │ │ ├── CoGroupedStreams.scala
│ │ │ ├── ConnectedStreams.scala
│ │ │ ├── DataStream.scala
│ │ │ ├── DataStreamUtils.scala
│ │ │ ├── JoinedStreams.scala
│ │ │ ├── KeyedStream.scala
│ │ │ ├── OutputTag.scala
│ │ │ ├── ScalaStreamOps.scala
│ │ │ ├── StreamExecutionEnvironment.scala
│ │ │ ├── WindowedStream.scala
│ │ │ ├── async
│ │ │ ├── AsyncFunction.scala
│ │ │ ├── JavaResultFutureWrapper.scala
│ │ │ ├── ResultFuture.scala
│ │ │ ├── RichAsyncFunction.scala
│ │ │ └── ScalaRichAsyncFunctionWrapper.scala
│ │ │ ├── conv.scala
│ │ │ ├── extensions
│ │ │ ├── impl
│ │ │ │ └── acceptPartialFunctions
│ │ │ │ │ ├── OnConnectedStream.scala
│ │ │ │ │ ├── OnDataStream.scala
│ │ │ │ │ ├── OnJoinedStream.scala
│ │ │ │ │ ├── OnKeyedStream.scala
│ │ │ │ │ └── OnWindowedStream.scala
│ │ │ └── ops.scala
│ │ │ ├── function
│ │ │ ├── AllWindowFunction.scala
│ │ │ ├── ProcessAllWindowFunction.scala
│ │ │ ├── ProcessWindowFunction.scala
│ │ │ ├── RichAllWindowFunction.scala
│ │ │ ├── RichWindowFunction.scala
│ │ │ ├── StatefulFunction.scala
│ │ │ ├── WindowFunction.scala
│ │ │ └── util
│ │ │ │ ├── ScalaAllWindowFunction.scala
│ │ │ │ ├── ScalaAllWindowFunctionWrapper.scala
│ │ │ │ ├── ScalaProcessWindowFunctionWrapper.scala
│ │ │ │ ├── ScalaReduceFunction.scala
│ │ │ │ ├── ScalaWindowFunction.scala
│ │ │ │ └── ScalaWindowFunctionWrapper.scala
│ │ │ ├── mapper
│ │ │ ├── BigDecMapper.scala
│ │ │ ├── BigIntMapper.scala
│ │ │ └── UuidMapper.scala
│ │ │ ├── serializer
│ │ │ ├── ArraySerializer.scala
│ │ │ ├── CollectionSerializerSnapshot.scala
│ │ │ ├── CoproductSerializer.scala
│ │ │ ├── ListCCSerializer.scala
│ │ │ ├── ListSerializer.scala
│ │ │ ├── MapSerializer.scala
│ │ │ ├── MappedSerializer.scala
│ │ │ ├── ScalaCaseObjectSerializer.scala
│ │ │ ├── SeqSerializer.scala
│ │ │ ├── SetSerializer.scala
│ │ │ └── VectorSerializer.scala
│ │ │ ├── serializers.scala
│ │ │ └── typeinfo
│ │ │ ├── CaseClassTypeInfo.scala
│ │ │ ├── CollectionTypeInformation.scala
│ │ │ ├── CoproductTypeInformation.scala
│ │ │ ├── EitherTypeInfo.scala
│ │ │ ├── MappedTypeInformation.scala
│ │ │ ├── OptionTypeInfo.scala
│ │ │ ├── ProductTypeInformation.scala
│ │ │ ├── SimpleTypeInformation.scala
│ │ │ └── UnitTypeInformation.scala
│ │ └── test
│ │ ├── resources
│ │ ├── click.dat
│ │ ├── logback.xml
│ │ ├── old-serializer-snapshot.dat
│ │ └── without-arity-test.dat
│ │ ├── scala-2
│ │ └── org
│ │ │ └── apache
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ └── GenericCaseClassScala2Test.scala
│ │ ├── scala-3
│ │ └── org
│ │ │ └── apache
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ ├── GenericCaseClassScala3Test.scala
│ │ │ └── Scala3EnumTest.scala
│ │ └── scala
│ │ └── org
│ │ └── apache
│ │ └── flinkx
│ │ └── api
│ │ ├── AnyTest.scala
│ │ ├── AsyncDataStreamTest.scala
│ │ ├── CatsTest.scala
│ │ ├── CoGroupedStreamsTest.scala
│ │ ├── DataStreamTest.scala
│ │ ├── ExampleTest.scala
│ │ ├── IntegrationTest.scala
│ │ ├── IntegrationTestSink.scala
│ │ ├── JoinedStreamsTest.scala
│ │ ├── MappedTypeInfoTest.scala
│ │ ├── ProcessFunctionTest.scala
│ │ ├── SchemaEvolutionTest.scala
│ │ ├── SerializerSnapshotTest.scala
│ │ ├── SerializerTest.scala
│ │ ├── StreamExecutionEnvironmentTest.scala
│ │ ├── TestUtils.scala
│ │ ├── TypeInfoTest.scala
│ │ └── serializer
│ │ ├── ArraySerializerTest.scala
│ │ └── CoproductSerializerTest.scala
├── flink-2-api
│ └── src
│ │ ├── main
│ │ ├── scala-2
│ │ │ └── org
│ │ │ │ └── apache
│ │ │ │ └── flinkx
│ │ │ │ └── api
│ │ │ │ └── LowPrioImplicits.scala
│ │ ├── scala-3
│ │ │ └── org
│ │ │ │ └── apache
│ │ │ │ └── flinkx
│ │ │ │ └── api
│ │ │ │ └── LowPrioImplicits.scala
│ │ └── scala
│ │ │ └── org
│ │ │ └── apache
│ │ │ ├── flink
│ │ │ └── streaming
│ │ │ │ └── util
│ │ │ │ └── typeutils
│ │ │ │ └── DefaultScalaProductFieldAccessorFactory.scala
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ ├── AllWindowedStream.scala
│ │ │ ├── AsyncDataStream.scala
│ │ │ ├── BroadcastConnectedStream.scala
│ │ │ ├── CloseableIterator.scala
│ │ │ ├── ClosureCleaner.scala
│ │ │ ├── CoGroupedStreams.scala
│ │ │ ├── ConnectedStreams.scala
│ │ │ ├── DataStream.scala
│ │ │ ├── DataStreamUtils.scala
│ │ │ ├── JoinedStreams.scala
│ │ │ ├── KeyedStream.scala
│ │ │ ├── OutputTag.scala
│ │ │ ├── ScalaStreamOps.scala
│ │ │ ├── StreamExecutionEnvironment.scala
│ │ │ ├── WindowedStream.scala
│ │ │ ├── async
│ │ │ ├── AsyncFunction.scala
│ │ │ ├── JavaResultFutureWrapper.scala
│ │ │ ├── ResultFuture.scala
│ │ │ ├── RichAsyncFunction.scala
│ │ │ └── ScalaRichAsyncFunctionWrapper.scala
│ │ │ ├── conv.scala
│ │ │ ├── extensions
│ │ │ ├── impl
│ │ │ │ └── acceptPartialFunctions
│ │ │ │ │ ├── OnConnectedStream.scala
│ │ │ │ │ ├── OnDataStream.scala
│ │ │ │ │ ├── OnJoinedStream.scala
│ │ │ │ │ ├── OnKeyedStream.scala
│ │ │ │ │ └── OnWindowedStream.scala
│ │ │ └── ops.scala
│ │ │ ├── function
│ │ │ ├── AllWindowFunction.scala
│ │ │ ├── ProcessAllWindowFunction.scala
│ │ │ ├── ProcessWindowFunction.scala
│ │ │ ├── RichAllWindowFunction.scala
│ │ │ ├── RichWindowFunction.scala
│ │ │ ├── StatefulFunction.scala
│ │ │ ├── WindowFunction.scala
│ │ │ └── util
│ │ │ │ ├── ScalaAllWindowFunction.scala
│ │ │ │ ├── ScalaAllWindowFunctionWrapper.scala
│ │ │ │ ├── ScalaProcessWindowFunctionWrapper.scala
│ │ │ │ ├── ScalaReduceFunction.scala
│ │ │ │ ├── ScalaWindowFunction.scala
│ │ │ │ └── ScalaWindowFunctionWrapper.scala
│ │ │ ├── mapper
│ │ │ ├── BigDecMapper.scala
│ │ │ ├── BigIntMapper.scala
│ │ │ └── UuidMapper.scala
│ │ │ ├── serializer
│ │ │ ├── ArraySerializer.scala
│ │ │ ├── CollectionSerializerSnapshot.scala
│ │ │ ├── CoproductSerializer.scala
│ │ │ ├── ListCCSerializer.scala
│ │ │ ├── ListSerializer.scala
│ │ │ ├── MapSerializer.scala
│ │ │ ├── MappedSerializer.scala
│ │ │ ├── ScalaCaseObjectSerializer.scala
│ │ │ ├── SeqSerializer.scala
│ │ │ ├── SetSerializer.scala
│ │ │ └── VectorSerializer.scala
│ │ │ ├── serializers.scala
│ │ │ └── typeinfo
│ │ │ ├── CaseClassTypeInfo.scala
│ │ │ ├── CollectionTypeInformation.scala
│ │ │ ├── CoproductTypeInformation.scala
│ │ │ ├── EitherTypeInfo.scala
│ │ │ ├── MappedTypeInformation.scala
│ │ │ ├── OptionTypeInfo.scala
│ │ │ ├── ProductTypeInformation.scala
│ │ │ ├── SimpleTypeInformation.scala
│ │ │ └── UnitTypeInformation.scala
│ │ └── test
│ │ └── scala
│ │ └── org
│ │ └── apache
│ │ └── flinkx
│ │ └── api
│ │ └── serializer
│ │ ├── CollectionSerializerSnapshotTest.scala
│ │ ├── CoproductSerializerSnapshotTest.scala
│ │ ├── MapSerializerSnapshotTest.scala
│ │ └── MappedSerializerSnapshotTest.scala
└── flink-common-api
│ └── src
│ ├── main
│ ├── java
│ │ └── org
│ │ │ └── apache
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ └── serializer
│ │ │ ├── ScalaCaseClassSerializerSnapshot.java
│ │ │ ├── ScalaEitherSerializerSnapshot.java
│ │ │ └── ScalaOptionSerializerSnapshot.java
│ ├── scala-2
│ │ └── org
│ │ │ └── apache
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ └── serializer
│ │ │ └── ConstructorCompat.scala
│ ├── scala-3
│ │ └── org
│ │ │ └── apache
│ │ │ └── flinkx
│ │ │ └── api
│ │ │ ├── TaggedDerivation.scala
│ │ │ ├── TypeTag.scala
│ │ │ ├── TypeTagMacro.scala
│ │ │ └── serializer
│ │ │ └── ConstructorCompat.scala
│ └── scala
│ │ └── org
│ │ └── apache
│ │ └── flinkx
│ │ └── api
│ │ ├── serializer
│ │ ├── CaseClassSerializer.scala
│ │ ├── EitherSerializer.scala
│ │ ├── ImmutableSerializer.scala
│ │ ├── MutableSerializer.scala
│ │ ├── NothingSerializer.scala
│ │ ├── OptionSerializer.scala
│ │ └── UnitSerializer.scala
│ │ ├── typeinfo
│ │ ├── CaseClassComparator.scala
│ │ ├── FailFastTypeInfoFactory.scala
│ │ └── OptionTypeComparator.scala
│ │ └── util
│ │ └── ClassUtil.scala
│ └── test
│ └── scala
│ └── org
│ └── apache
│ └── flinkx
│ └── api
│ ├── serializer
│ └── CaseClassSerializerTest.scala
│ └── util
│ └── ClassUtilTest.scala
├── project
├── build.properties
└── plugins.sbt
├── release.sh
└── version.sbt
/.git-blame-ignore-revs:
--------------------------------------------------------------------------------
1 | # Scala Steward: Reformat with scalafmt 3.5.9
2 | a19cdcb35801ca76ccc2d7263f5170309e2d266e
3 |
4 | # Scala Steward: Reformat with scalafmt 3.7.7
5 | a744067645fb512553ada2007190487c9019fb86
6 |
7 | # Scala Steward: Reformat with scalafmt 3.7.13
8 | 0c01ba437ef9e9590ffd52557c3e2242e5c7187c
9 |
10 | # Scala Steward: Reformat with scalafmt 3.7.15
11 | cd88dbd39d341e4d83dbaed57b1fe497159ba0be
12 |
13 | # Scala Steward: Reformat with scalafmt 3.8.0
14 | ac01058463c34eb216d3d3c0500dfce038fc4cff
15 |
16 | # Scala Steward: Reformat with scalafmt 3.8.1
17 | fade5fe3272078547e5ca2a7ef80a085a8d24b24
18 |
19 | # Scala Steward: Reformat with scalafmt 3.8.2
20 | 0931a26c8728f0f02202ce0a73085e6034110dd4
21 |
22 | # Scala Steward: Reformat with scalafmt 3.8.3
23 | 263d8f24393c0607dddc5a3b8cd2fbc1f0de0141
24 |
25 | # Scala Steward: Reformat with scalafmt 3.9.0
26 | 5428c1b0fd387b524dea6571e34ab17c5df39760
27 |
28 | # Scala Steward: Reformat with scalafmt 3.9.1
29 | ae9aadfa2b4779b5096a9004d6d7f5d9db2c2c7f
30 |
31 | # Scala Steward: Reformat with scalafmt 3.9.5
32 | 86e22a329b56041cf014ca16e78ab03e8fd7df00
33 |
34 | # Scala Steward: Reformat with scalafmt 3.9.6
35 | 9fd34d49283551ea5253cbe6177a10661be715d3
36 |
37 | # Scala Steward: Reformat with scalafmt 3.9.7
38 | a82026bb66f0e17cf76f2d91c6ea286431bbbd31
39 |
--------------------------------------------------------------------------------
/.gitattributes:
--------------------------------------------------------------------------------
1 | *.tsv filter=lfs diff=lfs merge=lfs -text
2 | *.gz filter=lfs diff=lfs merge=lfs -text
3 | *.json filter=lfs diff=lfs merge=lfs -text
4 | *.dat filter=lfs diff=lfs merge=lfs -text
5 | *.jpg filter=lfs diff=lfs merge=lfs -text
6 | *.svg filter=lfs diff=lfs merge=lfs -text
7 | *.png filter=lfs diff=lfs merge=lfs -text
8 |
--------------------------------------------------------------------------------
/.github/workflows/ci.yml:
--------------------------------------------------------------------------------
1 | # This workflow will build a Java project with Maven
2 | # For more information see: https://help.github.com/actions/language-and-framework-guides/building-and-testing-java-with-maven
3 |
4 | name: CI
5 |
6 | on:
7 | push:
8 | branches: [ master ]
9 | pull_request:
10 | branches: [ master ]
11 |
12 | jobs:
13 | build:
14 | runs-on: ubuntu-22.04
15 | strategy:
16 | matrix:
17 | java: [11]
18 | scala: [2.13.16, 3.3.6]
19 | flink: [1.18.1, 1.19.1]
20 | sbt-module: ['flink-1-api']
21 | include:
22 | - scala: 3.3.6
23 | java: 17
24 | flink: 1.20.0
25 | sbt-module: 'flink-1-api'
26 | - scala: 3.3.6
27 | java: 17
28 | flink: 2.0.0
29 | sbt-module: 'flink-2-api'
30 | env:
31 | JAVA_OPTIONS: '--add-opens java.base/java.lang=ALL-UNNAMED'
32 | steps:
33 | - uses: actions/checkout@v3
34 | with:
35 | fetch-depth: 0
36 | - name: Set up JDK
37 | uses: actions/setup-java@v3
38 | with:
39 | distribution: temurin
40 | java-version: ${{ matrix.java }}
41 | cache: sbt
42 | - name: Compile Docs
43 | run: JAVA_OPTS=$JAVA_OPTIONS sbt "++ ${{ matrix.scala }} docs/mdoc"
44 | - name: Run tests on examples
45 | # always running on Scala 3.x version by default
46 | if: ${{ !startsWith(matrix.flink, '1.18') && !startsWith(matrix.flink, '2.') }}
47 | run: JAVA_OPTS=$JAVA_OPTIONS sbt -DflinkVersion1=${{ matrix.flink }} "project examples; test"
48 | - name: Run tests on Flink API
49 | run: JAVA_OPTS=$JAVA_OPTIONS sbt -DflinkVersion1=${{ matrix.flink }} -DflinkVersion2=${{ matrix.flink }} "++ ${{ matrix.scala }}; project ${{ matrix.sbt-module }}; test"
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | .idea
2 | project/project
3 | project/target
4 | target
5 | .DS_STORE
6 | .git
7 | .bsp
8 | .run
9 | .bloop
10 | .metals
11 | .vscode
12 | metals.sbt
13 | .scala-build
14 | .ammonite/*
15 | *.jar
16 | *checkpoint
17 | sink-table
18 | *.iml
--------------------------------------------------------------------------------
/.scala-steward.conf:
--------------------------------------------------------------------------------
1 | updates.ignore = [
2 | { groupId = "org.apache.flink" }
3 | ]
--------------------------------------------------------------------------------
/.scalafmt.conf:
--------------------------------------------------------------------------------
1 | style = defaultWithAlign
2 | maxColumn = 120
3 | version = 3.9.7
4 | assumeStandardLibraryStripMargin = true
5 | align.stripMargin = true
6 | runner.dialect = scala3
--------------------------------------------------------------------------------
/modules/examples/scripts/debug-sql.sc:
--------------------------------------------------------------------------------
1 | import $ivy.`org.flinkextended::flink-scala-api:1.18.1_1.2.4`
2 |
3 | import $ivy.`org.apache.flink:flink-clients:1.18.1`
4 |
5 | import $ivy.`org.apache.flink:flink-streaming-scala_2.12:1.18.1`
6 |
7 | import $ivy.`org.apache.flink:flink-table-api-java:1.18.1`
8 | import $ivy.`org.apache.flink:flink-table-api-java-bridge:1.18.1`
9 | import $ivy.`org.apache.flink:flink-table-runtime:1.18.1`
10 | import $ivy.`org.apache.flink:flink-table-planner_2.12:1.18.1`
11 |
12 | import org.apache.flink.table.api._
13 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment
14 | import org.apache.flink.connector.datagen.table.DataGenConnectorOptions
15 |
16 | import org.apache.flinkx.api._
17 | import org.apache.flinkx.api.serializers._
18 |
19 | import java.lang.{Long => JLong}
20 |
21 | val env = StreamExecutionEnvironment.getExecutionEnvironment
22 | val tEnv = StreamTableEnvironment.create(env.getJavaEnv)
23 |
24 | val settings = EnvironmentSettings.newInstance().inStreamingMode().build()
25 |
26 | val table = TableEnvironment.create(settings)
27 |
28 | table.createTemporaryTable(
29 | "SourceTable",
30 | TableDescriptor
31 | .forConnector("datagen")
32 | .schema(
33 | Schema.newBuilder
34 | .column("BookId", DataTypes.INT())
35 | .build
36 | )
37 | .option(DataGenConnectorOptions.ROWS_PER_SECOND, new JLong(1))
38 | .build
39 | )
40 |
41 | val tableDescriptor = TableDescriptor
42 | .forConnector("datagen")
43 | .schema(
44 | Schema.newBuilder
45 | .column("id", DataTypes.INT.notNull)
46 | .column("a", DataTypes.ROW(DataTypes.FIELD("np", DataTypes.INT.notNull())).notNull())
47 | .build
48 | )
49 | .build
50 | table.createTemporaryTable("t1", tableDescriptor)
51 | table.createTemporaryTable("t2", tableDescriptor)
52 | // table.dropTemporaryTable("t1")
53 | // table.dropTemporaryTable("t2")
54 |
55 | val res = table.executeSql(
56 | "EXPLAIN SELECT a.id, COALESCE(a.a.np, b.a.np) c1, IFNULL(a.a.np, b.a.np) c2 FROM t1 a left JOIN t2 b ON a.id=b.id where a.a is null or a.a.np is null"
57 | )
58 | res.print
59 |
--------------------------------------------------------------------------------
/modules/examples/scripts/flink-amm.sc:
--------------------------------------------------------------------------------
1 | import $cp.lib.`flink-faker-0.4.0.jar`
2 |
3 | import $ivy.`org.flinkextended::flink-scala-api:1.15.4_1.0.0`
4 | import $ivy.`org.apache.flink:flink-clients:1.15.2`
5 | import $ivy.`org.apache.flink:flink-csv:1.15.2`
6 | import $ivy.`org.apache.flink:flink-table-api-java:1.15.2`
7 | import $ivy.`org.apache.flink:flink-table-api-java-bridge:1.15.2`
8 | import $ivy.`org.apache.flink:flink-table-runtime:1.15.2`
9 | import $ivy.`org.apache.flink:flink-table-planner-loader:1.15.2`
10 |
11 | import org.apache.flink.table.api._
12 | import org.apache.flink.table.api.bridge.java.StreamTableEnvironment
13 | import org.apache.flink.connector.datagen.table.DataGenConnectorOptions
14 |
15 | import org.apache.flink.api._
16 | import org.apache.flink.api.serializers._
17 |
18 | import _root_.java.lang.{Long => JLong}
19 |
20 | val env = StreamExecutionEnvironment.getExecutionEnvironment
21 | val tEnv = StreamTableEnvironment.create(env.getJavaEnv)
22 | val settings = EnvironmentSettings.newInstance().inStreamingMode().build()
23 | val table = TableEnvironment.create(settings)
24 |
25 | val tableDescriptor = TableDescriptor
26 | .forConnector("faker")
27 | .schema(
28 | Schema.newBuilder
29 | .column(
30 | "id",
31 | DataTypes.INT // .notNull
32 | )
33 | .column(
34 | "a",
35 | DataTypes.ROW(DataTypes.FIELD("np", DataTypes.INT))
36 | )
37 | .build
38 | )
39 | .option("fields.id.expression", "#{number.numberBetween '0','10'}")
40 | .option("fields.a.np.expression", "#{number.numberBetween '20','30'}")
41 | // .option("fields.a.np.null-rate", "0.5")
42 | .option("fields.a.null-rate", "0.5")
43 | .option("rows-per-second", "50")
44 | .build
45 | table.createTemporaryTable("t1", tableDescriptor)
46 | // table.dropTemporaryTable("t1")
47 |
48 | val res = table.executeSql(
49 | "SELECT a.id, COALESCE(a.a.np, a.id) c1, IFNULL(a.a.np, a.id) c2, a.a.np FROM t1 a"
50 | // "show create table t1"
51 | )
52 | res.print
53 |
--------------------------------------------------------------------------------
/modules/examples/scripts/flink-scala-cli.scala:
--------------------------------------------------------------------------------
1 | //> using dep "org.flinkextended::flink-scala-api:1.18.1_1.2.4"
2 | //> using dep "org.apache.flink:flink-clients:1.18.1"
3 |
4 | import org.apache.flinkx.api.*
5 | import org.apache.flinkx.api.serializers.*
6 | import org.slf4j.LoggerFactory
7 | import java.io.File
8 |
9 | @main def wordCountExample =
10 | val logger = LoggerFactory.getLogger(this.getClass())
11 | val files = File(".").listFiles ++ Option(File("/flink/lib/").listFiles)
12 | .getOrElse(Array.empty[File])
13 | val elems = files.filter(_.isFile).map(_.getAbsolutePath())
14 |
15 | val env = StreamExecutionEnvironment.getExecutionEnvironment
16 | val text = env.fromElements(elems*)
17 |
18 | text.addSink(logger.info(_))
19 |
20 | env.execute("wordCount")
21 |
--------------------------------------------------------------------------------
/modules/examples/scripts/gen-csv-file.sc:
--------------------------------------------------------------------------------
1 | //> using dep "org.flinkextended::flink-scala-api-1:1.2.7"
2 | //> using dep "org.apache.flink:flink-clients:1.20.1"
3 | //> using dep "org.apache.flink:flink-csv:1.20.1"
4 | //> using dep "org.apache.flink:flink-connector-files:1.20.1"
5 | //> using dep "org.apache.flink:flink-table-runtime:1.20.1"
6 | //> using dep "org.apache.flink:flink-table-planner-loader:1.20.1"
7 |
8 | import org.apache.flink.table.api._
9 | import org.apache.flink.connector.datagen.table.DataGenConnectorOptions
10 | import org.apache.flinkx.api._
11 | import org.apache.flinkx.api.serializers._
12 |
13 | import java.lang.{Long => JLong}
14 |
15 | val env = StreamExecutionEnvironment.getExecutionEnvironment
16 | val settings = EnvironmentSettings.newInstance.inStreamingMode.build
17 | val table = TableEnvironment.create(settings)
18 | val schema = Schema.newBuilder
19 | .column("id", DataTypes.INT())
20 | .column("bid_price", DataTypes.DOUBLE())
21 | .column("order_time", DataTypes.TIMESTAMP(2))
22 | .build
23 |
24 | table.createTemporaryTable(
25 | "SourceTable",
26 | TableDescriptor
27 | .forConnector("datagen")
28 | .schema(schema)
29 | .option(DataGenConnectorOptions.NUMBER_OF_ROWS, JLong(1000))
30 | .option("fields.id.kind", "sequence")
31 | .option("fields.id.start", "1")
32 | .option("fields.id.end", "10000")
33 | .build
34 | )
35 |
36 | val currentDirectory = java.io.File(".").getCanonicalPath
37 |
38 | table.createTemporaryTable(
39 | "SinkTable",
40 | TableDescriptor
41 | .forConnector("filesystem")
42 | .schema(schema)
43 | .option("format", "csv")
44 | .option("sink.rolling-policy.file-size", "124 kb")
45 | .option("path", s"file://$currentDirectory/sink-table")
46 | .build
47 | )
48 |
49 | table.executeSql("insert into SinkTable select * from SourceTable").print
50 |
--------------------------------------------------------------------------------
/modules/examples/scripts/gen-kafka-data.sc:
--------------------------------------------------------------------------------
1 | //> using dep "org.flinkextended::flink-scala-api:1.18.1_1.2.4"
2 | //> using dep "org.apache.flink:flink-clients:1.18.1"
3 | //> using dep "org.apache.flink:flink-csv:1.18.1"
4 | //> using dep "org.apache.flink:flink-connector-files:1.18.1"
5 | //> using dep "org.apache.flink:flink-connector-kafka:3.0.2-1.18"
6 | //> using dep "org.apache.flink:flink-table-runtime:1.18.1"
7 | //> using dep "org.apache.flink:flink-table-planner-loader:1.18.1"
8 |
9 | import org.apache.flink.table.api._
10 | import org.apache.flink.connector.datagen.table.DataGenConnectorOptions
11 | import org.apache.flinkx.api._
12 | import org.apache.flinkx.api.serializers._
13 |
14 | import java.lang.{Long => JLong}
15 |
16 | val env = StreamExecutionEnvironment.getExecutionEnvironment
17 | val settings = EnvironmentSettings.newInstance.inStreamingMode.build
18 | val table = TableEnvironment.create(settings)
19 | val schema = Schema.newBuilder
20 | .column("id", DataTypes.INT())
21 | .column("bid_price", DataTypes.DOUBLE())
22 | .column("order_time", DataTypes.TIMESTAMP(2))
23 | .build
24 |
25 | table.createTemporaryTable(
26 | "SourceTable",
27 | TableDescriptor
28 | .forConnector("datagen")
29 | .schema(schema)
30 | .option(DataGenConnectorOptions.NUMBER_OF_ROWS, JLong(1000))
31 | .option("fields.id.kind", "sequence")
32 | .option("fields.id.start", "10001")
33 | .option("fields.id.end", "20000")
34 | .build
35 | )
36 |
37 | val brokers = "confluentkafka-cp-kafka:9092"
38 |
39 | table.createTemporaryTable(
40 | "SinkTable",
41 | TableDescriptor
42 | .forConnector("kafka")
43 | .schema(schema)
44 | .option("properties.bootstrap.servers", brokers)
45 | .option("topic", "bids")
46 | .option("format", "csv")
47 | .option("value.format", "csv")
48 | .build
49 | )
50 |
51 | table.executeSql("insert into SinkTable select * from SourceTable").print
52 |
--------------------------------------------------------------------------------
/modules/examples/scripts/hybrid-source.sc:
--------------------------------------------------------------------------------
1 | //> using dep "org.flinkextended::flink-scala-api:1.18.1_1.2.4"
2 | //> using dep "org.apache.flink:flink-clients:1.18.1"
3 | //> using dep "org.apache.flink:flink-csv:1.18.1"
4 | //> using dep "org.apache.flink:flink-connector-files:1.18.1"
5 | //> using dep "org.apache.flink:flink-connector-kafka:3.0.2-1.18"
6 |
7 | import org.apache.flinkx.api.*
8 | import org.apache.flinkx.api.serializers.*
9 | import org.apache.flink.connector.file.src.FileSource
10 | import org.apache.flink.connector.file.src.reader.TextLineInputFormat
11 | import org.apache.flink.connector.file.src.impl.StreamFormatAdapter
12 | import org.apache.flink.connector.kafka.source.KafkaSource
13 | import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer
14 | import org.apache.flink.connector.base.source.hybrid.HybridSource
15 | import org.apache.flink.api.common.serialization.SimpleStringSchema
16 | import org.apache.flink.api.common.eventtime.WatermarkStrategy
17 | import org.apache.flink.core.fs.Path
18 |
19 | val currentDirectory = java.io.File(".").getCanonicalPath
20 |
21 | val fileSource = FileSource
22 | .forBulkFileFormat(
23 | StreamFormatAdapter(TextLineInputFormat()),
24 | Path(s"$currentDirectory/sink-table")
25 | )
26 | .build
27 |
28 | val switchTimestamp = -1L
29 | val brokers = "confluentkafka-cp-kafka:9092"
30 |
31 | val kafkaSource = KafkaSource
32 | .builder[String]()
33 | .setBootstrapServers(brokers)
34 | .setTopics("bids")
35 | .setStartingOffsets(OffsetsInitializer.timestamp(switchTimestamp + 1))
36 | .setValueOnlyDeserializer(SimpleStringSchema())
37 | .build
38 |
39 | val hybridSource = HybridSource
40 | .builder(fileSource)
41 | .addSource(kafkaSource)
42 | .build
43 |
44 | val env = StreamExecutionEnvironment.getExecutionEnvironment
45 | env
46 | .fromSource(hybridSource, WatermarkStrategy.noWatermarks(), "combined")
47 | .print()
48 |
49 | env.execute()
50 |
--------------------------------------------------------------------------------
/modules/examples/scripts/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | true
5 |
6 |
7 |
8 |
9 | %date{yyyy-MM-dd HH:mm:ss.SSSZ, UTC} %-16level %-43thread %-24logger{24} %message%n%xException
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/modules/examples/src/main/protobuf/simple.proto:
--------------------------------------------------------------------------------
1 | syntax = "proto2";
2 | package com.example;
3 | option java_package = "com.example";
4 | option java_multiple_files = true;
5 |
6 | message SimpleTest {
7 | optional int64 uid = 1;
8 | optional string name = 2;
9 | optional int32 category_type = 3;
10 | optional bytes content = 4;
11 | optional double price = 5;
12 | }
--------------------------------------------------------------------------------
/modules/examples/src/main/resources/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | true
5 |
6 |
7 |
8 |
9 | %date{yyyy-MM-dd HH:mm:ss.SSSZ, UTC} %-16level %-43thread %-24logger{24} %message%n%xException
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
19 |
20 |
21 |
22 |
23 |
24 |
25 |
--------------------------------------------------------------------------------
/modules/examples/src/main/scala/org/example/Job.scala:
--------------------------------------------------------------------------------
1 | package org.example
2 |
3 | import org.apache.flinkx.api.*
4 | import org.apache.flinkx.api.serializers.*
5 |
6 | class JobFailed(cause: Exception) extends Exception(cause)
7 |
8 | @main def job =
9 | try {
10 | val env = StreamExecutionEnvironment.getExecutionEnvironment
11 | env
12 | .fromElements(1, 2, 3, 4, 5, 6)
13 | .filter(_ % 2 == 1)
14 | .map(i => i * i)
15 | .print()
16 | throw new RuntimeException("boom")
17 | try env.execute()
18 | catch case e: Exception => throw JobFailed(e)
19 | } catch
20 | case e: JobFailed =>
21 | throw e.getCause
22 | case e: Throwable =>
23 | e.printStackTrace()
24 | // failure in main method, not in the Flink job
25 | val env = StreamExecutionEnvironment.getExecutionEnvironment
26 | env
27 | .fromElements("printing stacktrace")
28 | .print()
29 | env.execute()
30 |
--------------------------------------------------------------------------------
/modules/examples/src/main/scala/org/example/SocketTextStreamWordCount.scala:
--------------------------------------------------------------------------------
1 | package org.example
2 |
3 | /*
4 | * Licensed to the Apache Software Foundation (ASF) under one
5 | * or more contributor license agreements. See the NOTICE file
6 | * distributed with this work for additional information
7 | * regarding copyright ownership. The ASF licenses this file
8 | * to you under the Apache License, Version 2.0 (the
9 | * "License"); you may not use this file except in compliance
10 | * with the License. You may obtain a copy of the License at
11 | *
12 | * http://www.apache.org/licenses/LICENSE-2.0
13 | *
14 | * Unless required by applicable law or agreed to in writing, software
15 | * distributed under the License is distributed on an "AS IS" BASIS,
16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17 | * See the License for the specific language governing permissions and
18 | * limitations under the License.
19 | */
20 |
21 | import org.apache.flinkx.api.*
22 | import org.apache.flinkx.api.serializers.*
23 | import org.apache.flink.configuration.Configuration
24 | import org.apache.flink.configuration.ConfigConstants
25 | import org.apache.flink.configuration.RestOptions.BIND_PORT
26 | import scala.jdk.CollectionConverters.*
27 |
28 | /** This example shows an implementation of WordCount with data from a text socket. To run the example make sure that
29 | * the service providing the text data is already up and running.
30 | *
31 | * To start an example socket text stream on your local machine run netcat from a command line, where the parameter
32 | * specifies the port number:
33 | *
34 | * {{{
35 | * nc -lk 9999
36 | * }}}
37 | *
38 | * Usage:
39 | * {{{
40 | * SocketTextStreamWordCount