Skip to content

Commit f19cee8

Browse files
author
zhuanglw
committed
[Feature] Add lineage for flink 1.20
1 parent 9ea886b commit f19cee8

File tree

8 files changed

+1186
-4
lines changed

8 files changed

+1186
-4
lines changed

dinky-admin/pom.xml

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -379,6 +379,31 @@
379379
<groupId>org.dinky</groupId>
380380
<artifactId>dinky-alert-http</artifactId>
381381
</dependency>
382+
383+
<dependency>
384+
<groupId>io.openlineage</groupId>
385+
<artifactId>openlineage-java</artifactId>
386+
<version>1.21.1</version>
387+
<exclusions>
388+
<exclusion>
389+
<groupId>com.fasterxml.jackson.core</groupId>
390+
<artifactId>jackson-core</artifactId>
391+
</exclusion>
392+
</exclusions>
393+
</dependency>
394+
395+
<dependency>
396+
<groupId>org.dinky</groupId>
397+
<artifactId>dinky-client-${dinky.flink.version}</artifactId>
398+
<version>${revision}</version>
399+
<scope>provided</scope>
400+
</dependency>
401+
<dependency>
402+
<groupId>org.dinky</groupId>
403+
<artifactId>dinky-flink-${dinky.flink.version}</artifactId>
404+
<version>${revision}</version>
405+
<scope>provided</scope>
406+
</dependency>
382407
</dependencies>
383408

384409
<build>

dinky-admin/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java

