Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[flink] Remove compatibility utils that are used for flink-1.14 #3054

Merged
merged 3 commits into from
Mar 20, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
import org.apache.paimon.flink.sink.StoreSinkWrite;
import org.apache.paimon.flink.sink.StoreSinkWriteImpl;
import org.apache.paimon.flink.sink.WrappedManifestCommittableSerializer;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.manifest.WrappedManifestCommittable;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
Expand Down Expand Up @@ -133,10 +132,7 @@ public DataStreamSink<?> sinkFrom(
createCommittableStateManager()))
.setParallelism(input.getParallelism());
configureGlobalCommitter(
committed,
commitCpuCores,
commitHeapMemory,
StreamExecutionEnvironmentUtils.getConfiguration(env));
committed, commitCpuCores, commitHeapMemory, env.getConfiguration());
return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1);
}

Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.paimon.flink.FlinkCatalog;
import org.apache.paimon.flink.FlinkCatalogFactory;
import org.apache.paimon.flink.LogicalTypeConversion;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.options.CatalogOptions;
import org.apache.paimon.options.Options;
import org.apache.paimon.types.DataType;
Expand Down Expand Up @@ -80,7 +79,7 @@ private void initFlinkEnv(StreamExecutionEnvironment env) {
}

protected void execute(String defaultName) throws Exception {
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
String name = conf.getOptional(PipelineOptions.NAME).orElse(defaultName);
env.execute(name);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.paimon.flink.compact.UnawareBucketCompactionTopoBuilder;
import org.apache.paimon.flink.sink.CompactorSinkBuilder;
import org.apache.paimon.flink.source.CompactorSourceBuilder;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.table.FileStoreTable;

import org.apache.flink.api.common.RuntimeExecutionMode;
Expand Down Expand Up @@ -75,7 +74,7 @@ public CompactAction withPartitions(List<Map<String, String>> partitions) {

@Override
public void build() {
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
boolean isStreaming =
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING;
FileStoreTable fileStoreTable = (FileStoreTable) table;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import org.apache.paimon.flink.sink.MultiTablesCompactorSink;
import org.apache.paimon.flink.source.CompactorSourceBuilder;
import org.apache.paimon.flink.source.MultiTablesCompactorSourceBuilder;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.options.Options;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
Expand Down Expand Up @@ -161,7 +160,7 @@ private void buildForDividedMode() {
!tableMap.isEmpty(),
"no tables to be compacted. possible cause is that there are no tables detected after pattern matching");

ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
boolean isStreaming =
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING;
for (Map.Entry<String, FileStoreTable> entry : tableMap.entrySet()) {
Expand All @@ -186,7 +185,7 @@ private void buildForDividedMode() {

private void buildForCombinedMode() {

ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
boolean isStreaming =
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING;
// TODO: Currently, multi-tables compaction don't support tables which bucketmode is UNWARE.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,18 @@
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.sink.FlinkSinkBuilder;
import org.apache.paimon.flink.utils.TableEnvironmentUtils;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
import org.apache.paimon.utils.Preconditions;

import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.data.RowData;

import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand Down Expand Up @@ -68,8 +70,34 @@ public TableResult batchSink(DataStream<RowData> dataStream) {

List<String> sinkIdentifierNames = Collections.singletonList(identifier.getFullName());

return TableEnvironmentUtils.executeInternal(
batchTEnv, transformations, sinkIdentifierNames);
return executeInternal(transformations, sinkIdentifierNames);
}

/**
* Invoke {@code TableEnvironmentImpl#executeInternal(List<Transformation<?>>, List<String>)}
* from a {@link StreamTableEnvironment} instance through reflecting.
*/
private TableResult executeInternal(
List<Transformation<?>> transformations, List<String> sinkIdentifierNames) {
Class<?> clazz = batchTEnv.getClass().getSuperclass().getSuperclass();
try {
Method executeInternal =
clazz.getDeclaredMethod("executeInternal", List.class, List.class);
executeInternal.setAccessible(true);

return (TableResult)
executeInternal.invoke(batchTEnv, transformations, sinkIdentifierNames);
} catch (NoSuchMethodException e) {
throw new RuntimeException(
"Failed to get 'TableEnvironmentImpl#executeInternal(List, List)' method "
+ "from given StreamTableEnvironment instance by Java reflection. This is unexpected.",
e);
} catch (IllegalAccessException | InvocationTargetException e) {
throw new RuntimeException(
"Failed to invoke 'TableEnvironmentImpl#executeInternal(List, List)' method "
+ "from given StreamTableEnvironment instance by Java reflection. This is unexpected.",
e);
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.factories.Factory;
import org.apache.paimon.flink.action.ActionBase;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.table.Table;
import org.apache.paimon.utils.StringUtils;

Expand Down Expand Up @@ -69,13 +68,13 @@ protected String[] execute(

protected String[] execute(ProcedureContext procedureContext, JobClient jobClient) {
StreamExecutionEnvironment env = procedureContext.getExecutionEnvironment();
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
return execute(jobClient, conf.get(TABLE_DML_SYNC));
}

protected String[] execute(StreamExecutionEnvironment env, String defaultJobName)
throws Exception {
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
String name = conf.getOptional(PipelineOptions.NAME).orElse(defaultJobName);
return execute(env.executeAsync(name), conf.get(TABLE_DML_SYNC));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.paimon.data.InternalRow;
import org.apache.paimon.flink.utils.InternalTypeInfo;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.table.BucketMode;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
Expand All @@ -38,7 +37,7 @@
public class QueryService {

public static void build(StreamExecutionEnvironment env, Table table, int parallelism) {
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
Preconditions.checkArgument(
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING,
"Query Service only supports streaming mode.");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.paimon.CoreOptions.ChangelogProducer;
import org.apache.paimon.CoreOptions.TagCreationMode;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
Expand Down Expand Up @@ -190,8 +189,7 @@ public DataStream<Committable> doWrite(
DataStream<T> input, String commitUser, @Nullable Integer parallelism) {
StreamExecutionEnvironment env = input.getExecutionEnvironment();
boolean isStreaming =
StreamExecutionEnvironmentUtils.getConfiguration(env)
.get(ExecutionOptions.RUNTIME_MODE)
env.getConfiguration().get(ExecutionOptions.RUNTIME_MODE)
== RuntimeExecutionMode.STREAMING;

boolean writeOnly = table.coreOptions().writeOnly();
Expand Down Expand Up @@ -222,7 +220,7 @@ public DataStream<Committable> doWrite(

protected DataStreamSink<?> doCommit(DataStream<Committable> written, String commitUser) {
StreamExecutionEnvironment env = written.getExecutionEnvironment();
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
CheckpointConfig checkpointConfig = env.getCheckpointConfig();
boolean isStreaming =
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.flink.VersionedSerializerWrapper;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
import org.apache.paimon.manifest.WrappedManifestCommittable;
import org.apache.paimon.options.Options;

Expand Down Expand Up @@ -87,8 +86,7 @@ public SingleOutputStreamOperator<MultiTableCommittable> doWrite(
DataStream<RowData> input, String commitUser, Integer parallelism) {
StreamExecutionEnvironment env = input.getExecutionEnvironment();
boolean isStreaming =
StreamExecutionEnvironmentUtils.getConfiguration(env)
.get(ExecutionOptions.RUNTIME_MODE)
env.getConfiguration().get(ExecutionOptions.RUNTIME_MODE)
== RuntimeExecutionMode.STREAMING;

SingleOutputStreamOperator<MultiTableCommittable> written =
Expand All @@ -112,7 +110,7 @@ public SingleOutputStreamOperator<MultiTableCommittable> doWrite(
protected DataStreamSink<?> doCommit(
DataStream<MultiTableCommittable> written, String commitUser) {
StreamExecutionEnvironment env = written.getExecutionEnvironment();
ReadableConfig conf = StreamExecutionEnvironmentUtils.getConfiguration(env);
ReadableConfig conf = env.getConfiguration();
CheckpointConfig checkpointConfig = env.getCheckpointConfig();
boolean isStreaming =
conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING;
Expand Down

This file was deleted.

This file was deleted.

Loading