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

Provide Spark 3.4 Support for Spline w/ Backwards Compatibility #793

Open
wants to merge 9 commits into
base: develop
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package za.co.absa.spline.harvester.plugin.embedded
import org.apache.spark.Partition
import org.apache.spark.rdd.{HadoopRDD, RDD}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.execution.datasources.FileScanRDD
import org.apache.spark.sql.execution.datasources.{FileScanRDD, PartitionedFile}
import za.co.absa.spline.commons.reflect.ReflectionUtils
import za.co.absa.spline.harvester.builder._
import za.co.absa.spline.harvester.plugin.Plugin.{Precedence, ReadNodeInfo}
Expand All @@ -39,14 +39,22 @@ class RDDPlugin(

override def rddReadNodeProcessor: PartialFunction[RDD[_], ReadNodeInfo] = {
case fsr: FileScanRDD =>
val uris = fsr.filePartitions.flatMap(_.files.map(_.filePath))
val files = fsr.filePartitions.flatMap(_.files)
val uris = files.map(extractPath(_))
ReadNodeInfo(SourceIdentifier(None, uris: _*), Map.empty)
case hr: HadoopRDD[_, _] =>
val partitions = ReflectionUtils.extractValue[Array[Partition]](hr, "partitions_")
val uris = partitions.map(hadoopPartitionToUriString)
ReadNodeInfo(SourceIdentifier(None, uris: _*), Map.empty)
}

private def extractPath(file: PartitionedFile): String = {
rycowhi marked this conversation as resolved.
Show resolved Hide resolved
val path = ReflectionUtils.extractValue[AnyRef](file, "filePath")
// for Spark 3.3 and lower path is a String
// for Spark 3.4 path is org.apache.spark.paths.SparkPath
path.toString
}

private def hadoopPartitionToUriString(hadoopPartition: Partition): String = {
val inputSplit = ReflectionUtils.extractValue[AnyRef](hadoopPartition, "inputSplit")
val fileSplitT = ReflectionUtils.extractValue[AnyRef](inputSplit, "t")
Expand All @@ -56,5 +64,4 @@ class RDDPlugin(

uri.toString
}

}
17 changes: 16 additions & 1 deletion integration-tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,7 @@
<dependency>
<groupId>org.elasticsearch</groupId>
<artifactId>elasticsearch-spark-${elasticsearch.spark.sufix}_${scala.binary.version}</artifactId>
<version>8.2.2</version>
<version>8.9.1</version>
<scope>test</scope>
<exclusions>
<exclusion>
Expand Down Expand Up @@ -267,6 +267,21 @@
</dependency>
</dependencies>
</profile>
<profile>
<id>spark-3.4</id>
<properties>
<guava.version>16.0.1</guava.version>
<elasticsearch.spark.sufix>30</elasticsearch.spark.sufix>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-spark-runtime-3.4_${scala.binary.version}</artifactId>
<version>1.3.1</version>
<scope>test</scope>
</dependency>
</dependencies>
</profile>
rycowhi marked this conversation as resolved.
Show resolved Hide resolved
</profiles>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package za.co.absa.spline

import org.apache.spark.SPARK_VERSION
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.SaveMode._
Expand All @@ -24,9 +25,13 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
import org.scalatest.flatspec.AsyncFlatSpec
import org.scalatest.matchers.should.Matchers
import za.co.absa.spline.commons.io.TempDirectory
import za.co.absa.spline.commons.version.Version.VersionStringInterpolator
import za.co.absa.spline.producer.model.{ExecutionEvent, ExecutionPlan}
import za.co.absa.spline.test.fixture.SparkFixture
import za.co.absa.spline.test.fixture.spline.SplineFixture

import scala.concurrent.Future

class BasicIntegrationTests extends AsyncFlatSpec
with Matchers
with SparkFixture
Expand Down Expand Up @@ -117,15 +122,41 @@ class BasicIntegrationTests extends AsyncFlatSpec
.write.mode(Append).saveAsTable(tableName)
}