Lines changed: 537 additions & 0 deletions
Large diffs are not rendered by default.
Lines changed: 167 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,167 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.table.planner.delegation;
20+
21+
import org.apache.flink.table.api.TableException;
22+
import org.apache.flink.table.api.ValidationException;
23+
import org.apache.flink.table.catalog.CatalogManager;
24+
import org.apache.flink.table.catalog.UnresolvedIdentifier;
25+
import org.apache.flink.table.delegation.Parser;
26+
import org.apache.flink.table.expressions.ResolvedExpression;
27+
import org.apache.flink.table.operations.ModifyOperation;
28+
import org.apache.flink.table.operations.Operation;
29+
import org.apache.flink.table.planner.calcite.FlinkPlannerImpl;
30+
import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
31+
import org.apache.flink.table.planner.calcite.RexFactory;
32+
import org.apache.flink.table.planner.calcite.SqlToRexConverter;
33+
import org.apache.flink.table.planner.expressions.RexNodeExpression;
34+
import org.apache.flink.table.planner.operations.SqlNodeToOperationConversion;
35+
import org.apache.flink.table.planner.parse.CalciteParser;
36+
import org.apache.flink.table.planner.parse.ExtendedParser;
37+
import org.apache.flink.table.types.logical.LogicalType;
38+
import org.apache.flink.table.types.logical.RowType;
39+
import org.apache.flink.table.types.utils.TypeConversions;
40+
import org.apache.flink.util.Preconditions;
41+
42+
import org.apache.calcite.rex.RexNode;
43+
import org.apache.calcite.sql.SqlIdentifier;
44+
import org.apache.calcite.sql.SqlNode;
45+
import org.apache.calcite.sql.SqlNodeList;
46+
import org.apache.calcite.sql.advise.SqlAdvisor;
47+
import org.apache.calcite.sql.advise.SqlAdvisorValidator;
48+
import org.apache.lineage.flink.sql.metadata.LineageHandler;
49+
50+
import javax.annotation.Nullable;
51+
52+
import java.util.ArrayList;
53+
import java.util.Arrays;
54+
import java.util.Collections;
55+
import java.util.List;
56+
import java.util.Optional;
57+
import java.util.function.Supplier;
58+
import java.util.stream.Collectors;
59+
60+
/** Implementation of {@link Parser} that uses Calcite. */
61+
public class ParserImpl implements Parser {
62+
63+
private final CatalogManager catalogManager;
64+
65+
// we use supplier pattern here in order to use the most up to
66+
// date configuration. Users might change the parser configuration in a TableConfig in between
67+
// multiple statements parsing
68+
private final Supplier<FlinkPlannerImpl> validatorSupplier;
69+
private final Supplier<CalciteParser> calciteParserSupplier;
70+
private final RexFactory rexFactory;
71+
private static final ExtendedParser EXTENDED_PARSER = ExtendedParser.INSTANCE;
72+
73+
public ParserImpl(
74+
CatalogManager catalogManager,
75+
Supplier<FlinkPlannerImpl> validatorSupplier,
76+
Supplier<CalciteParser> calciteParserSupplier,
77+
RexFactory rexFactory) {
78+
this.catalogManager = catalogManager;
79+
this.validatorSupplier = validatorSupplier;
80+
this.calciteParserSupplier = calciteParserSupplier;
81+
this.rexFactory = rexFactory;
82+
}
83+
84+
/**
85+
* When parsing statement, it first uses {@link ExtendedParser} to parse statements. If {@link
86+
* ExtendedParser} fails to parse statement, it uses the {@link CalciteParser} to parse
87+
* statements.
88+
*
89+
* @param statement input statement.
90+
* @return parsed operations.
91+
*/
92+
@Override
93+
public List<Operation> parse(String statement) {
94+
CalciteParser parser = calciteParserSupplier.get();
95+
FlinkPlannerImpl planner = validatorSupplier.get();
96+
97+
Optional<Operation> command = EXTENDED_PARSER.parse(statement);
98+
if (command.isPresent()) {
99+
return Collections.singletonList(command.get());
100+
}
101+
102+
// parse the sql query
103+
// use parseSqlList here because we need to support statement end with ';' in sql client.
104+
SqlNodeList sqlNodeList = parser.parseSqlList(statement);
105+
List<SqlNode> parsed = sqlNodeList.getList();
106+
Preconditions.checkArgument(parsed.size() == 1, "only single statement supported");
107+
Operation operation = SqlNodeToOperationConversion.convert(planner, catalogManager, parsed.get(0))
108+
.orElseThrow(() -> new TableException("Unsupported query: " + statement));
109+
if(operation instanceof ModifyOperation){
110+
LineageHandler.analyze(this.catalogManager, operation, statement);
111+
}
112+
return Collections.singletonList(operation);
113+
}
114+
115+
@Override
116+
public UnresolvedIdentifier parseIdentifier(String identifier) {
117+
CalciteParser parser = calciteParserSupplier.get();
118+
SqlIdentifier sqlIdentifier = parser.parseIdentifier(identifier);
119+
return UnresolvedIdentifier.of(sqlIdentifier.names);
120+
}
121+
122+
@Override
123+
public ResolvedExpression parseSqlExpression(
124+
String sqlExpression, RowType inputRowType, @Nullable LogicalType outputType) {
125+
try {
126+
final SqlToRexConverter sqlToRexConverter =
127+
rexFactory.createSqlToRexConverter(inputRowType, outputType);
128+
final RexNode rexNode = sqlToRexConverter.convertToRexNode(sqlExpression);
129+
final LogicalType logicalType = FlinkTypeFactory.toLogicalType(rexNode.getType());
130+
// expand expression for serializable expression strings similar to views
131+
final String sqlExpressionExpanded = sqlToRexConverter.expand(sqlExpression);
132+
return new RexNodeExpression(
133+
rexNode,
134+
TypeConversions.fromLogicalToDataType(logicalType),
135+
sqlExpression,
136+
sqlExpressionExpanded);
137+
} catch (Throwable t) {
138+
throw new ValidationException(
139+
String.format("Invalid SQL expression: %s", sqlExpression), t);
140+
}
141+
}
142+
143+
public String[] getCompletionHints(String statement, int cursor) {
144+
List<String> candidates =
145+
new ArrayList<>(
146+
Arrays.asList(EXTENDED_PARSER.getCompletionHints(statement, cursor)));
147+
148+
// use sql advisor
149+
SqlAdvisorValidator validator = validatorSupplier.get().getSqlAdvisorValidator();
150+
SqlAdvisor advisor =
151+
new SqlAdvisor(validator, validatorSupplier.get().config().getParserConfig());
152+
String[] replaced = new String[1];
153+
154+
List<String> sqlHints =
155+
advisor.getCompletionHints(statement, cursor, replaced).stream()
156+
.map(item -> item.toIdentifier().toString())
157+
.collect(Collectors.toList());
158+
159+
candidates.addAll(sqlHints);
160+
161+
return candidates.toArray(new String[0]);
162+
}
163+
164+
public CatalogManager getCatalogManager() {
165+
return catalogManager;
166+
}
167+
}
Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,34 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.lineage.flink.sql.metadata;
20+
21+
public class Constant {
22+
23+
private Constant() {
24+
throw new IllegalStateException("Utility class");
25+
}
26+
27+
public static final String DELIMITER = ".";
28+
29+
public static final int INITIAL_CAPACITY = 16;
30+
31+
public static final String ILLEGAL_PARAM = "illegal param";
32+
33+
public static final Long DEFAULT_USER_ID = 0L;
34+
}

0 commit comments

Comments
 (0)