Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Remove kyuubi dependency of the spark lineage plugin #3537

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
20 changes: 7 additions & 13 deletions extensions/spark/kyuubi-spark-lineage/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
<groupId>org.apache.kyuubi</groupId>
<version>1.7.0-SNAPSHOT</version>
<relativePath>../../../pom.xml</relativePath>

</parent>
<modelVersion>4.0.0</modelVersion>

Expand All @@ -34,18 +33,6 @@
<url>https://kyuubi.apache.org/</url>

<dependencies>
<dependency>
<groupId>org.apache.kyuubi</groupId>
<artifactId>kyuubi-common_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.kyuubi</groupId>
<artifactId>kyuubi-events_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
Expand All @@ -58,6 +45,13 @@
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.kyuubi</groupId>
<artifactId>kyuubi-common_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.kyuubi</groupId>
<artifactId>kyuubi-common_${scala.binary.version}</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

package org.apache.kyuubi.plugin.lineage

import org.apache.spark.kyuubi.lineage.SparkContextHelper
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.util.QueryExecutionListener

import org.apache.kyuubi.events.EventBus
import org.apache.kyuubi.plugin.lineage.events.OperationLineageEvent
import org.apache.kyuubi.plugin.lineage.helper.SparkSQLLineageParseHelper

Expand All @@ -29,12 +29,12 @@ class SparkOperationLineageQueryExecutionListener extends QueryExecutionListener
override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = {
val lineage =
SparkSQLLineageParseHelper(qe.sparkSession).transformToLineage(qe.id, qe.optimizedPlan)
val eventTime = System.currentTimeMillis()
EventBus.post(OperationLineageEvent(qe.id, eventTime, lineage, None))
val event = OperationLineageEvent(qe.id, System.currentTimeMillis(), lineage, None)
SparkContextHelper.postEventToSparkListenerBus(event)
}

override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {
val eventTime = System.currentTimeMillis()
EventBus.post(OperationLineageEvent(qe.id, eventTime, None, Some(exception)))
val event = OperationLineageEvent(qe.id, System.currentTimeMillis(), None, Some(exception))
SparkContextHelper.postEventToSparkListenerBus(event)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,7 @@

package org.apache.kyuubi.plugin.lineage.events

import org.apache.kyuubi.Utils
import org.apache.kyuubi.events.KyuubiEvent
import org.apache.spark.scheduler.SparkListenerEvent

case class ColumnLineage(column: String, originalColumns: Set[String])

Expand Down Expand Up @@ -66,8 +65,4 @@ case class OperationLineageEvent(
executionId: Long,
eventTime: Long,
lineage: Option[Lineage],
exception: Option[Throwable]) extends KyuubiEvent {

override def partitions: Seq[(String, String)] =
("day", Utils.getDateFromTimestamp(eventTime)) :: Nil
}
exception: Option[Throwable]) extends SparkListenerEvent
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.kyuubi.plugin.lineage.helper

/**
* Encapsulate a component (Kyuubi/Spark/Hive/Flink etc.) version
* for the convenience of version checks.
*/
case class SemanticVersion(majorVersion: Int, minorVersion: Int) {
Copy link
Member

@yaooqinn yaooqinn Sep 29, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, before introducing this, the version comparison seems to be very simple. Now, plenty of copies of this class.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, before introducing this, the version comparison seems to be very simple. Now, plenty of copies of this class.

In this plugin we only use isSparkVersionAtMost method, do I need to simply reimplement a method instead of copying SemanticVersion?


def isVersionAtMost(targetVersionString: String): Boolean = {
this.compareVersion(
targetVersionString,
(targetMajor: Int, targetMinor: Int, runtimeMajor: Int, runtimeMinor: Int) =>
(runtimeMajor < targetMajor) || {
runtimeMajor == targetMajor && runtimeMinor <= targetMinor
})
}

def isVersionAtLeast(targetVersionString: String): Boolean = {
this.compareVersion(
targetVersionString,
(targetMajor: Int, targetMinor: Int, runtimeMajor: Int, runtimeMinor: Int) =>
(runtimeMajor > targetMajor) || {
runtimeMajor == targetMajor && runtimeMinor >= targetMinor
})
}

def isVersionEqualTo(targetVersionString: String): Boolean = {
this.compareVersion(
targetVersionString,
(targetMajor: Int, targetMinor: Int, runtimeMajor: Int, runtimeMinor: Int) =>
runtimeMajor == targetMajor && runtimeMinor == targetMinor)
}

def compareVersion(
targetVersionString: String,
callback: (Int, Int, Int, Int) => Boolean): Boolean = {
val targetVersion = SemanticVersion(targetVersionString)
val targetMajor = targetVersion.majorVersion
val targetMinor = targetVersion.minorVersion
callback(targetMajor, targetMinor, this.majorVersion, this.minorVersion)
}

override def toString: String = s"$majorVersion.$minorVersion"
}