(plan2, _) <- captor.lineageOf {
// Spark 3.4+ is creating 2 commands for both writes here so we need to ignore one
// We only want the one that is from CreateDataSourceTableAsSelectCommand
// The one we ignore here is an extra InsertIntoHadoopFsRelationCommand
// They can come out of order so we need to filter out which one is which.
(plan2, _) <- if (ver"$SPARK_VERSION" >= ver"3.4.0") {
Copy link
Author

Choose a reason for hiding this comment

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

I am not a fan of taking this approach but this does allow the tests to still pass - if we are comfortable with knowing that Spark is firing additional events here we get the same behavior.

Copy link
Contributor

@cerveada cerveada Mar 11, 2024

Choose a reason for hiding this comment

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

So the Spark creates two plans where it used to create just one? What is the new root/write command that it creates? Spline should react only on write commands.

Copy link
Author

@rycowhi rycowhi Mar 11, 2024

Choose a reason for hiding this comment

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

So this is what's confusing me a little bit - we get both a CreateDataSourceTableAsSelectCommand & InsertIntoHadoopFsRelationCommand (printed from LineageHarvester) in an a single Spark Action (and they aren't guaranteed to appear in the same order as well, hence why I needed to do that weird filter to find the right one).

I think I'm going to run a custom QueryExecutionListener and see if Spark itself happens to print out two actions as well. If not I'm going to be even more confused 😅

Copy link
Author

Choose a reason for hiding this comment

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

But yes - this didn't happen in < 3.4

Copy link
Author

Choose a reason for hiding this comment

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

Ok I think I've just confirmed that this is indeed Spark and not Spline doing this. I made a listener specifically for this test like so:

class TestListener extends QueryExecutionListener {

  override def onSuccess(funcName:  String, qe:  QueryExecution, durationNs:  Long): Unit = {
    println("A COMMAND JUST RAN")
    println(qe.commandExecuted.getClass.getCanonicalName)
  }
  override def onFailure(funcName:  _root_.scala.Predef.String, qe:  _root_.org.apache.spark.sql.execution.QueryExecution, exception:  scala.Exception): Unit = {

  }
}

3.3 tests give just per CTAS action.

A COMMAND JUST RAN
org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand

But 3.4 tests give this per CTAS action:

A COMMAND JUST RAN
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
A COMMAND JUST RAN
org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand

This seems to support what I saw from printing out in LineageHarvester.

I unfortunately don't have the context to say whether this is ok or not 😅 . I would imagine Spline UI users would see this additional event, no?

The correct lineage events are getting generated but there's extra noise. I know InsertIntoHadoopFsRelation is associated with other writes occurring so your theory of Spark changing something under the hood is probably correct.

Copy link
Author

Choose a reason for hiding this comment

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

I think I found the Spark PR that introduced this behavior change in 3.4 - TLDR v1 data writes originally in CTAS turned into the two relation's we're seeing here under the two types of CTAS supported. They call out command nesting in various places.

Copy link
Contributor

Choose a reason for hiding this comment

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

So you are sure that both of those commands are triggered by .write.mode(Append).saveAsTable(tableName)?

This is not a question for testing only, but generally how to handle this in the application. Usually there is only one lineage for one write. The simplest solution would be to ignore one of them, but how to check if what we are ignoring is actually duplication?

Copy link
Contributor

Choose a reason for hiding this comment

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

I read the Spark ticket, they actually separate the action of table creation and data insert, that is not a problem. We could generate events for both, but it should be clear that no data are inserted in the table creation lineage.

Copy link
Author

Choose a reason for hiding this comment

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

Would there be another suite/case that we'd want for something like this? I suppose that might depend on the actual events that Spline would be outputting here now.

captor.lineageOf {
Thread.sleep(5000)
}
} else Future[(ExecutionPlan, Seq[ExecutionEvent])](null, null)

(plan3, _) <- captor.lineageOf {
spark
.read.table(tableName)
.write.mode(Overwrite).saveAsTable("somewhere")
}

