diff --git a/fire-bundle/fire-bundle-spark/pom.xml b/fire-bundle/fire-bundle-spark/pom.xml index f2154d215cf66cf232a17e43d3c6a4c69f68bbfa..607c1d67759a1b0a0971a0c6cdbaca6336db2ccd 100644 --- a/fire-bundle/fire-bundle-spark/pom.xml +++ b/fire-bundle/fire-bundle-spark/pom.xml @@ -13,11 +13,11 @@ - + io.netty diff --git a/fire-common/pom.xml b/fire-common/pom.xml index e021e9696aae398ed16c062c2c64f1618ba62096..fd76428f9d477c23e3563604e78e89578329cd7a 100644 --- a/fire-common/pom.xml +++ b/fire-common/pom.xml @@ -102,6 +102,12 @@ jsqlparser ${jsqlparser.version} + + org.projectlombok + lombok + 1.18.26 + compile + diff --git a/fire-common/src/main/java/com/zto/fire/common/bean/lineage/LineageResult.java b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/LineageResult.java new file mode 100644 index 0000000000000000000000000000000000000000..4b7c707aa8cd3a7fbb4e31f69a3e1cc090acdd90 --- /dev/null +++ b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/LineageResult.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.common.bean.lineage; + +import com.zto.fire.common.util.Constant; +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * @description: Result + * @author: lmz + */ +@Data +@Builder +@AllArgsConstructor +public class LineageResult { + + private String sourceCatalog; + + private String sourceDatabase; + + private String sourceTable; + + private String sourceColumn; + + private String targetCatalog; + + private String targetDatabase; + + private String targetTable; + + private String targetColumn; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + public LineageResult(String sourceTablePath, String sourceColumn, String targetTablePath, String targetColumn, + String transform) { + String[] sourceItems = sourceTablePath.split("\\" + Constant.DELIMITER); + String[] targetItems = targetTablePath.split("\\" + Constant.DELIMITER); + + this.sourceCatalog = sourceItems[0]; + this.sourceDatabase = sourceItems[1]; + this.sourceTable = sourceItems[2]; + this.sourceColumn = sourceColumn; + this.targetCatalog = targetItems[0]; + this.targetDatabase = targetItems[1]; + this.targetTable = targetItems[2]; + this.targetColumn = targetColumn; + this.transform = transform; + } + + public LineageResult(String catalog, String database, String sourceTable, String sourceColumn, String targetTable, + String targetColumn) { + this.sourceCatalog = catalog; + this.sourceDatabase = database; + this.sourceTable = sourceTable; + this.sourceColumn = sourceColumn; + this.targetCatalog = catalog; + this.targetDatabase = database; + this.targetTable = targetTable; + this.targetColumn = targetColumn; + } + + public static List buildResult(String catalog, String database, String[][] expectedArray) { + return Stream.of(expectedArray) + .map(e -> { + LineageResult result = new LineageResult(catalog, database, e[0], e[1], e[2], e[3]); + // transform field is optional + if (e.length == 5) { + result.setTransform(e[4]); + } + return result; + }).collect(Collectors.toList()); + } +} diff --git a/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableColumnsRelations.java b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableColumnsRelations.java new file mode 100644 index 0000000000000000000000000000000000000000..c82aad7938c4238c0bfb2bff46632e4d61724f99 --- /dev/null +++ b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableColumnsRelations.java @@ -0,0 +1,68 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.zto.fire.common.bean.lineage; + +import java.util.Objects; + +/** + * 用于封装采集到SQL的实时血缘信息:字段级血缘 + * @author wsczm + */ +public class SQLTableColumnsRelations { + private String sourceColumn; + private String targetColumn; + + public SQLTableColumnsRelations(String sourceColumn, String targetColumn) { + this.sourceColumn = sourceColumn; + this.targetColumn = targetColumn; + } + + public String getSourceColumn() { + return sourceColumn; + } + + public void setSourceColumn(String sourceColumn) { + this.sourceColumn = sourceColumn; + } + + public String getTargetColumn() { + return targetColumn; + } + + + public void setTargetColumn(String targetColumn) { + this.targetColumn = targetColumn; + } + + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SQLTableColumnsRelations that = (SQLTableColumnsRelations) o; + return Objects.equals(sourceColumn, that.sourceColumn) && Objects.equals(targetColumn, that.targetColumn); + } + + @Override + public int hashCode() { + return Objects.hash(sourceColumn, targetColumn); + } + +} diff --git a/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableRelations.java b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableRelations.java index 65cdb564be6b1b7cb4c9a72c8770f8cfbe334d85..6f3d23999321e660ba6da7ac1058c68280ec9588 100644 --- a/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableRelations.java +++ b/fire-common/src/main/java/com/zto/fire/common/bean/lineage/SQLTableRelations.java @@ -17,6 +17,7 @@ package com.zto.fire.common.bean.lineage; +import java.util.HashSet; import java.util.Objects; /** @@ -37,12 +38,16 @@ public class SQLTableRelations { */ private String sinkTable; + private HashSet sqlTableColumnsRelations; + public SQLTableRelations() { } - public SQLTableRelations(String srcTable, String sinkTable) { + + public SQLTableRelations(String srcTable, String sinkTable, HashSet sqlTableColumnsRelations) { this.srcTable = srcTable; this.sinkTable = sinkTable; + this.sqlTableColumnsRelations = sqlTableColumnsRelations; } public void setSrcTable(String srcTable) { @@ -61,20 +66,24 @@ public class SQLTableRelations { return sinkTable; } + public HashSet getSqlTableColumnsRelations() { + return sqlTableColumnsRelations; + } + + public void setSqlTableColumnsRelations(HashSet sqlTableColumnsRelations) { + this.sqlTableColumnsRelations = sqlTableColumnsRelations; + } + @Override public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; SQLTableRelations that = (SQLTableRelations) o; - return Objects.equals(srcTable, that.srcTable) && Objects.equals(sinkTable, that.sinkTable); + return Objects.equals(srcTable, that.srcTable) && Objects.equals(sinkTable, that.sinkTable) && Objects.equals(sqlTableColumnsRelations, that.sqlTableColumnsRelations); } @Override public int hashCode() { - return Objects.hash(srcTable, sinkTable); + return Objects.hash(srcTable, sinkTable, sqlTableColumnsRelations); } } \ No newline at end of file diff --git a/fire-common/src/main/java/com/zto/fire/common/pool/ClassEntity.java b/fire-common/src/main/java/com/zto/fire/common/pool/ClassEntity.java new file mode 100644 index 0000000000000000000000000000000000000000..1b065f6e2c6db6ef0b2fb92156cc9c0c7d83530f --- /dev/null +++ b/fire-common/src/main/java/com/zto/fire/common/pool/ClassEntity.java @@ -0,0 +1,59 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package com.zto.fire.common.pool; + +import lombok.Getter; +import lombok.Setter; + +import java.util.Objects; + +/** + * ClassEntity + */ +@Getter +@Setter +public class ClassEntity { + private String name; + private String code; + private byte[] classByte; + + public ClassEntity(String name, String code) { + this.name = name; + this.code = code; + } + + public ClassEntity(String name, String code, byte[] classByte) { + this.name = name; + this.code = code; + this.classByte = classByte; + } + + public static ClassEntity build(String name, String code) { + return new ClassEntity(name, code); + } + + public boolean equals(ClassEntity entity) { + if (name.equals(entity.getName()) && code.equals(entity.getCode())) { + return true; + } else { + return false; + } + } +} diff --git a/fire-common/src/main/java/com/zto/fire/common/pool/ClassPool.java b/fire-common/src/main/java/com/zto/fire/common/pool/ClassPool.java new file mode 100644 index 0000000000000000000000000000000000000000..a3b4d6775ee5ea0807d40a344c9c5a6d7c8316b9 --- /dev/null +++ b/fire-common/src/main/java/com/zto/fire/common/pool/ClassPool.java @@ -0,0 +1,59 @@ + +package com.zto.fire.common.pool; + +import java.util.List; +import java.util.Vector; + +/** + * ClassPool + */ +public class ClassPool { + + private static volatile List classList = new Vector<>(); + + public static boolean exist(String name) { + for (ClassEntity executorEntity : classList) { + if (executorEntity.getName().equals(name)) { + return true; + } + } + return false; + } + + public static boolean exist(ClassEntity entity) { + for (ClassEntity executorEntity : classList) { + if (executorEntity.equals(entity)) { + return true; + } + } + return false; + } + + public static Integer push(ClassEntity executorEntity) { + if (exist(executorEntity.getName())) { + remove(executorEntity.getName()); + } + classList.add(executorEntity); + return classList.size(); + } + + public static Integer remove(String name) { + int count = classList.size(); + for (int i = 0; i < classList.size(); i++) { + if (name.equals(classList.get(i).getName())) { + classList.remove(i); + break; + } + } + return count - classList.size(); + } + + public static ClassEntity get(String name) { + for (ClassEntity executorEntity : classList) { + if (executorEntity.getName().equals(name)) { + return executorEntity; + } + } + return null; + } +} diff --git a/fire-common/src/main/java/com/zto/fire/common/util/Constant.java b/fire-common/src/main/java/com/zto/fire/common/util/Constant.java new file mode 100644 index 0000000000000000000000000000000000000000..fc59e64941156a5a610a472430720be3acea8174 --- /dev/null +++ b/fire-common/src/main/java/com/zto/fire/common/util/Constant.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.common.util; + +/** + * @description: Constant + * @author: lmz + */ +public class Constant { + + private Constant() { + throw new IllegalStateException("Utility class"); + } + + public static final String DELIMITER = "."; + + public static final int INITIAL_CAPACITY = 16; + + public static final String ILLEGAL_PARAM = "illegal param"; + + public static final Long DEFAULT_USER_ID = 0L; +} diff --git a/fire-common/src/main/scala/com/zto/fire/common/lineage/SQLLineageManager.scala b/fire-common/src/main/scala/com/zto/fire/common/lineage/SQLLineageManager.scala index 3710c1332fc7ac1e610c52c679d9be1f37030b18..a08e6efadeec853fd3582a29abb1c5d61756ca92 100644 --- a/fire-common/src/main/scala/com/zto/fire/common/lineage/SQLLineageManager.scala +++ b/fire-common/src/main/scala/com/zto/fire/common/lineage/SQLLineageManager.scala @@ -48,10 +48,20 @@ private[fire] object SQLLineageManager { * @param sinkTable * 目标表 */ - def addRelation(srcTableIdentifier: TableIdentifier, sinkTableIdentifier: TableIdentifier): Unit = { - this.relationSet.add(new SQLTableRelations(srcTableIdentifier.toString, sinkTableIdentifier.toString)) + def addRelation(srcTableIdentifier: TableIdentifier, sinkTableIdentifier: TableIdentifier, relationSet: JHashSet[SQLTableColumnsRelations]): Unit = { + this.relationSet.add(new SQLTableRelations(srcTableIdentifier.toString, sinkTableIdentifier.toString, relationSet)) } + /** + * 维护表与表字段之间的关系 + * + * @param sourceColumn + * 数据来源表 + * @param targetColumn + * 目标表 + */ + + /** * 获取SQL血缘信息 */ @@ -247,4 +257,5 @@ private[fire] object SQLLineageManager { } } + } diff --git a/fire-core/src/main/resources/fire.properties b/fire-core/src/main/resources/fire.properties index ff10838d74d7de5c52544631c923ae35a1d64209..672166446aeef7cec4504ba976ff700d9011e5a8 100644 --- a/fire-core/src/main/resources/fire.properties +++ b/fire-core/src/main/resources/fire.properties @@ -99,7 +99,9 @@ fire.lineage.debug.enable = # \u662F\u5426\u91C7\u96C6\u539F\u59CB\u7684SQL\u8BED\u53E5 fire.lineage.collect_sql.enable = true # \u662F\u5426\u5F00\u542F\u5C06\u8840\u7F18\u4FE1\u606F\u53D1\u9001\u5230\u6D88\u606F\u961F\u5217 -fire.lineage.send.mq.enable = false +fire.lineage.send.mq.enable = true +fire.lineage.send.mq.url = node01:9092 +fire.lineage.send.mq.topic = fire_lineage # \u8840\u7F18\u89E3\u6790\u5F02\u6B65\u7EBF\u7A0B\u6267\u884C\u7684\u6B21\u6570 fire.lineage.run.count = 360 # \u8840\u7F18\u89E3\u6790\u5B58\u653E\u7684\u961F\u5217\u6570\u6700\u5927\u503C diff --git a/fire-core/src/main/scala/com/zto/fire/core/sql/SqlParser.scala b/fire-core/src/main/scala/com/zto/fire/core/sql/SqlParser.scala index 292d8786d568f0661d08a65afbf3a27e47d20568..f4a70eea5fa9c5157a36d0d06c5b64e38c72f020 100644 --- a/fire-core/src/main/scala/com/zto/fire/core/sql/SqlParser.scala +++ b/fire-core/src/main/scala/com/zto/fire/core/sql/SqlParser.scala @@ -21,10 +21,10 @@ import com.zto.fire.common.anno.Internal import com.zto.fire.common.bean.TableIdentifier import com.zto.fire.common.conf.FireFrameworkConf._ import com.zto.fire.common.lineage.{LineageManager, SQLLineageManager} -import com.zto.fire.common.lineage.parser.connector._ import com.zto.fire.common.util.{Logging, SQLUtils, ThreadUtils} import com.zto.fire.predef._ + import java.util.concurrent.{CopyOnWriteArraySet, TimeUnit} /** @@ -46,7 +46,8 @@ private[fire] trait SqlParser extends Logging { protected def sqlParse: Unit = { if (lineageEnable) { ThreadUtils.scheduleWithFixedDelay({ - this.buffer.foreach(sql => this.sqlParser(sql)) + this.buffer.foreach( + sql => this.sqlParser(sql)) this.clear }, lineageRunInitialDelay, lineageRunPeriod, TimeUnit.SECONDS) } diff --git a/fire-engines/fire-flink/pom.xml b/fire-engines/fire-flink/pom.xml index bc67087c4bddb7c794fc6bf1a133f098c44d1745..736717751dc0aebb72d15ab523653b4f98104935 100644 --- a/fire-engines/fire-flink/pom.xml +++ b/fire-engines/fire-flink/pom.xml @@ -170,6 +170,16 @@ org.apache.hive hive-exec ${hive.flink.version} + + + org.apache.calcite + calcite + + + org.apache.calcite + calcite-core + + ${maven.scope} diff --git a/fire-engines/fire-flink/src/main/scala-flink-1.12/com/zto/fire/flink/lineage/LineageContext.scala b/fire-engines/fire-flink/src/main/scala-flink-1.12/com/zto/fire/flink/lineage/LineageContext.scala new file mode 100644 index 0000000000000000000000000000000000000000..5742e5c75d5bcadaa2818863472759e605ec141e --- /dev/null +++ b/fire-engines/fire-flink/src/main/scala-flink-1.12/com/zto/fire/flink/lineage/LineageContext.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.flink.lineage + +import com.zto.fire.common.bean.lineage.LineageResult +import com.zto.fire.common.util.{Constant, Logging} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelColumnOrigin, RelMetadataQueryBase} +import org.apache.commons.lang.StringUtils +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.operations.{CatalogSinkModifyOperation, Operation} +import org.apache.flink.table.planner.operations.PlannerQueryOperation +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.schema.TableSourceTable +import java.util +import scala.collection.JavaConverters.{asScalaBufferConverter, asScalaSetConverter, seqAsJavaListConverter} +import scala.collection.mutable.ListBuffer +import scala.language.postfixOps + +/** + * flink sql解析器 解析表字段血缘关系 + * @author wsczm + */ +class LineageContext(tableEnv: TableEnvironmentImpl) extends Logging { + + /** + * Sql 字段检验 + */ + private def validateSchema(sinkTable: String, relNode: RelNode, sinkFieldList: util.List[String]): Unit = { + val queryFieldList = relNode.getRowType.getFieldNames + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException( + String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)) + } + } + + /** + * 获取血缘关系 + * @param sinkTable + * @param optRelNode + */ + def buildFiledLineageResult(sinkTable: String, optRelNode: RelNode): ListBuffer[LineageResult] = { + val targetColumnList: util.List[String] = tableEnv.from(sinkTable) + .getSchema + .getTableColumns + .asScala + .map(tableColumn => tableColumn.getName) + .toList + .asJava + validateSchema(sinkTable, optRelNode, targetColumnList) + val metadataQuery = optRelNode.getCluster.getMetadataQuery + val resultList = ListBuffer[LineageResult]() + for (index <- 0 until targetColumnList.size) { + val targetColumn = targetColumnList.get(index) + val relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index).asScala + if (relColumnOriginSet.nonEmpty) { + for (rco: RelColumnOrigin <- relColumnOriginSet) { + // table + val table = rco.getOriginTable + val sourceTable = String.join(Constant.DELIMITER, table.getQualifiedName) + // field + val ordinal = rco.getOriginColumnOrdinal + val fieldNames = table.asInstanceOf[TableSourceTable].catalogTable.getSchema.getFieldNames + val sourceColumn = fieldNames(ordinal) + this.logger.info("----------------------------------------------------------") + this.logger.info("Source table: {}", sourceTable) + this.logger.info("Source column: {}", sourceColumn) + if (StringUtils.isNotEmpty(rco.getTransform)) { + this.logger.info("transform: {}", rco.getTransform) + } + // add record + resultList += new LineageResult(sourceTable, sourceColumn, sinkTable, targetColumn, rco.getTransform) + } + } + } + resultList + } + + /** + * 获取血缘关系 + * @param sql INSERT INTO + * 1、获取 RelNode + * 2、根据RelNode 构造血缘 + */ + def analyzeLineage(sql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val parsed = parseStatement(sql) + val sinkTable = parsed._1 + val oriRelNode = parsed._2 + buildFiledLineageResult(sinkTable, oriRelNode) + } + + /** + * 根据SqlNode和元数据信息构建关系表达式RelNode树 + * @param singleSql INSERT INTO + */ + private def parseStatement(singleSql: String): Tuple2[String, RelNode] = { + val operation = parseValidateConvert(singleSql) + operation match { + case sinkOperation: CatalogSinkModifyOperation => + val queryOperation = sinkOperation.getChild.asInstanceOf[PlannerQueryOperation] + val relNode = queryOperation.getCalciteTree + Tuple2(sinkOperation.getTableIdentifier.asSummaryString(), relNode) + case _ => + throw new TableException("Only insert is supported now.") + } + } + + /** + * 获取Sql对应的 Operation 类型 + * + * @param singleSql INSERT INTO + */ + private def parseValidateConvert(singleSql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operations: util.List[Operation] = tableEnv.getParser.parse(singleSql) + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement.") + } + operations.get(0) + } +} diff --git a/fire-engines/fire-flink/src/main/scala-flink-1.13/com/zto/fire/flink/lineage/LineageContext.scala b/fire-engines/fire-flink/src/main/scala-flink-1.13/com/zto/fire/flink/lineage/LineageContext.scala new file mode 100644 index 0000000000000000000000000000000000000000..86c2f65d10c6a5388bcea8bb1769ca6707c59432 --- /dev/null +++ b/fire-engines/fire-flink/src/main/scala-flink-1.13/com/zto/fire/flink/lineage/LineageContext.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.flink.lineage + +import com.zto.fire.common.bean.lineage.LineageResult +import com.zto.fire.common.util.{Constant, Logging} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelColumnOrigin, RelMetadataQueryBase} +import org.apache.commons.lang.StringUtils +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.operations.{CatalogSinkModifyOperation, Operation} +import org.apache.flink.table.planner.operations.PlannerQueryOperation +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.schema.TableSourceTable +import java.util +import scala.collection.JavaConverters.asScalaSetConverter +import scala.collection.mutable.ListBuffer +import scala.language.postfixOps + +/** + * flink sql解析器 解析表字段血缘关系 + * + * @author wsczm + */ +class LineageContext(tableEnv: TableEnvironmentImpl) extends Logging { + + /** + * Sql 字段检验 + */ + private def validateSchema(sinkTable: String, relNode: RelNode, sinkFieldList: util.List[String]): Unit = { + val queryFieldList = relNode.getRowType.getFieldNames + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException( + String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)) + } + } + + /** + * 获取血缘关系 + * @param sinkTable + * @param optRelNode + */ + def buildFiledLineageResult(sinkTable: String, optRelNode: RelNode): ListBuffer[LineageResult] = { + val targetColumnList = tableEnv.from(sinkTable).getResolvedSchema.getColumnNames + validateSchema(sinkTable, optRelNode, targetColumnList) + val metadataQuery = optRelNode.getCluster.getMetadataQuery + val resultList = ListBuffer[LineageResult]() + for (index <- 0 until targetColumnList.size) { + val targetColumn = targetColumnList.get(index) + val relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index).asScala + import scala.collection.JavaConverters + if (relColumnOriginSet.nonEmpty) { + for (rco: RelColumnOrigin <- relColumnOriginSet) { + // table + val table = rco.getOriginTable + val sourceTable = String.join(Constant.DELIMITER, table.getQualifiedName) + // field + val ordinal = rco.getOriginColumnOrdinal + val fieldNames = table.asInstanceOf[TableSourceTable].catalogTable.getResolvedSchema.getColumnNames + val sourceColumn = fieldNames.get(ordinal) + this.logger.info("----------------------------------------------------------") + this.logger.info("Source table: {}", sourceTable) + this.logger.info("Source column: {}", sourceColumn) + if (StringUtils.isNotEmpty(rco.getTransform)) { + this.logger.info("transform: {}", rco.getTransform) + } + // add record + resultList += new LineageResult(sourceTable, sourceColumn, sinkTable, targetColumn, rco.getTransform) + } + } + } + resultList + } + + /** + * 获取血缘关系 + * @param sql INSERT INTO + *1、获取 RelNode + *2、根据RelNode 构造血缘 + */ + def analyzeLineage(sql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val parsed = parseStatement(sql) + val sinkTable = parsed._1 + val oriRelNode = parsed._2 + buildFiledLineageResult(sinkTable, oriRelNode) + } + + /** + * 根据SqlNode和元数据信息构建关系表达式RelNode树 + * @param singleSql INSERT INTO + */ + private def parseStatement(singleSql: String): Tuple2[String, RelNode] = { + val operation = parseValidateConvert(singleSql) + operation match { + case sinkOperation: CatalogSinkModifyOperation => + val queryOperation = sinkOperation.getChild.asInstanceOf[PlannerQueryOperation] + val relNode = queryOperation.getCalciteTree + Tuple2(sinkOperation.getTableIdentifier.asSummaryString(), relNode) + case _ => + throw new TableException("Only insert is supported now.") + } + } + + /** + * 获取Sql对应的 Operation 类型 + * @param singleSql INSERT INTO + */ + private def parseValidateConvert(singleSql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operations: util.List[Operation] = tableEnv.getParser.parse(singleSql) + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement.") + } + operations.get(0) + } +} diff --git a/fire-engines/fire-flink/src/main/scala-flink-1.14/com/zto/fire/flink/lineage/LineageContext.scala b/fire-engines/fire-flink/src/main/scala-flink-1.14/com/zto/fire/flink/lineage/LineageContext.scala new file mode 100644 index 0000000000000000000000000000000000000000..50d72f0cabdeb7ba5b0b19ff0525aba0b13aa465 --- /dev/null +++ b/fire-engines/fire-flink/src/main/scala-flink-1.14/com/zto/fire/flink/lineage/LineageContext.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.flink.lineage + +import com.zto.fire.common.bean.lineage.LineageResult +import com.zto.fire.common.util.Constant +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelColumnOrigin, RelMetadataQueryBase} +import org.apache.commons.lang3.StringUtils +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.operations.{CatalogSinkModifyOperation, Operation} +import org.apache.flink.table.planner.operations.PlannerQueryOperation +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.schema.TableSourceTable +import java.util +import scala.collection.JavaConverters.asScalaSetConverter +import scala.collection.mutable.ListBuffer +import scala.language.postfixOps + +/** + * flink sql解析器 解析表字段血缘关系 + * + * @author wsczm + */ +class LineageContext(tableEnv: TableEnvironmentImpl) { + + /** + * Sql 字段检验 + */ + private def validateSchema(sinkTable: String, relNode: RelNode, sinkFieldList: util.List[String]): Unit = { + val queryFieldList = relNode.getRowType.getFieldNames + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException( + String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)) + } + } + + /** + * 获取血缘关系 + */ + def buildFiledLineageResult(sinkTable: String, optRelNode: RelNode):ListBuffer[LineageResult] = { + val targetColumnList = tableEnv.from(sinkTable) + .getResolvedSchema + .getColumnNames + validateSchema(sinkTable, optRelNode, targetColumnList) + val metadataQuery = optRelNode.getCluster.getMetadataQuery + val resultList = ListBuffer[LineageResult]() + + for (index <- 0 until targetColumnList.size) { + val targetColumn = targetColumnList.get(index) + + val relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index).asScala + if (relColumnOriginSet.nonEmpty) { + for (rco: RelColumnOrigin <- relColumnOriginSet) { + // table + val table = rco.getOriginTable + val sourceTable = String.join(Constant.DELIMITER, table.getQualifiedName) + // field + val ordinal = rco.getOriginColumnOrdinal + val fieldNames = table.asInstanceOf[TableSourceTable].catalogTable.getResolvedSchema.getColumnNames + val sourceColumn = fieldNames.get(ordinal) + println("----------------------------------------------------------") + println("Source table: {}", sourceTable) + println("Source column: {}", sourceColumn) + if (StringUtils.isNotEmpty(rco.getTransform)) { + println("transform: {}", rco.getTransform) + } + // add record + resultList += new LineageResult(sourceTable, sourceColumn, sinkTable, targetColumn, rco.getTransform) + } + } + } + resultList + } + + /** + * 获取血缘关系 + * @param sql INSERT INTO + * 1、获取 RelNode + * 2、根据RelNode 构造血缘 + */ + def analyzeLineage(sql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val parsed = parseStatement(sql) + val sinkTable = parsed._1 + val oriRelNode = parsed._2 + buildFiledLineageResult(sinkTable, oriRelNode) + } + + /** + * 根据SqlNode和元数据信息构建关系表达式RelNode树 + * @param singleSql INSERT INTO + */ + private def parseStatement(singleSql: String): Tuple2[String, RelNode] = { + val operation = parseValidateConvert(singleSql) + operation match { + case sinkOperation: CatalogSinkModifyOperation => + val queryOperation = sinkOperation.getChild.asInstanceOf[PlannerQueryOperation] + val relNode = queryOperation.getCalciteTree + Tuple2(sinkOperation.getTableIdentifier.asSummaryString(), relNode) + case _ => + throw new TableException("Only insert is supported now.") + } + } + + /** + * 获取Sql对应的 Operation 类型 + * @param singleSql INSERT INTO + */ + private def parseValidateConvert(singleSql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operations: util.List[Operation] = tableEnv.getParser.parse(singleSql) + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement.") + } + operations.get(0) + } +} diff --git a/fire-engines/fire-flink/src/main/scala-flink-1.15/com/zto/fire/flink/lineage/LineageContext.scala b/fire-engines/fire-flink/src/main/scala-flink-1.15/com/zto/fire/flink/lineage/LineageContext.scala new file mode 100644 index 0000000000000000000000000000000000000000..e25be5d387e314673fe46dc0753aab080c4a8409 --- /dev/null +++ b/fire-engines/fire-flink/src/main/scala-flink-1.15/com/zto/fire/flink/lineage/LineageContext.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.zto.fire.flink.lineage + +import com.zto.fire.common.bean.lineage.LineageResult +import com.zto.fire.common.util.Constant +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelColumnOrigin, RelMetadataQueryBase} +import org.apache.commons.lang3.StringUtils +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.operations.{Operation, SinkModifyOperation} +import org.apache.flink.table.planner.operations.PlannerQueryOperation +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.schema.TableSourceTable +import java.util +import scala.collection.mutable.ListBuffer +import scala.language.postfixOps +import scala.collection.JavaConversions._ + + +class LineageContext(tableEnv: TableEnvironmentImpl) { + + /** + * Sql 字段检验 + */ + private def validateSchema(sinkTable: String, relNode: RelNode, sinkFieldList: util.List[String]): Unit = { + val queryFieldList = relNode.getRowType.getFieldNames + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException( + String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)) + } + } + + /** + * 获取血缘关系 + * @param sinkTable + * @param optRelNode + */ + def buildFiledLineageResult(sinkTable: String, optRelNode: RelNode):ListBuffer[LineageResult] = { + val targetColumnList = tableEnv.from(sinkTable) + .getResolvedSchema + .getColumnNames + validateSchema(sinkTable, optRelNode, targetColumnList) + val metadataQuery = optRelNode.getCluster.getMetadataQuery + val resultList = ListBuffer[LineageResult]() + for (index <- 0 until targetColumnList.size) { + val targetColumn = targetColumnList.get(index) + val relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index) + if (!relColumnOriginSet.nonEmpty) { + for (rco: RelColumnOrigin <- relColumnOriginSet) { + // table + val table = rco.getOriginTable + val sourceTable = String.join(Constant.DELIMITER, table.getQualifiedName) + // field + val ordinal = rco.getOriginColumnOrdinal + val fieldNames = table.asInstanceOf[TableSourceTable].contextResolvedTable.getResolvedSchema.getColumnNames + val sourceColumn = fieldNames.get(ordinal) + println("----------------------------------------------------------") + println("Source table: {}", sourceTable) + println("Source column: {}", sourceColumn) + if (StringUtils.isNotEmpty(rco.getTransform)) { + println("transform: {}", rco.getTransform) + } + // add record + resultList += new LineageResult(sourceTable, sourceColumn, sinkTable, targetColumn, rco.getTransform) + } + } + } + resultList + } + + /** + * 获取血缘关系 + * @param sql INSERT INTO + * 1、获取 RelNode + * 2、根据RelNode 构造血缘 + */ + def analyzeLineage(sql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val parsed = parseStatement(sql) + val sinkTable = parsed._1 + val oriRelNode = parsed._2 + buildFiledLineageResult(sinkTable, oriRelNode) + } + + /** + * 根据SqlNode和元数据信息构建关系表达式RelNode树 + * @param singleSql INSERT INTO + */ + private def parseStatement(singleSql: String): Tuple2[String, RelNode] = { + val operation = parseValidateConvert(singleSql) + operation match { + case sinkOperation: SinkModifyOperation => + val queryOperation = sinkOperation.getChild.asInstanceOf[PlannerQueryOperation] + val relNode = queryOperation.getCalciteTree + Tuple2(sinkOperation.getContextResolvedTable.getIdentifier.asSummaryString(), relNode) + case _ => + throw new TableException("Only insert is supported now.") + } + } + + /** + * 获取Sql对应的 Operation 类型 + * @param singleSql INSERT INTO + */ + private def parseValidateConvert(singleSql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operations: util.List[Operation] = tableEnv.getParser.parse(singleSql) + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement.") + } + operations.get(0) + } +} diff --git a/fire-engines/fire-flink/src/main/scala-flink-1.16/com/zto/fire/flink/lineage/LineageContext.scala b/fire-engines/fire-flink/src/main/scala-flink-1.16/com/zto/fire/flink/lineage/LineageContext.scala new file mode 100644 index 0000000000000000000000000000000000000000..285c41cdde7879c4e9fee104f5356b1054cd75b7 --- /dev/null +++ b/fire-engines/fire-flink/src/main/scala-flink-1.16/com/zto/fire/flink/lineage/LineageContext.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.zto.fire.flink.lineage + +import com.zto.fire.common.bean.lineage.LineageResult +import com.zto.fire.common.util.Constant +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelColumnOrigin, RelMetadataQueryBase} +import org.apache.commons.lang3.StringUtils +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.operations.{Operation, SinkModifyOperation} +import org.apache.flink.table.planner.operations.PlannerQueryOperation +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.schema.TableSourceTable + +import java.util +import scala.collection.mutable.ListBuffer +import scala.language.postfixOps + + +class LineageContext(tableEnv: TableEnvironmentImpl) { + + /** + * Sql 字段检验 + */ + private def validateSchema(sinkTable: String, relNode: RelNode, sinkFieldList: util.List[String]): Unit = { + val queryFieldList = relNode.getRowType.getFieldNames + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException( + String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)) + } + } + + /** + * 获取血缘关系 + * @param sinkTable + * @param optRelNode + */ + def buildFiledLineageResult(sinkTable: String, optRelNode: RelNode): ListBuffer[LineageResult] = { + val targetColumnList = tableEnv.from(sinkTable) + .getResolvedSchema + .getColumnNames + validateSchema(sinkTable, optRelNode, targetColumnList) + val metadataQuery = optRelNode.getCluster.getMetadataQuery + val resultList = ListBuffer[LineageResult]() + + for (index <- 0 until targetColumnList.size) { + val targetColumn = targetColumnList.get(index) + + val relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index) + import scala.collection.JavaConversions._ + + if (!relColumnOriginSet.nonEmpty) { + for (rco: RelColumnOrigin <- relColumnOriginSet) { + // table + val table = rco.getOriginTable + val sourceTable = String.join(Constant.DELIMITER, table.getQualifiedName) + // field + val ordinal = rco.getOriginColumnOrdinal + val fieldNames = table.asInstanceOf[TableSourceTable].contextResolvedTable.getResolvedSchema.getColumnNames + val sourceColumn = fieldNames.get(ordinal) + println("----------------------------------------------------------") + println("Source table: {}", sourceTable) + println("Source column: {}", sourceColumn) + if (StringUtils.isNotEmpty(rco.getTransform)) { + println("transform: {}", rco.getTransform) + } + // add record + resultList += new LineageResult(sourceTable, sourceColumn, sinkTable, targetColumn, rco.getTransform) + } + } + } + resultList + } + + /** + * 获取血缘关系 + * @param sql INSERT INTO + * 1、获取 RelNode + * 2、根据RelNode 构造血缘 + */ + def analyzeLineage(sql: String) = { + + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val parsed = parseStatement(sql) + val sinkTable = parsed._1 + val oriRelNode = parsed._2 + buildFiledLineageResult(sinkTable, oriRelNode) + } + + /** + * 根据SqlNode和元数据信息构建关系表达式RelNode树 + * @param singleSql INSERT INTO + */ + private def parseStatement(singleSql: String): Tuple2[String, RelNode] = { + val operation = parseValidateConvert(singleSql) + operation match { + case sinkOperation: SinkModifyOperation => + val queryOperation = sinkOperation.getChild.asInstanceOf[PlannerQueryOperation] + val relNode = queryOperation.getCalciteTree + Tuple2(sinkOperation.getContextResolvedTable.getIdentifier.asSummaryString(), relNode) + case _ => + throw new TableException("Only insert is supported now.") + } + } + + /** + * 获取Sql对应的 Operation 类型 + * @param singleSql INSERT INTO + */ + private def parseValidateConvert(singleSql: String) = { + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operations: util.List[Operation] = tableEnv.getParser.parse(singleSql) + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement.") + } + operations.get(0) + } +} diff --git a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/ext/stream/StreamExecutionEnvExt.scala b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/ext/stream/StreamExecutionEnvExt.scala index fb402937ecedd4a344ef8a5cd31b8a277d038b8f..10f07563a68f7c1e56fdec2eac8d0b473b02b06d 100644 --- a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/ext/stream/StreamExecutionEnvExt.scala +++ b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/ext/stream/StreamExecutionEnvExt.scala @@ -627,5 +627,7 @@ private[fire] object StreamExecutionEnvExt { /** * 创建并返回StatementSet对象实例 */ - def createStatementSet: StatementSet = FlinkSingletonFactory.getTableEnv.createStatementSet() + def createStatementSet: StatementSet = + + FlinkSingletonFactory.getTableEnv.createStatementSet() } \ No newline at end of file diff --git a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/sql/FlinkSqlParserBase.scala b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/sql/FlinkSqlParserBase.scala index 56dd2eebac3775c5d48515995c87578787c657e6..f2b3225be697a23f584237392e8b25d69ccef165 100644 --- a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/sql/FlinkSqlParserBase.scala +++ b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/sql/FlinkSqlParserBase.scala @@ -20,6 +20,7 @@ package com.zto.fire.flink.sql import com.zto.fire._ import com.zto.fire.common.anno.Internal import com.zto.fire.common.bean.TableIdentifier +import com.zto.fire.common.bean.lineage.SQLTableColumnsRelations import com.zto.fire.common.conf.FireHiveConf import com.zto.fire.common.enu.{Datasource, Operation} import com.zto.fire.common.lineage.SQLLineageManager @@ -27,14 +28,20 @@ import com.zto.fire.common.lineage.parser.ConnectorParserManager import com.zto.fire.common.util.{ReflectionUtils, RegularUtils} import com.zto.fire.core.sql.SqlParser import com.zto.fire.flink.conf.FireFlinkConf +import com.zto.fire.flink.lineage.LineageContext import com.zto.fire.flink.util.{FlinkSingletonFactory, FlinkUtils} import com.zto.fire.jdbc.conf.FireJdbcConf +import com.zto.fire.predef.JHashSet import org.apache.calcite.sql._ +import org.apache.flink.configuration.Configuration import org.apache.flink.sql.parser.SqlProperty import org.apache.flink.sql.parser.ddl._ import org.apache.flink.sql.parser.dml._ import org.apache.flink.sql.parser.hive.dml.RichSqlHiveInsert +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api.EnvironmentSettings import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment +import org.apache.flink.table.api.internal.TableEnvironmentImpl import org.apache.flink.table.catalog.ObjectPath import org.apache.flink.table.catalog.hive.HiveCatalog import org.apache.hadoop.hive.metastore.api.Table @@ -52,6 +59,13 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { // calcite parser config protected lazy val tableEnv = FlinkSingletonFactory.getTableEnv.asInstanceOf[StreamTableEnvironment] protected lazy val hiveTableMetaDataMap = new JConcurrentHashMap[String, Table]() + private val env: StreamExecutionEnvironment = FlinkSingletonFactory.getStreamEnv + + private val configuration = new Configuration + configuration.setBoolean("table.dynamic-table-options.enabled", true) + private val settings = EnvironmentSettings.newInstance.inStreamingMode.build + private val stableEnv: TableEnvironmentImpl = StreamTableEnvironment.create(env, settings).asInstanceOf[TableEnvironmentImpl] + private val context = new LineageContext(stableEnv) /** * 用于解析给定的SQL语句 @@ -59,17 +73,23 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { override def sqlParser(sql: String): Unit = { try { FlinkUtils.sqlNodeParser(sql) match { - case select: SqlSelect => this.parseSqlNode(select) + case select: SqlSelect => + this.parseSqlNode(select) case insert: RichSqlInsert => { - this.parseSqlNode(insert.getTargetTable, Operation.INSERT_INTO) - this.parsePartitions(insert.getTargetTable.asInstanceOf[SqlIdentifier], Seq(insert.getStaticPartitions)) - this.parseSqlNode(insert.getSource, Operation.SELECT, targetTable = Some(insert.getTargetTable)) + val results = context.analyzeLineage(sql) + val relationses = new JHashSet[SQLTableColumnsRelations]() + for (x <- results) { + relationses.add(new SQLTableColumnsRelations(x.getSourceColumn, x.getTargetColumn)) + } + SQLLineageManager.addRelation(TableIdentifier(results.last.getSourceTable), TableIdentifier(results.last.getTargetTable), relationses) } case createView: SqlCreateView => { this.parseSqlNode(createView.getViewName, Operation.CREATE_VIEW) this.parseSqlNode(createView.getQuery, Operation.SELECT) } - case createTable: SqlCreateTable => parseCreateTable(createTable) + case createTable: SqlCreateTable => + parseCreateTable(createTable) + stableEnv.executeSql(sql) case _ => this.hiveSqlParser(sql) } } catch { @@ -101,7 +121,7 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { case sqlHiveInsert: RichSqlHiveInsert => this.parseHiveInsert(sqlHiveInsert) case _ => this.logger.info(s"可忽略异常:实时血缘解析SQL报错,SQL:\n$sql") } - } (this.logger, catchLog = s"可忽略异常:实时血缘解析SQL报错,SQL:\n$sql", isThrow = false, hook = false) + }(this.logger, catchLog = s"可忽略异常:实时血缘解析SQL报错,SQL:\n$sql", isThrow = false, hook = false) } /** @@ -133,7 +153,7 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { val tableIdentifier = toFireTableIdentifier(sqlIdentifier, isHive) this.addCatalog(tableIdentifier, operation) if (targetTable.isDefined) { - SQLLineageManager.addRelation(tableIdentifier, TableIdentifier(targetTable.get.toString)) + SQLLineageManager.addRelation(tableIdentifier, TableIdentifier(targetTable.get.toString),null) } } case sqlNodeList: SqlNodeList => JavaConversions.asScalaBuffer(sqlNodeList.getList).foreach(this.parseSqlNode(_)) @@ -231,7 +251,7 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { hiveCatalog.tableExists(this.toFlinkTableIdentifier(tableIdentifier)) } } else false - } (this.logger, catchLog = s"判断${tableIdentifier}是否为hive表失败", hook = false) + }(this.logger, catchLog = s"判断${tableIdentifier}是否为hive表失败", hook = false) } } @@ -414,6 +434,7 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { /** * 用于解析sql中的options + * * @param tableIdentifier * 表名 * @param options @@ -429,6 +450,7 @@ private[fire] trait FlinkSqlParserBase extends SqlParser { /** * 解析字段列表信息 + * * @param tableIdentifier * 表名 * @param columnList diff --git a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/util/FlinkSingletonFactory.scala b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/util/FlinkSingletonFactory.scala index a8f9a333e5ba8d70be900727d4a0465ff6a6f0b6..ace447061808d49bec916d6ac11684ca536c4005 100644 --- a/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/util/FlinkSingletonFactory.scala +++ b/fire-engines/fire-flink/src/main/scala/com/zto/fire/flink/util/FlinkSingletonFactory.scala @@ -72,4 +72,7 @@ object FlinkSingletonFactory extends SingletonFactory { * TableEnv实例 */ private[fire] def getTableEnv: TableEnvironment = this.tableEnv + + private [fire] def getStreamEnv: StreamExecutionEnvironment = this.streamEnv + } diff --git a/fire-enhance/apache-flink/pom.xml b/fire-enhance/apache-flink/pom.xml index 4504e8d1f67ad3670913c7d5e2dab964f0fd2345..606da22056232732fae4496858a1ed0abedcb6cf 100644 --- a/fire-enhance/apache-flink/pom.xml +++ b/fire-enhance/apache-flink/pom.xml @@ -162,6 +162,12 @@ org.apache.hive hive-exec ${hive.apache.version} + + + org.apache.calcite + calcite-core + + ${maven.scope} @@ -218,12 +224,12 @@ ${curator.verrsion} - + diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000000000000000000000000000000000..c0a19a237a0946aedaca13fc989791fa0c148b29 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,126 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import lombok.Getter; +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

+ * Modification point: + *

    + *
  1. add transform field and related code. + *
+ * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + /** + * -- GETTER -- + * Returns table of origin. + */ + @Getter + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived, + String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + + // ~ Methods ---------------------------------------------------------------- + // TODO: ------------ start:二次开发代码 --------------- // + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + // TODO: ------------ end:二次开发代码 --------------- // + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals( + other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000000000000000000000000000000000..9bab76613baddf577657d8720dec7dddd80f94e8 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.12/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,523 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.*; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.*; +import org.apache.calcite.util.BuiltInMethod; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

Modification point: + *

    + *
  1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
  2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
  3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
  4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
  5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
  6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
  7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
      + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + public static final String DELIMITER = "."; + + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() { + } + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, + RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn + - rel.getGroupCount()); + + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + Set inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet, call.toString(), true); + if (inputSet != null) { + set.addAll(inputSet); + } + } + return set; + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + // TODO: ------------ start:二次开发代码 --------------- // + + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + + List leftFieldList = rel.getLeft().getRowType().getFieldList(); + + int nLeftColumns = leftFieldList.size(); + Set set; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + RexCall rexCall = (RexCall) tableFunctionScan.getCall(); + // support only one field in table function + RexFieldAccess rexFieldAccess = (RexFieldAccess) rexCall.getOperands().get(0); + String fieldName = rexFieldAccess.getField().getName(); + + int leftFieldIndex = 0; + for (int i = 0; i < nLeftColumns; i++) { + if (leftFieldList.get(i).getName().equalsIgnoreCase(fieldName)) { + leftFieldIndex = i; + break; + } + } + /** + * Get the fields from the left table, don't go to + * getColumnOrigins(TableFunctionScan rel,RelMetadataQuery mq, int iOutputColumn), + * otherwise the return is null, and the UDTF field origin cannot be parsed + */ + set = mq.getColumnOrigins(rel.getLeft(), leftFieldIndex); + + // process transform for udtf + String transform = rexCall.toString().replace(rexFieldAccess.toString(), fieldName) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + set = createDerivedColumnOrigins(set, transform, false); + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + return set; + } + + public Set getColumnOrigins(SetOp rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode.toString(), true); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + int orderCount = rel.getOrderKeys().getKeys().size(); + + if (iOutputColumn < orderCount) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getMeasures().values().asList().get(iOutputColumn - orderCount); + + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + String originTransform = rexNode instanceof RexCall + ? ((RexCall) rexNode).getOperands().get(0).toString() + : null; + return createDerivedColumnOrigins(set, originTransform, true); + } + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List projects = new ArrayList<>(); + for (RexNode rex : rexShuttle.apply(rel.getProgram().getProjectList())) { + projects.add(rex); + } + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + + // TODO: ------------ end:二次开发代码 --------------- // + public Set getColumnOrigins(Filter rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return Collections.emptySet(); + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + // Catch-all rule when none of the others apply. + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, + RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet, String transform, boolean originTransform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = originTransform ? computeTransform(inputSet, transform) : transform; + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true, + finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, String transform) { + LOG.debug("origin transform: {}", transform); + Pattern pattern = Pattern.compile("\\$\\d+"); + Matcher matcher = pattern.matcher(transform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + LOG.info("operandSet is empty"); + return null; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn("The number [{}] of fields in the source tables are not equal to operands [{}]", inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = new HashMap<>(); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + operandSet.forEach(e -> sourceColumnMap.put(e, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + + matcher = pattern.matcher(transform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + transform = transform.replace(temp, sourceColumnMap.get(temp)); + } + + // temporary special treatment + transform = transform.replace("_UTF-16LE", ""); + LOG.debug("transform: {}", transform); + return transform; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null + ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toSet()); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, + final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000000000000000000000000000000000..f54eaf640e1710dc286d82e538217f740fc05cca --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,126 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

      + * Modification point: + *

        + *
      1. add transform field and related code. + *
      + * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived, + String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + // TODO: ------------ start:二次开发代码 --------------- // + // ~ Methods ---------------------------------------------------------------- + /** + * Returns table of origin. + */ + public RelOptTable getOriginTable() { + return originTable; + } + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + + // TODO: ------------ end:二次开发代码 --------------- // + + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals( + other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000000000000000000000000000000000..02452e5842089b0967ad0e53bc321336cfe3c74c --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,522 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.*; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.*; +import org.apache.calcite.util.BuiltInMethod; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

      Modification point: + *

        + *
      1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
      2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
      3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
      4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
      5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
      6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
      7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
          + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + public static final String DELIMITER = "."; + + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() { + } + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, + RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn + - rel.getGroupCount()); + + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + Set inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet, call.toString(), true); + if (inputSet != null) { + set.addAll(inputSet); + } + } + return set; + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + // TODO: ------------ start:二次开发代码 --------------- // + + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + + List leftFieldList = rel.getLeft().getRowType().getFieldList(); + + int nLeftColumns = leftFieldList.size(); + Set set; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + RexCall rexCall = (RexCall) tableFunctionScan.getCall(); + // support only one field in table function + RexFieldAccess rexFieldAccess = (RexFieldAccess) rexCall.getOperands().get(0); + String fieldName = rexFieldAccess.getField().getName(); + + int leftFieldIndex = 0; + for (int i = 0; i < nLeftColumns; i++) { + if (leftFieldList.get(i).getName().equalsIgnoreCase(fieldName)) { + leftFieldIndex = i; + break; + } + } + /** + * Get the fields from the left table, don't go to + * getColumnOrigins(TableFunctionScan rel,RelMetadataQuery mq, int iOutputColumn), + * otherwise the return is null, and the UDTF field origin cannot be parsed + */ + set = mq.getColumnOrigins(rel.getLeft(), leftFieldIndex); + + // process transform for udtf + String transform = rexCall.toString().replace(rexFieldAccess.toString(), fieldName) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + set = createDerivedColumnOrigins(set, transform, false); + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + return set; + } + + public Set getColumnOrigins(SetOp rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode.toString(), true); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + int orderCount = rel.getOrderKeys().getKeys().size(); + + if (iOutputColumn < orderCount) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getMeasures().values().asList().get(iOutputColumn - orderCount); + + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + String originTransform = rexNode instanceof RexCall + ? ((RexCall) rexNode).getOperands().get(0).toString() + : null; + return createDerivedColumnOrigins(set, originTransform, true); + } + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List projects = new ArrayList<>(); + for (RexNode rex : rexShuttle.apply(rel.getProgram().getProjectList())) { + projects.add(rex); + } + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + + // TODO: ------------ end:二次开发代码 --------------- // + public Set getColumnOrigins(Filter rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return Collections.emptySet(); + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + // Catch-all rule when none of the others apply. + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, + RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet, String transform, boolean originTransform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = originTransform ? computeTransform(inputSet, transform) : transform; + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true, + finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, String transform) { + LOG.debug("origin transform: {}", transform); + Pattern pattern = Pattern.compile("\\$\\d+"); + Matcher matcher = pattern.matcher(transform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + LOG.info("operandSet is empty"); + return null; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn("The number [{}] of fields in the source tables are not equal to operands [{}]", inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = new HashMap<>(); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + operandSet.forEach(e -> sourceColumnMap.put(e, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + + matcher = pattern.matcher(transform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + transform = transform.replace(temp, sourceColumnMap.get(temp)); + } + + // temporary special treatment + transform = transform.replace("_UTF-16LE", ""); + LOG.debug("transform: {}", transform); + return transform; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null + ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toSet()); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, + final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/sql/SqlSelect.java b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/sql/SqlSelect.java new file mode 100644 index 0000000000000000000000000000000000000000..0cfaa75883eee72e1894143757677cc628d7a6db --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.13/org/apache/calcite/sql/SqlSelect.java @@ -0,0 +1,355 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.sql; + + +import com.google.common.collect.ImmutableList; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorScope; +import org.apache.calcite.util.ImmutableNullableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A SqlSelect is a node of a parse tree which represents a select statement. It + * warrants its own node type just because we have a lot of methods to put somewhere. + * + * @description: Modify the value method of where to addCondition() to support row-level permission + * filtering + * @author: baisong + * @version: 1.0.0 + * @date: 2022/12/14 12:24 PM + */ +public class SqlSelect extends SqlCall { + + private static final Logger LOG = LoggerFactory.getLogger(SqlSelect.class); + + public static final int FROM_OPERAND = 2; + public static final int WHERE_OPERAND = 3; + public static final int HAVING_OPERAND = 5; + + SqlNodeList keywordList; + SqlNodeList selectList; + SqlNode from; + SqlNode where; + SqlNodeList groupBy; + SqlNode having; + SqlNodeList windowDecls; + SqlNodeList orderBy; + SqlNode offset; + SqlNode fetch; + SqlNodeList hints; + + public SqlSelect(SqlParserPos pos, + SqlNodeList keywordList, + SqlNodeList selectList, + SqlNode from, + SqlNode where, + SqlNodeList groupBy, + SqlNode having, + SqlNodeList windowDecls, + SqlNodeList orderBy, + SqlNode offset, + SqlNode fetch, + SqlNodeList hints) { + super(pos); + this.keywordList = Objects.requireNonNull(keywordList != null + ? keywordList + : new SqlNodeList(pos)); + this.selectList = selectList; + this.from = from; + this.groupBy = groupBy; + this.having = having; + this.windowDecls = Objects.requireNonNull(windowDecls != null + ? windowDecls + : new SqlNodeList(pos)); + this.orderBy = orderBy; + this.offset = offset; + this.fetch = fetch; + this.hints = hints; + + // add row level filter condition for where clause +// this.where = addCondition(from, where, false); + } + + /** + * The main process of controlling row-level permissions + */ +/* private SqlNode addCondition(SqlNode from, SqlNode where, boolean fromJoin) { + if (from instanceof SqlIdentifier) { + String tableName = from.toString(); + // the table name is used as an alias for join + String tableAlias = fromJoin ? tableName : null; + return addPermission(where, tableName, tableAlias); + } else if (from instanceof SqlJoin) { + SqlJoin sqlJoin = (SqlJoin) from; + // support recursive processing, such as join for three tables, process left sqlNode + where = addCondition(sqlJoin.getLeft(), where, true); + // process right sqlNode + return addCondition(sqlJoin.getRight(), where, true); + } else if (from instanceof SqlBasicCall) { + // Table has an alias or comes from a subquery + SqlNode[] tableNodes = ((SqlBasicCall) from).getOperands(); + *//** + * If there is a subquery in the Join, row-level filtering has been appended to the subquery. + * What is returned here is the SqlSelect type, just return the original where directly + *//* + if (!(tableNodes[0] instanceof SqlIdentifier)) { + return where; + } + String tableName = tableNodes[0].toString(); + String tableAlias = tableNodes[1].toString(); + return addPermission(where, tableName, tableAlias); + } + return where; + }*/ + + /** + * Add row-level filtering based on user-configured permission points + */ +/* private SqlNode addPermission(SqlNode where, String tableName, String tableAlias) { + SqlBasicCall permissions = null; + ConcurrentHashMap permissionsMap = RowLevelPermissionsContext.get(); + if (permissionsMap != null) { + String permissionsStatement = permissionsMap.get(tableName); + if (permissionsStatement != null && !"".equals(permissionsStatement)) { + permissions = (SqlBasicCall) CustomTableEnvironmentContext.get().getParser() + .parseExpression(permissionsStatement); + } + } + + // add an alias + if (permissions != null && tableAlias != null) { + ImmutableList namesList = ImmutableList.of(tableAlias, permissions.getOperands()[0].toString()); + permissions.getOperands()[0] = new SqlIdentifier(namesList, null, new SqlParserPos(0, 0), null); + } + + return buildWhereClause(where, permissions); + }*/ + + /** + * Rebuild the where clause + */ + private SqlNode buildWhereClause(SqlNode where, SqlBasicCall permissions) { + if (permissions != null) { + if (where == null) { + return permissions; + } + SqlBinaryOperator sqlBinaryOperator = new SqlBinaryOperator(SqlKind.AND.name(), SqlKind.AND, 0, true, null, + null, null); + SqlNode[] operands = new SqlNode[2]; + operands[0] = where; + operands[1] = permissions; + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + return new SqlBasicCall(sqlBinaryOperator, operands, sqlParserPos); + } + return where; + } + + @Override + public SqlOperator getOperator() { + return SqlSelectOperator.INSTANCE; + } + + @Override + public SqlKind getKind() { + return SqlKind.SELECT; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(keywordList, selectList, from, where, + groupBy, having, windowDecls, orderBy, offset, fetch, hints); + } + + @Override + public void setOperand(int i, SqlNode operand) { + switch (i) { + case 0: + keywordList = Objects.requireNonNull((SqlNodeList) operand); + break; + case 1: + selectList = (SqlNodeList) operand; + break; + case 2: + from = operand; + break; + case 3: + where = operand; + break; + case 4: + groupBy = (SqlNodeList) operand; + break; + case 5: + having = operand; + break; + case 6: + windowDecls = Objects.requireNonNull((SqlNodeList) operand); + break; + case 7: + orderBy = (SqlNodeList) operand; + break; + case 8: + offset = operand; + break; + case 9: + fetch = operand; + break; + default: + throw new AssertionError(i); + } + } + + public final boolean isDistinct() { + return getModifierNode(SqlSelectKeyword.DISTINCT) != null; + } + + public final SqlNode getModifierNode(SqlSelectKeyword modifier) { + for (SqlNode keyword : keywordList) { + SqlSelectKeyword keyword2 = ((SqlLiteral) keyword).symbolValue(SqlSelectKeyword.class); + if (keyword2 == modifier) { + return keyword; + } + } + return null; + } + + public final SqlNode getFrom() { + return from; + } + + public void setFrom(SqlNode from) { + this.from = from; + } + + public final SqlNodeList getGroup() { + return groupBy; + } + + public void setGroupBy(SqlNodeList groupBy) { + this.groupBy = groupBy; + } + + public final SqlNode getHaving() { + return having; + } + + public void setHaving(SqlNode having) { + this.having = having; + } + + public final SqlNodeList getSelectList() { + return selectList; + } + + public void setSelectList(SqlNodeList selectList) { + this.selectList = selectList; + } + + public final SqlNode getWhere() { + return where; + } + + public void setWhere(SqlNode whereClause) { + this.where = whereClause; + } + + @Nonnull + public final SqlNodeList getWindowList() { + return windowDecls; + } + + public final SqlNodeList getOrderList() { + return orderBy; + } + + public void setOrderBy(SqlNodeList orderBy) { + this.orderBy = orderBy; + } + + public final SqlNode getOffset() { + return offset; + } + + public void setOffset(SqlNode offset) { + this.offset = offset; + } + + public final SqlNode getFetch() { + return fetch; + } + + public void setFetch(SqlNode fetch) { + this.fetch = fetch; + } + + public void setHints(SqlNodeList hints) { + this.hints = hints; + } + + public SqlNodeList getHints() { + return this.hints; + } + + public boolean hasHints() { + // The hints may be passed as null explicitly. + return this.hints != null && this.hints.size() > 0; + } + + @Override + public void validate(SqlValidator validator, SqlValidatorScope scope) { + validator.validateQuery(this, scope, validator.getUnknownType()); + } + + /** + * Override SqlCall, to introduce a sub-query frame. + */ + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + if (!writer.inQuery()) { + // If this SELECT is the topmost item in a sub-query, introduce a new + // frame. (The topmost item in the sub-query might be a UNION or + // ORDER. In this case, we don't need a wrapper frame.) + final SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.SUB_QUERY, "(", ")"); + writer.getDialect().unparseCall(writer, this, 0, 0); + writer.endList(frame); + } else { + writer.getDialect().unparseCall(writer, this, leftPrec, rightPrec); + } + } + + public boolean hasOrderBy() { + return orderBy != null && orderBy.size() != 0; + } + + public boolean hasWhere() { + return where != null; + } + + public boolean isKeywordPresent(SqlSelectKeyword targetKeyWord) { + return getModifierNode(targetKeyWord) != null; + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000000000000000000000000000000000..29f8e0b23a882262a72f60929ff2853f28a9fac4 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

          + * Modification point: + *

            + *
          1. add transform field and related code. + *
          + * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived, + String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + // TODO: ------------ start:二次开发代码 --------------- // + // ~ Methods ---------------------------------------------------------------- + + /** + * Returns table of origin. + */ + public RelOptTable getOriginTable() { + return originTable; + } + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + // TODO: ------------ end:二次开发代码 --------------- // + + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals( + other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000000000000000000000000000000000..e0576893739240d2eea1b713f3b482baca33a11d --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.14/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,550 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.*; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.*; +import org.apache.calcite.util.BuiltInMethod; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static com.zto.fire.common.util.Constant.DELIMITER; +import static com.zto.fire.common.util.Constant.INITIAL_CAPACITY; + + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

          Modification point: + *

            + *
          1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
          2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
          3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
          4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
          5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
          6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
          7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
          8. Support TVF, modify method getColumnOrigins(TableFunctionScan rel, RelMetadataQuery mq, int iOutputColumn) + *
          + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + private final Pattern pattern = Pattern.compile("\\$[\\w.]+"); + + public static final RelMetadataProvider SOURCE = + ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() { + } + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn - rel.getGroupCount()); + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + set.addAll(mq.getColumnOrigins(rel.getInput(), iInput)); + } + return createDerivedColumnOrigins(set, call); + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + // TODO: ------------ start:二次开发代码 --------------- // + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + List fieldNameList = rel.getLeft().getRowType().getFieldNames(); + int nLeftColumns = fieldNameList.size(); + if (iOutputColumn < nLeftColumns) { + return mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + final Set set = new LinkedHashSet<>(); + for (Integer iInput : rel.getRequiredColumns().asList()) { + set.addAll(mq.getColumnOrigins(rel.getLeft(), iInput)); + } + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + String transform = computeTransform(set, tableFunctionScan.getCall()) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + return createDerivedColumnOrigins(set, transform); + } + return mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + + public Set getColumnOrigins(SetOp rel, RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + List fieldNameList = input.getRowType().getFieldNames(); + String fieldName = rel.getRowType().getFieldNames().get(iOutputColumn); + + // 1. get the column names of the partitioned keys. + Set partitionKeySet = rel.getPartitionKeys().toList() + .stream() + .map(fieldNameList::get) + .collect(Collectors.toSet()); + + // 2. get the lineage of these partitioned columns. + if (partitionKeySet.contains(fieldName)) { + return mq.getColumnOrigins(input, fieldNameList.indexOf(fieldName)); + } + + // 3. the rest of the iOutputColumn must be derived by `MEASURES`. + RexNode rexNode = rel.getMeasures().get(fieldName); + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + if (rexNode instanceof RexCall) { + return createDerivedColumnOrigins(set, ((RexCall) rexNode).getOperands().get(0)); + } else { + return createDerivedColumnOrigins(set); + } + } + // 4. something unsupported yet. + LOG.warn("Parse column lineage failed, rel:[{}], iOutputColumn:[{}]", rel, iOutputColumn); + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + + List projects = new ArrayList<>(rexShuttle.apply(rel.getProgram().getProjectList())); + + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + // TODO: ------------ end:二次开发代码 --------------- // + public Set getColumnOrigins(Filter rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, RelMetadataQuery mq, int iOutputColumn) { + Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + RelNode input = rel.getInput(0); + List inputFieldList = input.getRowType().getFieldList(); + int nInputColumns = inputFieldList.size(); + if (iOutputColumn < nInputColumns) { + return mq.getColumnOrigins(input, iOutputColumn); + } else { + RexCall rexCall = (RexCall) rel.getCall(); + List operands = rexCall.getOperands(); + RexInputRef rexInputRef = (RexInputRef) ((RexCall) operands.get(1)).getOperands().get(0); + set = mq.getColumnOrigins(input, rexInputRef.getIndex()); + + String transform = rexCall.op.getName() + + DELIMITER + + rexCall.getType().getFieldNames().get(iOutputColumn); + return createDerivedColumnOrigins(set, transform); + } + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + /** + * Catch-all rule when none of the others apply. + */ + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins(Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = + new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins(Set inputSet, Object transform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = computeTransform(inputSet, transform); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = + new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true, + finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, Object transform) { + LOG.debug("origin transform: {}, class: {}", transform, transform.getClass()); + String finalTransform = transform.toString(); + + Matcher matcher = pattern.matcher(finalTransform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + return finalTransform; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn("The number [{}] of fields in the source tables are not equal to operands [{}]", inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = buildSourceColumnMap(inputSet, transform); + + matcher = pattern.matcher(finalTransform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + finalTransform = finalTransform.replace(temp, sourceColumnMap.get(temp)); + } + // temporary special treatment + finalTransform = finalTransform.replace("_UTF-16LE", ""); + LOG.debug("final transform: {}", finalTransform); + return finalTransform; + } + + /** + * According to the order of generating inputSet, generate the corresponding index number. + * + *

          + * for example: ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $3 DESC NULLS LAST), + * The order of inputSet is $3, $0, instead of $0, $3 obtained by traversing the above string normally + */ + private Map buildSourceColumnMap(Set inputSet, Object transform) { + Set traversalSet = new LinkedHashSet<>(); + if (transform instanceof AggregateCall) { + AggregateCall call = ((AggregateCall) transform); + traversalSet.addAll(call.getArgList()); + } else if (transform instanceof RexNode) { + RexNode rexNode = (RexNode) transform; + RexVisitor visitor = + new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + traversalSet.add(inputRef.getIndex()); + return null; + } + + @Override + public Void visitPatternFieldRef(RexPatternFieldRef fieldRef) { + traversalSet.add(fieldRef.getIndex()); + return null; + } + + @Override + public Void visitFieldAccess(RexFieldAccess fieldAccess) { + traversalSet.add(fieldAccess.toString().replace("$", "")); + return null; + } + }; + rexNode.accept(visitor); + } + Map sourceColumnMap = new HashMap<>(INITIAL_CAPACITY); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + traversalSet.forEach(index -> sourceColumnMap.put("$" + index, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + return sourceColumnMap; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toCollection(LinkedHashSet::new)); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = + new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = + mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000000000000000000000000000000000..74ca219ccca5b2675aac89879c3f22b758b527b8 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,126 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

          + * Modification point: + *

            + *
          1. add transform field and related code. + *
          + * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived, + String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + // TODO: ------------ start:二次开发代码 --------------- // + // ~ Methods ---------------------------------------------------------------- + + /** + * Returns table of origin. + */ + public RelOptTable getOriginTable() { + return originTable; + } + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + + // TODO: ------------ end:二次开发代码 --------------- // + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals( + other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000000000000000000000000000000000..766e80b68e9c2c284e486b8dc8ce0e968a3c4e39 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.15/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,551 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Match; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Snapshot; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexFieldAccess; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLocalRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.util.BuiltInMethod; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

          Modification point: + *

            + *
          1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
          2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
          3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
          4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
          5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
          6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
          7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
              + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + public static final String DELIMITER = "."; + + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() { + } + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, + RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn + - rel.getGroupCount()); + + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + Set inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet, call.toString(), true); + if (inputSet != null) { + set.addAll(inputSet); + } + } + return set; + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + // TODO: ------------ start:二次开发代码 --------------- // + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + + List leftFieldList = rel.getLeft().getRowType().getFieldList(); + + int nLeftColumns = leftFieldList.size(); + Set set; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + RexCall rexCall = (RexCall) tableFunctionScan.getCall(); + // support only one field in table function + RexFieldAccess rexFieldAccess = (RexFieldAccess) rexCall.getOperands().get(0); + String fieldName = rexFieldAccess.getField().getName(); + + int leftFieldIndex = 0; + for (int i = 0; i < nLeftColumns; i++) { + if (leftFieldList.get(i).getName().equalsIgnoreCase(fieldName)) { + leftFieldIndex = i; + break; + } + } + /** + * Get the fields from the left table, don't go to + * getColumnOrigins(TableFunctionScan rel,RelMetadataQuery mq, int iOutputColumn), + * otherwise the return is null, and the UDTF field origin cannot be parsed + */ + set = mq.getColumnOrigins(rel.getLeft(), leftFieldIndex); + + // process transform for udtf + String transform = rexCall.toString().replace(rexFieldAccess.toString(), fieldName) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + set = createDerivedColumnOrigins(set, transform, false); + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + return set; + } + + public Set getColumnOrigins(SetOp rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode.toString(), true); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + int orderCount = rel.getOrderKeys().getKeys().size(); + + if (iOutputColumn < orderCount) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getMeasures().values().asList().get(iOutputColumn - orderCount); + + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + String originTransform = rexNode instanceof RexCall + ? ((RexCall) rexNode).getOperands().get(0).toString() + : null; + return createDerivedColumnOrigins(set, originTransform, true); + } + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List projects = new ArrayList<>(); + for (RexNode rex : rexShuttle.apply(rel.getProgram().getProjectList())) { + projects.add(rex); + } + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + // TODO: ------------ end:二次开发代码 --------------- // + public Set getColumnOrigins(Filter rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return Collections.emptySet(); + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + // Catch-all rule when none of the others apply. + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, + RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet, String transform, boolean originTransform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = originTransform ? computeTransform(inputSet, transform) : transform; + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true, + finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, String transform) { + LOG.debug("origin transform: {}", transform); + Pattern pattern = Pattern.compile("\\$\\d+"); + Matcher matcher = pattern.matcher(transform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + LOG.info("operandSet is empty"); + return null; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn("The number [{}] of fields in the source tables are not equal to operands [{}]", inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = new HashMap<>(); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + operandSet.forEach(e -> sourceColumnMap.put(e, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + + matcher = pattern.matcher(transform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + transform = transform.replace(temp, sourceColumnMap.get(temp)); + } + + // temporary special treatment + transform = transform.replace("_UTF-16LE", ""); + LOG.debug("transform: {}", transform); + return transform; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null + ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toSet()); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, + final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000000000000000000000000000000000..74ca219ccca5b2675aac89879c3f22b758b527b8 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,126 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

              + * Modification point: + *

                + *
              1. add transform field and related code. + *
              + * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin( + RelOptTable originTable, + int iOriginColumn, + boolean isDerived, + String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + // TODO: ------------ start:二次开发代码 --------------- // + // ~ Methods ---------------------------------------------------------------- + + /** + * Returns table of origin. + */ + public RelOptTable getOriginTable() { + return originTable; + } + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + + // TODO: ------------ end:二次开发代码 --------------- // + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals( + other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000000000000000000000000000000000..1dbabd8c399d582206cd655f2c6b3d45764ae039 --- /dev/null +++ b/fire-enhance/apache-flink/src/main/java-flink-1.16/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,552 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Match; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Snapshot; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexFieldAccess; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLocalRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.util.BuiltInMethod; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

              Modification point: + *

                + *
              1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
              2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
              3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
              4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
              5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
              6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
              7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
                  + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + public static final String DELIMITER = "."; + + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() { + } + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, + RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins(rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn + - rel.getGroupCount()); + + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + Set inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet, call.toString(), true); + if (inputSet != null) { + set.addAll(inputSet); + } + } + return set; + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, + int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + + // TODO: ------------ start:二次开发代码 --------------- // + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + + List leftFieldList = rel.getLeft().getRowType().getFieldList(); + + int nLeftColumns = leftFieldList.size(); + Set set; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + RexCall rexCall = (RexCall) tableFunctionScan.getCall(); + // support only one field in table function + RexFieldAccess rexFieldAccess = (RexFieldAccess) rexCall.getOperands().get(0); + String fieldName = rexFieldAccess.getField().getName(); + + int leftFieldIndex = 0; + for (int i = 0; i < nLeftColumns; i++) { + if (leftFieldList.get(i).getName().equalsIgnoreCase(fieldName)) { + leftFieldIndex = i; + break; + } + } + /** + * Get the fields from the left table, don't go to + * getColumnOrigins(TableFunctionScan rel,RelMetadataQuery mq, int iOutputColumn), + * otherwise the return is null, and the UDTF field origin cannot be parsed + */ + set = mq.getColumnOrigins(rel.getLeft(), leftFieldIndex); + + // process transform for udtf + String transform = rexCall.toString().replace(rexFieldAccess.toString(), fieldName) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + set = createDerivedColumnOrigins(set, transform, false); + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + return set; + } + + public Set getColumnOrigins(SetOp rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode.toString(), true); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + int orderCount = rel.getOrderKeys().getKeys().size(); + + if (iOutputColumn < orderCount) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getMeasures().values().asList().get(iOutputColumn - orderCount); + + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + String originTransform = rexNode instanceof RexCall + ? ((RexCall) rexNode).getOperands().get(0).toString() + : null; + return createDerivedColumnOrigins(set, originTransform, true); + } + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, + final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List projects = new ArrayList<>(); + for (RexNode rex : rexShuttle.apply(rel.getProgram().getProjectList())) { + projects.add(rex); + } + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + // TODO: ------------ start:二次开发代码 --------------- // + public Set getColumnOrigins(Filter rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, + int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, + RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, + RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return Collections.emptySet(); + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + // Catch-all rule when none of the others apply. + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, + RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet, String transform, boolean originTransform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = originTransform ? computeTransform(inputSet, transform) : transform; + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin( + rco.getOriginTable(), + rco.getOriginColumnOrdinal(), + true, + finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, String transform) { + LOG.debug("origin transform: {}", transform); + Pattern pattern = Pattern.compile("\\$\\d+"); + Matcher matcher = pattern.matcher(transform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + LOG.info("operandSet is empty"); + return null; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn("The number [{}] of fields in the source tables are not equal to operands [{}]", inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = new HashMap<>(); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + operandSet.forEach(e -> sourceColumnMap.put(e, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + + matcher = pattern.matcher(transform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + transform = transform.replace(temp, sourceColumnMap.get(temp)); + } + + // temporary special treatment + transform = transform.replace("_UTF-16LE", ""); + LOG.debug("transform: {}", transform); + return transform; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null + ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toSet()); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, + final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/fire-examples/flink-examples/pom.xml b/fire-examples/flink-examples/pom.xml index 0f9f1ead6818914de8487c6178788ec925213b39..8d4e1741a0b1ac5b04bf461153dd2d6eeea8ca22 100644 --- a/fire-examples/flink-examples/pom.xml +++ b/fire-examples/flink-examples/pom.xml @@ -248,11 +248,11 @@ - + com.oracle @@ -260,6 +260,21 @@ 11.2.0.3 ${maven.scope} + + com.ververica + flink-sql-connector-sqlserver-cdc + 2.2.0 + + + org.apache.doris + flink-doris-connector-1.15 + 1.2.0 + + + com.twitter + chill_2.12 + 0.10.0 + diff --git a/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkDemo.scala b/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkDemo.scala index 6598d7710c39455c7a1f3500a9b659c100aed39c..ab46495fda872b34ce14cf505fb61846034bcba9 100644 --- a/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkDemo.scala +++ b/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkDemo.scala @@ -29,22 +29,24 @@ import com.zto.fire.flink.anno.Streaming * * @contact Fire框架技术交流群(钉钉):35373471 */ -@Config( +/*@Config( """ |# 支持Flink调优参数、Fire框架参数、用户自定义参数等 |state.checkpoints.num-retained=30 |state.checkpoints.dir=hdfs:///user/flink/checkpoint |""") -@Hive("thrift://localhost:9083") // 配置连接到指定的hive +@Hive("thrift://localhost:9083") // 配置连接到指定的hive*/ @Streaming(interval = 100, unaligned = true, parallelism = 4) // 100s做一次checkpoint,开启非对齐checkpoint -@Kafka(brokers = "localhost:9092", topics = "fire", groupId = "fire") +@Kafka(brokers = "10.50.0.35:9092", topics = "mo_alarm_event", groupId = "fire1") object FlinkDemo extends FlinkStreaming { @Process def kafkaSource: Unit = { val dstream = this.fire.createKafkaDirectStream() // 使用api的方式消费kafka - sql("""create table statement ...""") - sql("""insert into statement ...""") +/* sql("""create table statement ...""") + sql("""insert into statement ...""")*/ + dstream.print() +// this.fire.start } } diff --git a/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkSQLDemo.scala b/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkSQLDemo.scala index 9b264002c9e8b6b5671122321aa015d1752eeeef..e0e62e8b11ed5b57723e65060b0cf364f1d8915a 100644 --- a/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkSQLDemo.scala +++ b/fire-examples/flink-examples/src/main/scala/com/zto/fire/examples/flink/FlinkSQLDemo.scala @@ -17,7 +17,7 @@ package com.zto.fire.examples.flink -import com.zto.fire.core.anno.lifecycle.{Step1, Step2, Step3} +import com.zto.fire.core.anno.lifecycle.{Step1, Step2, Step3, Step4, Step5, Step6} import com.zto.fire.flink.FlinkStreaming import com.zto.fire.flink.anno.Streaming @@ -81,4 +81,48 @@ object FlinkSQLDemo extends FlinkStreaming { |group by id, name, age, createTime, sex |""".stripMargin) } + + @Step4("定义源表表结构") + def sourceTable1: Unit = { + sql( + s""" + | CREATE TABLE t_student1 ( + | id BIGINT, + | name STRING, + | age INT, + | createTime TIMESTAMP(13), + | sex Boolean + |) WITH ( + | 'connector' = 'datagen', + | 'rows-per-second'='100', -- 5000/s + | 'fields.id.min'='1', -- id字段,1到1000之间 + | 'fields.id.max'='1000', + | 'fields.name.length'='5', -- name字段,长度为5 + | 'fields.age.min'='1', -- age字段,1到120岁 + | 'fields.age.max'='120' + |) + |""".stripMargin) + } + + @Step5("定义目标表结构") + def destTable1: Unit = { + sql( + s""" + |CREATE TABLE t_print_table1 WITH ('connector' = 'print') + |LIKE t_student (EXCLUDING ALL) + |""".stripMargin) + } + + @Step6("执行insert语句") + def insertStatement1: Unit = { + sql( + s""" + |insert into t_print_table1 + |select + | id, name, age, createTime, sex + |from t_student1 + |group by id, name, age, createTime, sex + |""".stripMargin) + } + } diff --git a/pom.xml b/pom.xml index 804e09ee8452500b3d2c5053cd7072eec5a53fe6..cb6f9db8d0f23b404d45644135423b86ef4e03a6 100644 --- a/pom.xml +++ b/pom.xml @@ -469,6 +469,10 @@ org.apache.spark spark-core_2.10 + + org.apache.calcite + calcite-core +