1
0

[HUDI-4357] Support flink 1.15.x (#6050)

This commit is contained in:
Danny Chan
2022-07-06 13:42:58 +08:00
committed by GitHub
parent b18c32379f
commit 7eeaff9ee0
91 changed files with 7799 additions and 82 deletions

View File

@@ -118,12 +118,12 @@
<!-- Flink -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<artifactId>${flink.streaming.java.artifactId}</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients_${scala.binary.version}</artifactId>
<artifactId>${flink.clients.artifactId}</artifactId>
<scope>compile</scope>
<exclusions>
<exclusion>
@@ -138,7 +138,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka_${scala.binary.version}</artifactId>
<artifactId>${flink.connector.kafka.artifactId}</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
@@ -148,12 +148,12 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
<artifactId>${flink.hadoop.compatibility.artifactId}</artifactId>
<version>${flink.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-parquet_${scala.binary.version}</artifactId>
<artifactId>${flink.parquet.artifactId}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
@@ -183,7 +183,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-statebackend-rocksdb_${scala.binary.version}</artifactId>
<artifactId>${flink.statebackend.rocksdb.artifactId}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
@@ -304,17 +304,11 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<!-- <dependency>-->
<!-- <groupId>org.apache.hudi</groupId>-->
<!-- <artifactId>hudi-flink_${scala.binary.version}</artifactId>-->
<!-- <version>${project.version}</version>-->
<!-- <scope>test</scope>-->
<!-- </dependency>-->
<!-- Flink dependencies -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
<artifactId>${flink.test.utils.artifactId}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
@@ -327,7 +321,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<artifactId>${flink.streaming.java.artifactId}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.examples.quickstart.source;
import org.apache.hudi.adapter.DataStreamScanProviderAdapter;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
@@ -28,7 +30,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
@@ -74,7 +75,7 @@ public class ContinuousFileSource implements ScanTableSource {
@Override
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
return new DataStreamScanProvider() {
return new DataStreamScanProviderAdapter() {
@Override
public boolean isBounded() {