1
0

[HUDI-3389] Bump flink version to 1.14.3 (#4776)

This commit is contained in:
Danny Chan
2022-02-10 11:32:01 +08:00
committed by GitHub
parent 464027ec37
commit b3b44236fe
13 changed files with 72 additions and 30 deletions

View File

@@ -60,7 +60,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
<artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
@@ -159,7 +159,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<artifactId>flink-runtime</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<classifier>tests</classifier>

View File

@@ -164,13 +164,13 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
<artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
@@ -307,7 +307,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-runtime_${scala.binary.version}</artifactId>
<artifactId>flink-runtime</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>
@@ -321,7 +321,7 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime-blink_${scala.binary.version}</artifactId>
<artifactId>flink-table-runtime_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>

View File

@@ -415,13 +415,11 @@ public class StreamWriteOperatorCoordinator
CompletableFuture<?>[] futures = Arrays.stream(this.gateways).filter(Objects::nonNull)
.map(gw -> gw.sendEvent(CommitAckEvent.getInstance(checkpointId)))
.toArray(CompletableFuture<?>[]::new);
try {
CompletableFuture.allOf(futures).get();
} catch (Throwable throwable) {
if (!sendToFinishedTasks(throwable)) {
throw new HoodieException("Error while waiting for the commit ack events to finish sending", throwable);
CompletableFuture.allOf(futures).whenComplete((resp, error) -> {
if (!sendToFinishedTasks(error)) {
throw new HoodieException("Error while waiting for the commit ack events to finish sending", error);
}
}
});
}
/**

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.RateLimiter;
import org.apache.flink.shaded.guava30.com.google.common.util.concurrent.RateLimiter;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;

View File

@@ -39,11 +39,14 @@ import org.apache.hudi.sink.partitioner.BucketAssignFunction;
import org.apache.hudi.sink.transform.RowDataToHoodieFunctions;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.api.operators.ProcessOperator;
import org.apache.flink.table.data.RowData;
@@ -64,7 +67,11 @@ public class Pipelines {
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION)) {
// shuffle by partition keys
dataStream = dataStream.keyBy(rowDataKeyGen::getPartitionPath);
// use #partitionCustom instead of #keyBy to avoid duplicate sort operations,
// see BatchExecutionUtils#applyBatchExecutionSettings for details.
Partitioner<String> partitioner = (key, channels) ->
KeyGroupRangeAssignment.assignKeyToParallelOperator(key, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM, channels);
dataStream = dataStream.partitionCustom(partitioner, rowDataKeyGen::getPartitionPath);
}
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_BY_PARTITION)) {
SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields);

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.source;
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.flink.api.common.operators.MailboxExecutor;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.runtime.state.JavaSerializer;
@@ -29,7 +30,6 @@ import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.MailboxExecutor;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
import org.apache.flink.streaming.api.operators.StreamOperator;
@@ -54,7 +54,7 @@ import java.util.concurrent.LinkedBlockingDeque;
* StreamReadMonitoringFunction}. Contrary to the {@link StreamReadMonitoringFunction} which has a parallelism of 1,
* this operator can have multiple parallelism.
*
* <p>As soon as an input split {@link MergeOnReadInputSplit} is received, it is put in a queue,
* <p>As soon as an input split {@link MergeOnReadInputSplit} is received, it is put into a queue,
* the {@link MailboxExecutor} read the actual data of the split.
* This architecture allows the separation of split reading from processing the checkpoint barriers,
* thus removing any potential back-pressure.
@@ -118,10 +118,10 @@ public class StreamReadOperator extends AbstractStreamOperator<RowData>
getOperatorConfig().getTimeCharacteristic(),
getProcessingTimeService(),
new Object(), // no actual locking needed
getContainingTask().getStreamStatusMaintainer(),
output,
getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(),
-1);
-1,
true);
// Enqueue to process the recovered input splits.
enqueueProcessSplits();
@@ -205,8 +205,8 @@ public class StreamReadOperator extends AbstractStreamOperator<RowData>
}
@Override
public void dispose() throws Exception {
super.dispose();
public void close() throws Exception {
super.close();
if (format != null) {
format.close();
@@ -218,8 +218,8 @@ public class StreamReadOperator extends AbstractStreamOperator<RowData>
}
@Override
public void close() throws Exception {
super.close();
public void finish() throws Exception {
super.finish();
output.close();
if (sourceContext != null) {
sourceContext.emitWatermark(Watermark.MAX_WATERMARK);

View File

@@ -172,7 +172,7 @@ public class StreamWriteITCase extends TestLogger {
DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
Pipelines.clean(conf, pipeline);
Pipelines.compact(conf, pipeline);
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph());
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish
@@ -229,7 +229,7 @@ public class StreamWriteITCase extends TestLogger {
DataStream<Object> pipeline = Pipelines.hoodieStreamWrite(conf, parallelism, hoodieRecordDataStream);
execEnv.addOperator(pipeline.getTransformation());
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
JobClient client = execEnv.executeAsync(conf.getString(FlinkOptions.TABLE_NAME));
// wait for the streaming job to finish
client.getJobExecutionResult().get();

View File

@@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.OutputTag;
@@ -49,6 +50,11 @@ public class CollectorOutput<T> implements Output<StreamRecord<T>> {
list.add(mark);
}
@Override
public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {
}
@Override
public void emitLatencyMarker(LatencyMarker latencyMarker) {
list.add(latencyMarker);

View File

@@ -38,6 +38,7 @@ import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus;
import org.apache.flink.util.Collector;
import org.apache.flink.util.OutputTag;
@@ -102,6 +103,11 @@ public class CompactFunctionWrapper {
}
@Override
public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {
}
@Override
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> streamRecord) {

View File

@@ -23,6 +23,8 @@ import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StatePartitionStreamProvider;
import java.util.OptionalLong;
/**
* A {@link FunctionInitializationContext} for testing purpose.
*/
@@ -39,6 +41,11 @@ public class MockStateInitializationContext implements StateInitializationContex
return false;
}
@Override
public OptionalLong getRestoredCheckpointId() {
return OptionalLong.empty();
}
@Override
public MockOperatorStateStore getOperatorStateStore() {
return operatorStateStore;

View File

@@ -19,7 +19,7 @@ package org.apache.hudi.sink.utils;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.OperatorMetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.memory.MemoryManager;
@@ -69,8 +69,8 @@ public class MockStreamingRuntimeContext extends StreamingRuntimeContext {
}
@Override
public MetricGroup getMetricGroup() {
return new UnregisteredMetricsGroup();
public OperatorMetricGroup getMetricGroup() {
return UnregisteredMetricsGroup.createOperatorMetricGroup();
}
@Override

View File

@@ -31,10 +31,12 @@ import org.apache.hudi.utils.factory.CollectSinkTableFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
import org.apache.flink.table.catalog.ObjectPath;
@@ -86,8 +88,24 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
execConf.setString("restart-strategy", "fixed-delay");
execConf.setString("restart-strategy.fixed-delay.attempts", "0");
Configuration conf = new Configuration();
// for batch upsert use cases: current suggestion is to disable these 2 options,
// from 1.14, flink runtime execution mode has switched from streaming
// to batch for batch execution mode(before that, both streaming and batch use streaming execution mode),
// current batch execution mode has these limitations:
//
// 1. the keyed stream default to always sort the inputs by key;
// 2. the batch state-backend requires the inputs sort by state key
//
// For our hudi batch pipeline upsert case, we rely on the consuming sequence for index records and data records,
// the index records must be loaded first before data records for BucketAssignFunction to keep upsert semantics correct,
// so we suggest disabling these 2 options to use streaming state-backend for batch execution mode
// to keep the strategy before 1.14.
conf.setBoolean("execution.sorted-inputs.enabled", false);
conf.setBoolean("execution.batch-state-backend.enabled", false);
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(conf);
settings = EnvironmentSettings.newInstance().inBatchMode().build();
batchTableEnv = TableEnvironmentImpl.create(settings);
batchTableEnv = StreamTableEnvironment.create(execEnv, settings);
batchTableEnv.getConfig().getConfiguration()
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
}
@@ -861,7 +879,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
.getContextClassLoader().getResource("debezium_json.data")).toString();
String sourceDDL = ""
+ "CREATE TABLE debezium_source(\n"
+ " id INT NOT NULL,\n"
+ " id INT NOT NULL PRIMARY KEY NOT ENFORCED,\n"
+ " ts BIGINT,\n"
+ " name STRING,\n"
+ " description STRING,\n"

View File

@@ -117,7 +117,7 @@
<http.version>4.4.1</http.version>
<spark.version>${spark2.version}</spark.version>
<sparkbundle.version></sparkbundle.version>
<flink.version>1.13.1</flink.version>
<flink.version>1.14.3</flink.version>
<spark2.version>2.4.4</spark2.version>
<spark3.version>3.2.0</spark3.version>
<hudi.spark.module>hudi-spark2</hudi.spark.module>