ulysses-you commented on code in PR #3185:
URL: https://github.com/apache/incubator-kyuubi/pull/3185#discussion_r944144310


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkSQLLineageParseHelper.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.spark.kyuubi
+
+import scala.collection.immutable.ListMap
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, HiveTableRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeSet, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, 
TableCatalog}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2ScanRelation}
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.isSparkVersionAtMost
+import org.apache.kyuubi.engine.spark.events.Lineage
+
+trait LineageParser {
+  def sparkSession: SparkSession
+
+  type AttributeMap[A] = ListMap[Attribute, A]
+
+  def parse(plan: LogicalPlan): Lineage = {
+    val columnsLineage =
+      extractColumnsLineage(plan, ListMap[Attribute, 
AttributeSet]()).toList.collect {
+        case (k, attrs) =>
+          k.name -> attrs.map(_.name).toSet
+      }
+    val (inputTables, outputTables) = columnsLineage.foldLeft((List[String](), 
List[String]())) {
+      case ((inputs, outputs), (out, in)) =>
+        val x = (inputs ++ 
in.map(_.split('.').init.mkString("."))).filter(_.nonEmpty)
+        val y = outputs ++ 
List(out.split('.').init.mkString(".")).filter(_.nonEmpty)
+        (x, y)
+    }
+    Lineage(inputTables.distinct, outputTables.distinct, columnsLineage)
+  }
+
+  private def mergeColumnsLineage(
+      left: AttributeMap[AttributeSet],
+      right: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    left ++ right.map {
+      case (k, attrs) =>
+        k -> (attrs ++ left.getOrElse(k, AttributeSet.empty))
+    }
+  }
+
+  private def joinColumnsLineage(
+      parent: AttributeMap[AttributeSet],
+      child: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    if (parent.isEmpty) child
+    else {
+      val childMap = child.map { case (k, attrs) => (k.exprId, attrs) }
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.flatMap(attr =>
+          childMap.getOrElse(attr.exprId, AttributeSet.empty)))
+      }
+    }
+  }
+
+  private def getSelectColumnLineage(
+      named: Seq[NamedExpression]): AttributeMap[AttributeSet] = {
+    val exps = named.map {
+      case a: Alias => a.toAttribute -> a.references
+      case a: Attribute => a -> a.references
+    }
+    ListMap(exps: _*)
+  }
+
+  private def joinRelationColumnLineage(
+      parent: AttributeMap[AttributeSet],
+      relationAttrs: Seq[Attribute],
+      tableName: String = ""): AttributeMap[AttributeSet] = {
+    val relationAttrSet = AttributeSet(relationAttrs)
+    if (parent.nonEmpty) {
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.collect {
+          case attr if relationAttrSet.contains(attr) =>
+            attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))
+        })
+      }
+    } else {
+      ListMap(relationAttrs.map { attr =>
+        (
+          attr,
+          AttributeSet(attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))))
+      }: _*)
+    }
+  }
+
+  private def extractColumnsLineage(
+      plan: LogicalPlan,
+      parentColumnsLineage: AttributeMap[AttributeSet]): 
AttributeMap[AttributeSet] = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getCurrentPlanField[T](curPlan: LogicalPlan, field: String): T = {
+      getFieldVal[T](curPlan, field)
+    }
+
+    def getPlanMethod[T](name: String): T = {
+      getMethod[T](plan, name)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan match {
+      // For command
+      case p if p.nodeName == "AlterViewAsCommand" =>
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)

Review Comment:
   why we need this ?



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkSQLLineageParseHelper.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.spark.kyuubi
+
+import scala.collection.immutable.ListMap
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, HiveTableRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeSet, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, 
TableCatalog}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2ScanRelation}
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.isSparkVersionAtMost
+import org.apache.kyuubi.engine.spark.events.Lineage
+
+trait LineageParser {
+  def sparkSession: SparkSession
+
+  type AttributeMap[A] = ListMap[Attribute, A]
+
+  def parse(plan: LogicalPlan): Lineage = {
+    val columnsLineage =
+      extractColumnsLineage(plan, ListMap[Attribute, 
AttributeSet]()).toList.collect {
+        case (k, attrs) =>
+          k.name -> attrs.map(_.name).toSet
+      }
+    val (inputTables, outputTables) = columnsLineage.foldLeft((List[String](), 
List[String]())) {
+      case ((inputs, outputs), (out, in)) =>
+        val x = (inputs ++ 
in.map(_.split('.').init.mkString("."))).filter(_.nonEmpty)
+        val y = outputs ++ 
List(out.split('.').init.mkString(".")).filter(_.nonEmpty)
+        (x, y)
+    }
+    Lineage(inputTables.distinct, outputTables.distinct, columnsLineage)
+  }
+
+  private def mergeColumnsLineage(
+      left: AttributeMap[AttributeSet],
+      right: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    left ++ right.map {
+      case (k, attrs) =>
+        k -> (attrs ++ left.getOrElse(k, AttributeSet.empty))
+    }
+  }
+
+  private def joinColumnsLineage(
+      parent: AttributeMap[AttributeSet],
+      child: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    if (parent.isEmpty) child
+    else {
+      val childMap = child.map { case (k, attrs) => (k.exprId, attrs) }
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.flatMap(attr =>
+          childMap.getOrElse(attr.exprId, AttributeSet.empty)))
+      }
+    }
+  }
+
+  private def getSelectColumnLineage(
+      named: Seq[NamedExpression]): AttributeMap[AttributeSet] = {
+    val exps = named.map {
+      case a: Alias => a.toAttribute -> a.references
+      case a: Attribute => a -> a.references
+    }
+    ListMap(exps: _*)
+  }
+
+  private def joinRelationColumnLineage(
+      parent: AttributeMap[AttributeSet],
+      relationAttrs: Seq[Attribute],
+      tableName: String = ""): AttributeMap[AttributeSet] = {
+    val relationAttrSet = AttributeSet(relationAttrs)
+    if (parent.nonEmpty) {
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.collect {
+          case attr if relationAttrSet.contains(attr) =>
+            attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))
+        })
+      }
+    } else {
+      ListMap(relationAttrs.map { attr =>
+        (
+          attr,
+          AttributeSet(attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))))
+      }: _*)
+    }
+  }
+
+  private def extractColumnsLineage(
+      plan: LogicalPlan,
+      parentColumnsLineage: AttributeMap[AttributeSet]): 
AttributeMap[AttributeSet] = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getCurrentPlanField[T](curPlan: LogicalPlan, field: String): T = {
+      getFieldVal[T](curPlan, field)
+    }
+
+    def getPlanMethod[T](name: String): T = {
+      getMethod[T](plan, name)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan match {
+      // For command
+      case p if p.nodeName == "AlterViewAsCommand" =>
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateViewCommand" =>
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        val outputCols =
+          getPlanField[Seq[(String, 
Option[String])]]("userSpecifiedColumns").map(_._1)
+        val query =
+          if (isSparkVersionAtMost("3.1")) {
+            getPlanField[LogicalPlan]("child")
+          } else {
+            getPlanField[LogicalPlan]("plan")
+          }
+        val analyzed = sparkSession.sessionState.analyzer.execute(query)
+        extractColumnsLineage(analyzed, parentColumnsLineage).zipWithIndex.map 
{
+          case ((k, v), i) if outputCols.nonEmpty => 
k.withName(s"$view.${outputCols(i)}") -> v
+          case ((k, v), _) => k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateDataSourceTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("table").qualifiedName
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateHiveTableAsSelectCommand" ||
+            p.nodeName == "OptimizedCreateHiveTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("tableDesc").qualifiedName
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateTableAsSelect" ||
+            p.nodeName == "ReplaceTableAsSelect" =>
+        val (table, namespace, catalog) =
+          if (isSparkVersionAtMost("3.2")) {
+            (
+              getPlanField[Identifier]("tableName").name,
+              getPlanField[Identifier]("tableName").namespace.mkString("."),
+              getPlanField[TableCatalog]("catalog").name())
+          } else {
+            (
+              getPlanMethod[Identifier]("tableName").name(),
+              getPlanMethod[Identifier]("tableName").namespace().mkString("."),
+              getCurrentPlanField[CatalogPlugin](
+                getPlanMethod[LogicalPlan]("left"),
+                "catalog").name())
+          }
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$catalog.$namespace.$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoDataSourceCommand" =>
+        val logicalRelation = getPlanField[LogicalRelation]("logicalRelation")
+        val table = 
logicalRelation.catalogTable.map(_.qualifiedName).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoHadoopFsRelationCommand" =>
+        val table =
+          
getPlanField[Option[CatalogTable]]("catalogTable").map(_.qualifiedName).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "InsertIntoDataSourceDirCommand" ||
+            p.nodeName == "InsertIntoHiveDirCommand" =>
+        val dir =
+          
getPlanField[CatalogStorageFormat]("storage").locationUri.map(_.toString).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"`$dir`.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoHiveTable" =>
+        val table = getPlanField[CatalogTable]("table").qualifiedName
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "SaveIntoDataSourceCommand" =>
+        extractColumnsLineage(getQuery, parentColumnsLineage)
+
+      // For query
+      case p: Project =>
+        val nextColumnsLineage =
+          joinColumnsLineage(parentColumnsLineage, 
getSelectColumnLineage(p.projectList))
+        p.children.map(extractColumnsLineage(_, 
nextColumnsLineage)).reduce(mergeColumnsLineage)
+
+      case p: Aggregate =>
+        val nextColumnsLineage =
+          joinColumnsLineage(parentColumnsLineage, 
getSelectColumnLineage(p.aggregateExpressions))
+        p.children.map(extractColumnsLineage(_, 
nextColumnsLineage)).reduce(mergeColumnsLineage)
+
+      case p: Union =>
+        // merge all children in to one derivedColumns
+        val childrenUnion =
+          p.children.map(extractColumnsLineage(_, ListMap[Attribute, 
AttributeSet]())).map(
+            _.values).reduce {
+            (left, right) =>
+              left.zip(right).map(attr => attr._1 ++ attr._2)
+          }
+        val childrenColumnsLineage = ListMap(p.output.zip(childrenUnion): _*)
+        joinColumnsLineage(parentColumnsLineage, childrenColumnsLineage)
+
+      case p: LogicalRelation if p.catalogTable.nonEmpty =>
+        val tableName = p.catalogTable.get.qualifiedName
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: HiveTableRelation =>
+        val tableName = p.tableMeta.qualifiedName
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: DataSourceV2Relation =>

Review Comment:
   For an optimzied plan, the v2 scan should always be 
`DataSourceV2ScanRelation`. We can remove this pattern



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkSQLLineageParseHelper.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.spark.kyuubi
+
+import scala.collection.immutable.ListMap
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, HiveTableRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeSet, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, 
TableCatalog}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2ScanRelation}
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.isSparkVersionAtMost
+import org.apache.kyuubi.engine.spark.events.Lineage
+
+trait LineageParser {
+  def sparkSession: SparkSession
+
+  type AttributeMap[A] = ListMap[Attribute, A]
+
+  def parse(plan: LogicalPlan): Lineage = {
+    val columnsLineage =
+      extractColumnsLineage(plan, ListMap[Attribute, 
AttributeSet]()).toList.collect {
+        case (k, attrs) =>
+          k.name -> attrs.map(_.name).toSet
+      }
+    val (inputTables, outputTables) = columnsLineage.foldLeft((List[String](), 
List[String]())) {
+      case ((inputs, outputs), (out, in)) =>
+        val x = (inputs ++ 
in.map(_.split('.').init.mkString("."))).filter(_.nonEmpty)
+        val y = outputs ++ 
List(out.split('.').init.mkString(".")).filter(_.nonEmpty)
+        (x, y)
+    }
+    Lineage(inputTables.distinct, outputTables.distinct, columnsLineage)
+  }
+
+  private def mergeColumnsLineage(
+      left: AttributeMap[AttributeSet],
+      right: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    left ++ right.map {
+      case (k, attrs) =>
+        k -> (attrs ++ left.getOrElse(k, AttributeSet.empty))
+    }
+  }
+
+  private def joinColumnsLineage(
+      parent: AttributeMap[AttributeSet],
+      child: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    if (parent.isEmpty) child
+    else {
+      val childMap = child.map { case (k, attrs) => (k.exprId, attrs) }
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.flatMap(attr =>
+          childMap.getOrElse(attr.exprId, AttributeSet.empty)))
+      }
+    }
+  }
+
+  private def getSelectColumnLineage(
+      named: Seq[NamedExpression]): AttributeMap[AttributeSet] = {
+    val exps = named.map {
+      case a: Alias => a.toAttribute -> a.references
+      case a: Attribute => a -> a.references
+    }
+    ListMap(exps: _*)
+  }
+
+  private def joinRelationColumnLineage(
+      parent: AttributeMap[AttributeSet],
+      relationAttrs: Seq[Attribute],
+      tableName: String = ""): AttributeMap[AttributeSet] = {
+    val relationAttrSet = AttributeSet(relationAttrs)
+    if (parent.nonEmpty) {
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.collect {
+          case attr if relationAttrSet.contains(attr) =>
+            attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))
+        })
+      }
+    } else {
+      ListMap(relationAttrs.map { attr =>
+        (
+          attr,
+          AttributeSet(attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))))
+      }: _*)
+    }
+  }
+
+  private def extractColumnsLineage(
+      plan: LogicalPlan,
+      parentColumnsLineage: AttributeMap[AttributeSet]): 
AttributeMap[AttributeSet] = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getCurrentPlanField[T](curPlan: LogicalPlan, field: String): T = {
+      getFieldVal[T](curPlan, field)
+    }
+
+    def getPlanMethod[T](name: String): T = {
+      getMethod[T](plan, name)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan match {
+      // For command
+      case p if p.nodeName == "AlterViewAsCommand" =>
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateViewCommand" =>
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        val outputCols =
+          getPlanField[Seq[(String, 
Option[String])]]("userSpecifiedColumns").map(_._1)
+        val query =
+          if (isSparkVersionAtMost("3.1")) {
+            getPlanField[LogicalPlan]("child")
+          } else {
+            getPlanField[LogicalPlan]("plan")
+          }
+        val analyzed = sparkSession.sessionState.analyzer.execute(query)
+        extractColumnsLineage(analyzed, parentColumnsLineage).zipWithIndex.map 
{
+          case ((k, v), i) if outputCols.nonEmpty => 
k.withName(s"$view.${outputCols(i)}") -> v
+          case ((k, v), _) => k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateDataSourceTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("table").qualifiedName
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateHiveTableAsSelectCommand" ||
+            p.nodeName == "OptimizedCreateHiveTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("tableDesc").qualifiedName
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateTableAsSelect" ||
+            p.nodeName == "ReplaceTableAsSelect" =>
+        val (table, namespace, catalog) =
+          if (isSparkVersionAtMost("3.2")) {
+            (
+              getPlanField[Identifier]("tableName").name,
+              getPlanField[Identifier]("tableName").namespace.mkString("."),
+              getPlanField[TableCatalog]("catalog").name())
+          } else {
+            (
+              getPlanMethod[Identifier]("tableName").name(),
+              getPlanMethod[Identifier]("tableName").namespace().mkString("."),
+              getCurrentPlanField[CatalogPlugin](
+                getPlanMethod[LogicalPlan]("left"),
+                "catalog").name())
+          }
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$catalog.$namespace.$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoDataSourceCommand" =>
+        val logicalRelation = getPlanField[LogicalRelation]("logicalRelation")
+        val table = 
logicalRelation.catalogTable.map(_.qualifiedName).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoHadoopFsRelationCommand" =>
+        val table =
+          
getPlanField[Option[CatalogTable]]("catalogTable").map(_.qualifiedName).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "InsertIntoDataSourceDirCommand" ||
+            p.nodeName == "InsertIntoHiveDirCommand" =>
+        val dir =
+          
getPlanField[CatalogStorageFormat]("storage").locationUri.map(_.toString).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"`$dir`.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoHiveTable" =>
+        val table = getPlanField[CatalogTable]("table").qualifiedName
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "SaveIntoDataSourceCommand" =>
+        extractColumnsLineage(getQuery, parentColumnsLineage)
+
+      // For query
+      case p: Project =>
+        val nextColumnsLineage =
+          joinColumnsLineage(parentColumnsLineage, 
getSelectColumnLineage(p.projectList))
+        p.children.map(extractColumnsLineage(_, 
nextColumnsLineage)).reduce(mergeColumnsLineage)
+
+      case p: Aggregate =>
+        val nextColumnsLineage =
+          joinColumnsLineage(parentColumnsLineage, 
getSelectColumnLineage(p.aggregateExpressions))
+        p.children.map(extractColumnsLineage(_, 
nextColumnsLineage)).reduce(mergeColumnsLineage)
+
+      case p: Union =>
+        // merge all children in to one derivedColumns
+        val childrenUnion =
+          p.children.map(extractColumnsLineage(_, ListMap[Attribute, 
AttributeSet]())).map(
+            _.values).reduce {
+            (left, right) =>
+              left.zip(right).map(attr => attr._1 ++ attr._2)
+          }
+        val childrenColumnsLineage = ListMap(p.output.zip(childrenUnion): _*)
+        joinColumnsLineage(parentColumnsLineage, childrenColumnsLineage)
+
+      case p: LogicalRelation if p.catalogTable.nonEmpty =>
+        val tableName = p.catalogTable.get.qualifiedName
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: HiveTableRelation =>
+        val tableName = p.tableMeta.qualifiedName
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: DataSourceV2Relation =>
+        val tableName = p.name
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: DataSourceV2ScanRelation =>
+        val tableName = p.name
+        joinRelationColumnLineage(parentColumnsLineage, p.output, tableName)
+
+      case p: LocalRelation =>
+        joinRelationColumnLineage(parentColumnsLineage, p.output)

Review Comment:
   tableName = `__local__` ? so we can remove the default empty tableName



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/kyuubi/SparkSQLLineageParseHelper.scala:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.spark.kyuubi
+
+import scala.collection.immutable.ListMap
+import scala.util.{Failure, Success, Try}
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, HiveTableRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, 
AttributeSet, NamedExpression}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, 
TableCatalog}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
DataSourceV2ScanRelation}
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.isSparkVersionAtMost
+import org.apache.kyuubi.engine.spark.events.Lineage
+
+trait LineageParser {
+  def sparkSession: SparkSession
+
+  type AttributeMap[A] = ListMap[Attribute, A]
+
+  def parse(plan: LogicalPlan): Lineage = {
+    val columnsLineage =
+      extractColumnsLineage(plan, ListMap[Attribute, 
AttributeSet]()).toList.collect {
+        case (k, attrs) =>
+          k.name -> attrs.map(_.name).toSet
+      }
+    val (inputTables, outputTables) = columnsLineage.foldLeft((List[String](), 
List[String]())) {
+      case ((inputs, outputs), (out, in)) =>
+        val x = (inputs ++ 
in.map(_.split('.').init.mkString("."))).filter(_.nonEmpty)
+        val y = outputs ++ 
List(out.split('.').init.mkString(".")).filter(_.nonEmpty)
+        (x, y)
+    }
+    Lineage(inputTables.distinct, outputTables.distinct, columnsLineage)
+  }
+
+  private def mergeColumnsLineage(
+      left: AttributeMap[AttributeSet],
+      right: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    left ++ right.map {
+      case (k, attrs) =>
+        k -> (attrs ++ left.getOrElse(k, AttributeSet.empty))
+    }
+  }
+
+  private def joinColumnsLineage(
+      parent: AttributeMap[AttributeSet],
+      child: AttributeMap[AttributeSet]): AttributeMap[AttributeSet] = {
+    if (parent.isEmpty) child
+    else {
+      val childMap = child.map { case (k, attrs) => (k.exprId, attrs) }
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.flatMap(attr =>
+          childMap.getOrElse(attr.exprId, AttributeSet.empty)))
+      }
+    }
+  }
+
+  private def getSelectColumnLineage(
+      named: Seq[NamedExpression]): AttributeMap[AttributeSet] = {
+    val exps = named.map {
+      case a: Alias => a.toAttribute -> a.references
+      case a: Attribute => a -> a.references
+    }
+    ListMap(exps: _*)
+  }
+
+  private def joinRelationColumnLineage(
+      parent: AttributeMap[AttributeSet],
+      relationAttrs: Seq[Attribute],
+      tableName: String = ""): AttributeMap[AttributeSet] = {
+    val relationAttrSet = AttributeSet(relationAttrs)
+    if (parent.nonEmpty) {
+      parent.map { case (k, attrs) =>
+        k -> AttributeSet(attrs.collect {
+          case attr if relationAttrSet.contains(attr) =>
+            attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))
+        })
+      }
+    } else {
+      ListMap(relationAttrs.map { attr =>
+        (
+          attr,
+          AttributeSet(attr.withName(Seq(tableName, 
attr.name).filter(_.nonEmpty).mkString("."))))
+      }: _*)
+    }
+  }
+
+  private def extractColumnsLineage(
+      plan: LogicalPlan,
+      parentColumnsLineage: AttributeMap[AttributeSet]): 
AttributeMap[AttributeSet] = {
+
+    def getPlanField[T](field: String): T = {
+      getFieldVal[T](plan, field)
+    }
+
+    def getCurrentPlanField[T](curPlan: LogicalPlan, field: String): T = {
+      getFieldVal[T](curPlan, field)
+    }
+
+    def getPlanMethod[T](name: String): T = {
+      getMethod[T](plan, name)
+    }
+
+    def getQuery: LogicalPlan = {
+      getPlanField[LogicalPlan]("query")
+    }
+
+    plan match {
+      // For command
+      case p if p.nodeName == "AlterViewAsCommand" =>
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateViewCommand" =>
+        val view = getPlanField[TableIdentifier]("name").unquotedString
+        val outputCols =
+          getPlanField[Seq[(String, 
Option[String])]]("userSpecifiedColumns").map(_._1)
+        val query =
+          if (isSparkVersionAtMost("3.1")) {
+            getPlanField[LogicalPlan]("child")
+          } else {
+            getPlanField[LogicalPlan]("plan")
+          }
+        val analyzed = sparkSession.sessionState.analyzer.execute(query)
+        extractColumnsLineage(analyzed, parentColumnsLineage).zipWithIndex.map 
{
+          case ((k, v), i) if outputCols.nonEmpty => 
k.withName(s"$view.${outputCols(i)}") -> v
+          case ((k, v), _) => k.withName(s"$view.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "CreateDataSourceTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("table").qualifiedName
+        val analyzed = sparkSession.sessionState.analyzer.execute(getQuery)
+        extractColumnsLineage(analyzed, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateHiveTableAsSelectCommand" ||
+            p.nodeName == "OptimizedCreateHiveTableAsSelectCommand" =>
+        val table = getPlanField[CatalogTable]("tableDesc").qualifiedName
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v
+        }
+
+      case p
+          if p.nodeName == "CreateTableAsSelect" ||
+            p.nodeName == "ReplaceTableAsSelect" =>
+        val (table, namespace, catalog) =
+          if (isSparkVersionAtMost("3.2")) {
+            (
+              getPlanField[Identifier]("tableName").name,
+              getPlanField[Identifier]("tableName").namespace.mkString("."),
+              getPlanField[TableCatalog]("catalog").name())
+          } else {
+            (
+              getPlanMethod[Identifier]("tableName").name(),
+              getPlanMethod[Identifier]("tableName").namespace().mkString("."),
+              getCurrentPlanField[CatalogPlugin](
+                getPlanMethod[LogicalPlan]("left"),
+                "catalog").name())
+          }
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$catalog.$namespace.$table.${k.name}") -> v
+        }
+
+      case p if p.nodeName == "InsertIntoDataSourceCommand" =>
+        val logicalRelation = getPlanField[LogicalRelation]("logicalRelation")
+        val table = 
logicalRelation.catalogTable.map(_.qualifiedName).getOrElse("")
+        extractColumnsLineage(getQuery, parentColumnsLineage).map { case (k, 
v) =>
+          k.withName(s"$table.${k.name}") -> v

Review Comment:
   the `.` of `$table.` should be merged to `$table` in case the table is empty



##########
externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/spark/kyuubi/SparkSQLLineageParserHelperSuite.scala:
##########
@@ -0,0 +1,690 @@
+/*
+ * 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.spark.kyuubi
+
+import scala.collection.immutable.List
+import scala.reflect.io.File
+
+import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, 
CatalogTable, CatalogTableType}
+import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation, 
SchemaRelationProvider}
+import org.apache.spark.sql.types.{IntegerType, StringType, StructType}
+
+import org.apache.kyuubi.KyuubiFunSuite
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil.isSparkVersionAtMost
+import org.apache.kyuubi.engine.spark.WithSparkSQLEngine
+import org.apache.kyuubi.engine.spark.events.Lineage
+
+class SparkSQLLineageParserHelperSuite extends WithSparkSQLEngine with 
KyuubiFunSuite {
+  val catalogName =
+    if (isSparkVersionAtMost("3.1")) 
"org.apache.spark.sql.connector.InMemoryTableCatalog"
+    else "org.apache.spark.sql.connector.catalog.InMemoryTableCatalog"
+
+  override def withKyuubiConf: Map[String, String] = Map(
+    "spark.sql.catalog.v2_catalog" -> catalogName)
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    spark.sql(s"create database if not exists test_db0")
+    spark.sql(s"create table if not exists test_db0.test_table0" +
+      s" (key int, value string) using parquet")
+    spark.sql(s"create table if not exists test_db0.test_table_part0" +
+      s" (key int, value string, pid string) using parquet" +
+      s"  partitioned by(pid)")
+  }
+
+  override def afterAll(): Unit = {
+    Seq("test_db0.test_table0", "test_db0.test_table_part0").foreach { t =>
+      spark.sql(s"drop table if exists $t")
+    }
+    spark.sql(s"drop database if exists test_db0")
+    spark.stop()
+    super.afterAll()
+  }
+
+  protected def withTable(t: String)(f: String => Unit): Unit = {
+    try {
+      f(t)
+    } finally {
+      spark.sql(s"DROP TABLE IF EXISTS $t")
+    }
+  }
+
+  test("columns lineage extract - AlterViewAsCommand") {
+    spark.sql(s"create view alterviewascommand as select key from 
test_db0.test_table0")

Review Comment:
   ```scala
   withTempView("") {
     ...
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to