Skip to content

Commit

Permalink
[Optimization-3909][core] Optimize FlinkDDL execution sequence (#3910)
Browse files Browse the repository at this point in the history
  • Loading branch information
aiwenmo authored Nov 13, 2024
1 parent 237b45a commit a67ac9a
Show file tree
Hide file tree
Showing 16 changed files with 606 additions and 212 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@

package org.dinky.trans;

import org.dinky.executor.CustomTableEnvironment;

import org.apache.flink.table.operations.Operation;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -27,7 +31,7 @@
*
* @since 2021/6/14 18:18
*/
public class AbstractOperation {
public class AbstractOperation implements Operation {

protected static final Logger logger = LoggerFactory.getLogger(AbstractOperation.class);

Expand All @@ -50,4 +54,13 @@ public void setStatement(String statement) {
public boolean noExecute() {
return true;
}

public String explain(CustomTableEnvironment tEnv) {
return asSummaryString();
}

@Override
public String asSummaryString() {
return "";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,11 @@

/** */
public interface ExtendOperation extends Operation {

Optional<? extends TableResult> execute(CustomTableEnvironment tEnv);

String explain(CustomTableEnvironment tEnv);

TableResult TABLE_RESULT_OK = TableResultImpl.builder()
.resultKind(ResultKind.SUCCESS)
.schema(ResolvedSchema.of(Column.physical("result", DataTypes.STRING())))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,11 @@ public Optional<? extends TableResult> execute(CustomTableEnvironment tEnv) {
return Optional.of(TABLE_RESULT_OK);
}

@Override
public String explain(CustomTableEnvironment tEnv) {
return asSummaryString();
}

private void callSet(SetOperation setOperation, CustomTableEnvironment environment) {
if (!setOperation.getKey().isPresent() || !setOperation.getValue().isPresent()) {
return;
Expand All @@ -121,6 +126,11 @@ private void setConfiguration(CustomTableEnvironment environment, Map<String, St

@Override
public String asSummaryString() {
return null;
return toString();
}

@Override
public String toString() {
return "CustomSetOperation{" + "key='" + key + '\'' + ", value='" + value + '\'' + '}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ public String asSummaryString() {
return statement;
}

public Pipeline explain(CustomTableEnvironment tEnv) {
public Pipeline explainJar(CustomTableEnvironment tEnv) {
return getStreamGraph(tEnv);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,10 @@ public class SqlExplainResult {
private boolean parseTrue;
private boolean explainTrue;
private LocalDateTime explainTime;
private boolean isInvalid;

public static final SqlExplainResult INVALID_EXPLAIN_RESULT =
new Builder().invalid().build();

public SqlExplainResult() {}

Expand Down Expand Up @@ -71,6 +75,7 @@ private SqlExplainResult(Builder builder) {
setParseTrue(builder.parseTrue);
setExplainTrue(builder.explainTrue);
setExplainTime(builder.explainTime);
setInvalid(builder.isInvalid);
}

public static SqlExplainResult success(String type, String sql, String explain) {
Expand Down Expand Up @@ -167,6 +172,14 @@ public void setExplainTime(LocalDateTime explainTime) {
this.explainTime = explainTime;
}

public boolean isInvalid() {
return isInvalid;
}

public void setInvalid(boolean invalid) {
isInvalid = invalid;
}

@Override
public String toString() {
return String.format(
Expand All @@ -185,6 +198,7 @@ public static final class Builder {
private boolean parseTrue;
private boolean explainTrue;
private LocalDateTime explainTime;
private boolean isInvalid = false;

private Builder() {}

Expand Down Expand Up @@ -237,6 +251,11 @@ public Builder explainTime(LocalDateTime val) {
return this;
}

public Builder invalid() {
isInvalid = true;
return this;
}

public SqlExplainResult build() {
return new SqlExplainResult(this);
}
Expand Down
14 changes: 11 additions & 3 deletions dinky-core/src/main/java/org/dinky/executor/Executor.java
Original file line number Diff line number Diff line change
Expand Up @@ -243,15 +243,19 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr
if (Asserts.isNotNullString(statement) && !pretreatExecute(statement).isNoExecute()) {
return tableEnvironment.explainSqlRecord(statement, extraDetails);
}
return null;
return SqlExplainResult.INVALID_EXPLAIN_RESULT;
}

public StreamGraph getStreamGraphFromStatement(List<String> statements) {
return tableEnvironment.getStreamGraphFromInserts(statements);
}

public ObjectNode getStreamGraph(List<String> statements) {
StreamGraph streamGraph = tableEnvironment.getStreamGraphFromInserts(statements);
return getStreamGraphJsonNode(streamGraph);
}

private ObjectNode getStreamGraphJsonNode(StreamGraph streamGraph) {
public ObjectNode getStreamGraphJsonNode(StreamGraph streamGraph) {
JSONGenerator jsonGenerator = new JSONGenerator(streamGraph);
String json = jsonGenerator.getJSON();
ObjectMapper mapper = new ObjectMapper();
Expand All @@ -261,14 +265,18 @@ private ObjectNode getStreamGraphJsonNode(StreamGraph streamGraph) {
} catch (JsonProcessingException e) {
logger.error("Get stream graph json node error.", e);
}

return objectNode;
}

public StreamGraph getStreamGraph() {
return environment.getStreamGraph();
}

public StreamGraph getStreamGraphFromCustomStatements(List<String> statements) {
statements.forEach(this::executeSql);
return getStreamGraph();
}

public ObjectNode getStreamGraphFromDataStream(List<String> statements) {
statements.forEach(this::executeSql);
return getStreamGraphJsonNode(getStreamGraph());
Expand Down
Loading

0 comments on commit a67ac9a

Please sign in to comment.