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

[Refactor][Flink] Refactor flink sql parse #2509

Merged
merged 7 commits into from
Nov 8, 2023
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 @@ -70,13 +70,16 @@
import org.apache.flink.table.typeutils.FieldInfoUtils;
import org.apache.flink.types.Row;

import java.io.File;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
Expand Down Expand Up @@ -124,13 +127,7 @@ public static CustomTableEnvironmentImpl createBatch(StreamExecutionEnvironment
configuration.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH);
TableConfig tableConfig = new TableConfig();
tableConfig.addConfiguration(configuration);
return create(
executionEnvironment,
EnvironmentSettings.newInstance()
.useBlinkPlanner()
.inBatchMode()
.build(),
tableConfig);
return create(executionEnvironment, EnvironmentSettings.inBatchMode(), tableConfig);
}

public static CustomTableEnvironmentImpl create(
Expand Down Expand Up @@ -295,15 +292,14 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr
return record;
}

public boolean parseAndLoadConfiguration(
String statement, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
public boolean parseAndLoadConfiguration(String statement, Map<String, Object> setMap) {
List<Operation> operations = getParser().parse(statement);
for (Operation operation : operations) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, environment, setMap);
callSet((SetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, environment, setMap);
callReset((ResetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
}
}
Expand Down Expand Up @@ -378,6 +374,19 @@ public void executeCTAS(Operation operation) {
}
}

@Override
public void addJar(File... jarPath) {
Configuration configuration = this.getRootConfiguration();
List<String> jars = configuration.get(PipelineOptions.JARS);
if (jars == null) {
configuration.set(
PipelineOptions.JARS,
Arrays.stream(jarPath).map(File::getAbsolutePath).collect(Collectors.toList()));
} else {
CollUtil.addAll(jars, jarPath);
}
}

@Override
public <T> void createTemporaryView(String path, DataStream<T> dataStream, Expression... fields) {
createTemporaryView(path, fromDataStream(dataStream, fields));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,10 +65,13 @@
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.types.Row;

import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
Expand Down Expand Up @@ -192,6 +195,19 @@ public ObjectNode getStreamGraph(String statement) {
}
}

@Override
public void addJar(File... jarPath) {
Configuration configuration = this.getRootConfiguration();
List<String> jars = configuration.get(PipelineOptions.JARS);
if (jars == null) {
configuration.set(
PipelineOptions.JARS,
Arrays.stream(jarPath).map(File::getAbsolutePath).collect(Collectors.toList()));
} else {
CollUtil.addAll(jars, jarPath);
}
}

@Override
public JobPlanInfo getJobPlanInfo(List<String> statements) {
return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements)));
Expand Down Expand Up @@ -254,15 +270,14 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr
return record;
}

public boolean parseAndLoadConfiguration(
String statement, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
public boolean parseAndLoadConfiguration(String statement, Map<String, Object> setMap) {
List<Operation> operations = getParser().parse(statement);
for (Operation operation : operations) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, environment, setMap);
callSet((SetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, environment, setMap);
callReset((ResetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,9 @@
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.types.Row;

import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -102,6 +104,33 @@ public static CustomTableEnvironmentImpl create(
return new CustomTableEnvironmentImpl(streamTableEnvironment);
}

public boolean parseAndLoadConfiguration(String statement, Map<String, Object> setMap) {
List<Operation> operations = getParser().parse(statement);
for (Operation operation : operations) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
}
}
return false;
}

@Override
public void addJar(File... jarPath) {
Configuration configuration = this.getRootConfiguration();
List<String> jars = configuration.get(PipelineOptions.JARS);
if (jars == null) {
configuration.set(
PipelineOptions.JARS,
Arrays.stream(jarPath).map(File::getAbsolutePath).collect(Collectors.toList()));
} else {
CollUtil.addAll(jars, jarPath);
}
}

public ObjectNode getStreamGraph(String statement) {
List<Operation> operations = super.getParser().parse(statement);
if (operations.size() != 1) {
Expand Down Expand Up @@ -198,20 +227,6 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr
return record;
}

public boolean parseAndLoadConfiguration(
String statement, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
for (Operation operation : getParser().parse(statement)) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, environment, setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, environment, setMap);
return true;
}
}
return false;
}

private void callSet(
SetOperation setOperation, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
if (!setOperation.getKey().isPresent() || !setOperation.getValue().isPresent()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,9 @@
import org.apache.flink.table.operations.ddl.CreateTableOperation;
import org.apache.flink.types.Row;

import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -103,6 +105,35 @@ public static CustomTableEnvironmentImpl create(
return new CustomTableEnvironmentImpl(streamTableEnvironment);
}

@Override
public void addJar(File... jarPath) {
Configuration configuration = this.getRootConfiguration();
List<String> jars = configuration.get(PipelineOptions.JARS);
if (jars == null) {
configuration.set(
PipelineOptions.JARS,
Arrays.stream(jarPath).map(File::getAbsolutePath).collect(Collectors.toList()));
} else {
CollUtil.addAll(jars, jarPath);
}
}

@Override
public boolean parseAndLoadConfiguration(String statement, Map<String, Object> setMap) {
List<Operation> operations = getParser().parse(statement);
for (Operation operation : operations) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, getStreamExecutionEnvironment(), setMap);
return true;
}
}
return false;
}

@Override
public ObjectNode getStreamGraph(String statement) {
List<Operation> operations = super.getParser().parse(statement);
if (operations.size() != 1) {
Expand Down Expand Up @@ -142,6 +173,7 @@ public JobPlanInfo getJobPlanInfo(List<String> statements) {
return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements)));
}

@Override
public StreamGraph getStreamGraphFromInserts(List<String> statements) {
List<ModifyOperation> modifyOperations = new ArrayList<>();
statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> {
Expand All @@ -161,10 +193,12 @@ public StreamGraph getStreamGraphFromInserts(List<String> statements) {
return transOperatoinsToStreamGraph(modifyOperations);
}

@Override
public JobGraph getJobGraphFromInserts(List<String> statements) {
return getStreamGraphFromInserts(statements).getJobGraph();
}

@Override
public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) {
List<Operation> operations = getParser().parse(statement);
if (operations.size() != 1) {
Expand Down Expand Up @@ -194,20 +228,6 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr
return data;
}

public boolean parseAndLoadConfiguration(
String statement, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
for (Operation operation : getParser().parse(statement)) {
if (operation instanceof SetOperation) {
callSet((SetOperation) operation, environment, setMap);
return true;
} else if (operation instanceof ResetOperation) {
callReset((ResetOperation) operation, environment, setMap);
return true;
}
}
return false;
}

private void callSet(
SetOperation setOperation, StreamExecutionEnvironment environment, Map<String, Object> setMap) {
if (!setOperation.getKey().isPresent() || !setOperation.getValue().isPresent()) {
Expand Down
Loading
Loading