object SemanticVersion {

def apply(versionString: String): SemanticVersion = {
"""^(\d+)\.(\d+)(\..*)?$""".r.findFirstMatchIn(versionString) match {
case Some(m) =>
SemanticVersion(m.group(1).toInt, m.group(2).toInt)
case None =>
throw new IllegalArgumentException(s"Tried to parse '$versionString' as a project" +
s" version string, but it could not find the major and minor version numbers.")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@ package org.apache.kyuubi.plugin.lineage.helper

import org.apache.spark.SPARK_VERSION

import org.apache.kyuubi.engine.SemanticVersion

object SparkListenerHelper {

lazy val sparkMajorMinorVersion: (Int, Int) = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.kyuubi.plugin.lineage.helper
import scala.collection.immutable.ListMap
import scala.util.{Failure, Success, Try}

import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, HiveTableRelation}
Expand All @@ -32,7 +33,6 @@ import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, TableC
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation

import org.apache.kyuubi.Logging
import org.apache.kyuubi.plugin.lineage.events.Lineage
import org.apache.kyuubi.plugin.lineage.helper.SparkListenerHelper.isSparkVersionAtMost

Expand Down Expand Up @@ -359,7 +359,7 @@ case class SparkSQLLineageParseHelper(sparkSession: SparkSession) extends Lineag
plan: LogicalPlan): Option[Lineage] = {
Try(parse(plan)).recover {
case e: Exception =>
warn(s"Extract Statement[$executionId] columns lineage failed.", e)
logWarning(s"Extract Statement[$executionId] columns lineage failed.", e)
throw e
}.toOption
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.lineage

import org.apache.spark.SparkContext
import org.apache.spark.scheduler.SparkListenerEvent
import org.apache.spark.sql.SparkSession

object SparkContextHelper {

def globalSparkContext: SparkContext = SparkSession.active.sparkContext

def postEventToSparkListenerBus(
event: SparkListenerEvent,
sc: SparkContext = globalSparkContext) {
sc.listenerBus.post(event)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@

package org.apache.kyuubi.plugin.lineage.events

import java.util.concurrent.CountDownLatch
import java.util.concurrent.{CountDownLatch, TimeUnit}

import org.apache.spark.SparkConf
import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent}
import org.apache.spark.sql.SparkListenerExtensionTest

import org.apache.kyuubi.KyuubiFunSuite
import org.apache.kyuubi.events.EventBus
import org.apache.kyuubi.plugin.lineage.helper.SparkListenerHelper.isSparkVersionAtMost

class OperationLineageEventSuite extends KyuubiFunSuite with SparkListenerExtensionTest {
Expand All @@ -45,13 +45,19 @@ class OperationLineageEventSuite extends KyuubiFunSuite with SparkListenerExtens
test("operation lineage event capture: for execute sql") {
val countDownLatch = new CountDownLatch(1)
var actual: Lineage = null
EventBus.register[OperationLineageEvent] { event =>
event.lineage.foreach {
case lineage if lineage.inputTables.nonEmpty =>
actual = lineage
countDownLatch.countDown()
spark.sparkContext.addSparkListener(new SparkListener {
override def onOtherEvent(event: SparkListenerEvent): Unit = {
event match {
case lineageEvent: OperationLineageEvent =>
lineageEvent.lineage.foreach {
case lineage if lineage.inputTables.nonEmpty =>
actual = lineage
countDownLatch.countDown()
}
case _ =>
}
}
}
})

withTable("test_table0") { _ =>
spark.sql("create table test_table0(a string, b string)")
Expand All @@ -62,7 +68,7 @@ class OperationLineageEventSuite extends KyuubiFunSuite with SparkListenerExtens
List(
("col0", Set("default.test_table0.a")),
("col1", Set("default.test_table0.b"))))
countDownLatch.await()
countDownLatch.await(20, TimeUnit.SECONDS)
assert(actual == expected)
}
}
Expand Down