(plan4, _) <- if (ver"$SPARK_VERSION" >= ver"3.4.0") {
captor.lineageOf {
Thread.sleep(5000)
}
} else Future[(ExecutionPlan, Seq[ExecutionEvent])](null, null)
} yield {
println("yield")
val writeUri = plan1.operations.write.outputSource
val readUri = plan2.operations.reads.head.inputSources.head

val writePlan = Seq(plan1, plan2)
Copy link
Author

Choose a reason for hiding this comment

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

There is some common code here on filtering out the correct options we want here between this and the Hive CTAS code - not sure how much level of DRY you all are looking to reduce here but this is potential.

.filter(null.!=)
.find(_.operations.write.name == "CreateDataSourceTableAsSelectCommand")
.get
val readPlan = Seq(plan3, plan4)
.filter(null.!=)
.find(_.operations.write.name == "CreateDataSourceTableAsSelectCommand")
.get

val writeUri = writePlan.operations.write.outputSource
val readUri = readPlan.operations.reads.head.inputSources.head

writeUri shouldEqual readUri
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ class DeltaSpec extends AsyncFlatSpec
private val deltaPath = TempDirectory(prefix = "delta", pathOnly = true).deleteOnExit().toURI.toString

it should "support Delta Lake as a source" taggedAs
ignoreIf(ver"$SPARK_VERSION" < ver"2.4.2") in
ignoreIf(ver"$SPARK_VERSION" < ver"2.4.2" || ver"$SPARK_VERSION" >= ver"3.4.0") in
withNewSparkSession { implicit spark =>
withLineageTracking { captor =>
val testData: DataFrame = {
Expand Down Expand Up @@ -79,7 +79,7 @@ class DeltaSpec extends AsyncFlatSpec
}

it should "support insert into existing Delta Lake table" taggedAs
ignoreIf(ver"$SPARK_VERSION" < ver"2.4.2") in
ignoreIf(ver"$SPARK_VERSION" < ver"2.4.2" || ver"$SPARK_VERSION" >= ver"3.4.0") in
withNewSparkSession { implicit spark =>
withLineageTracking { lineageCaptor =>
val testData: DataFrame = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ class KafkaSinkSpec

(plan2, _) <- captor.lineageOf(
reader
.option("subscribe", s"$topicName,anotherTopic")
.option("subscribe", s"$topicName")
wajda marked this conversation as resolved.
Show resolved Hide resolved
.load()
.write.mode(Overwrite).save(TempFile(pathOnly = true).deleteOnExit().path.toString))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import za.co.absa.spline.test.fixture.spline.SplineFixture
import za.co.absa.spline.test.fixture.{SparkDatabaseFixture, SparkFixture}

import java.util.UUID
import scala.concurrent.Future
import scala.language.reflectiveCalls
import scala.util.Try

Expand Down Expand Up @@ -376,11 +377,25 @@ class LineageHarvesterSpec extends AsyncFlatSpec
val df = spark.createDataset(Seq(TestRow(1, 2.3, "text")))

for {
(plan, _) <- captor.lineageOf {
(plan1, _) <- captor.lineageOf {
df.createOrReplaceTempView("tempView")
spark.sql("CREATE TABLE users_sales AS SELECT i, d, s FROM tempView ")
}
// Spark 3.4+ is creating 2 commands for this CTAS here so we need to ignore one
// We only want the one that is from CreateHiveTableAsSelectCommand
// The one we ignore here is an extra InsertIntoHiveTableCommand
// They can come out of order so we need to filter out which one is which.
(plan2, _) <- if (ver"$SPARK_VERSION" >= ver"3.4.0") {
captor.lineageOf {
Thread.sleep(5000)
}
} else Future[(ExecutionPlan, Seq[ExecutionEvent])](null, null)
} yield {
val plan = Seq(plan1, plan2)
.filter(null.!=)
.find(_.operations.write.name == "CreateHiveTableAsSelectCommand")
.get

val writeOperation = plan.operations.write
writeOperation.id shouldEqual "op-0"
writeOperation.append shouldEqual false
Expand Down Expand Up @@ -500,7 +515,7 @@ class LineageHarvesterSpec extends AsyncFlatSpec
plan should not be null
event.durationNs should be(empty)
event.error should not be empty
event.error.get.toString should include(s"path ${tmpLocal.toURI.toString.stripSuffix("/")} already exists")
event.error.get.toString.toLowerCase should include(s"path ${tmpLocal.toURI.toString.stripSuffix("/")} already exists")
rycowhi marked this conversation as resolved.
Show resolved Hide resolved
}
}
}
Expand Down
28 changes: 28 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@
<spark-31.version>3.1.3</spark-31.version>
<spark-32.version>3.2.3</spark-32.version>
<spark-33.version>3.3.1</spark-33.version>
<spark-34.version>3.4.1</spark-34.version>

<!-- Delta -->

Expand All @@ -100,6 +101,8 @@
<delta-10.version>1.0.0</delta-10.version>
<delta-20.version>2.0.0</delta-20.version>
<delta-21.version>2.1.0</delta-21.version>
<delta-24.version>2.4.0</delta-24.version>


<!-- Cassandra -->
<cassandra-connector.version>${cassandra-connector-24.version}</cassandra-connector.version>
Expand All @@ -108,6 +111,7 @@
<cassandra-connector-31.version>3.1.0</cassandra-connector-31.version>
<cassandra-connector-32.version>3.2.0</cassandra-connector-32.version>
<cassandra-connector-33.version>3.3.0</cassandra-connector-33.version>
<cassandra-connector-34.version>3.4.1</cassandra-connector-34.version>

<spark-excel.version>0.13.7</spark-excel.version>

Expand Down Expand Up @@ -210,6 +214,7 @@
<groupId>com.github.cerveada</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
<configuration>
<stdout>I</stdout>
rycowhi marked this conversation as resolved.
Show resolved Hide resolved
<reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
<junitxml>.</junitxml>
<filereports>WDF TestSuite.txt</filereports>
Expand Down Expand Up @@ -815,6 +820,29 @@
</dependencyManagement>
</profile>

<profile>
<id>spark-3.4</id>
<properties>
<spark.version>${spark-34.version}</spark.version>
<delta.version>${delta-24.version}</delta.version>
<spark-excel.version>${spark-34.version}_0.19.0</spark-excel.version>
<cassandra-connector.version>${cassandra-connector-34.version}</cassandra-connector.version>
</properties>
<dependencyManagement>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<type>pom</type>
<scope>import</scope>
</dependency>
</dependencies>
</dependencyManagement>
</profile>



<!-- Binary compatibility checking profile -->

<profile>
Expand Down