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

Conversation

rycowhi
Copy link

@rycowhi rycowhi commented Mar 8, 2024

Addressing #705 ,

  • Makes changes to failing tests to perform different behavior based on Spark version
  • No new functionality actually required to meet compatibility

TODO:

  • Create Agent Bundle POM
  • Validate approach is sensible
  • Rule out additional regression from changed Spark behavior
  • Request CI addition for Spark 3.4

@rycowhi rycowhi mentioned this pull request Mar 8, 2024
// 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.

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.

pom.xml Show resolved Hide resolved
@rycowhi rycowhi changed the title Provide Spark 3.4 Support for Spline w/ Backwards Compatability Provide Spark 3.4 Support for Spline w/ Backwards Compatibility Mar 8, 2024
@@ -0,0 +1,1344 @@
<?xml version="1.0" encoding="UTF-8"?>
Copy link
Author

@rycowhi rycowhi Mar 8, 2024

Choose a reason for hiding this comment

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

Want to make sure I've done this right -

  1. Ran the jars-pommefizer on spark-3.4.1-bin-hadoop3
  2. Copied over some basic pom setup info from bundle-3.3 (higher level project template
  3. Moved dependencies from pommefizer output into dependencyManagement
  4. Filled in dependencies whose groupIds were blank from pommefizer (copied from 3.3, maybe a bug in pommefizer?)
  5. Copied build plugins from 3.3
  6. Manually removed versions from everything mentioned in dependencyManagement (this was a doozy 😄 )

Copy link
Contributor

Choose a reason for hiding this comment

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

Seems alright, @wajda what do you think?

Copy link

sonarcloud bot commented Mar 11, 2024

Quality Gate Passed Quality Gate passed

Issues
1 New issue
0 Accepted issues

Measures
0 Security Hotspots
No data about Coverage
0.0% Duplication on New Code

See analysis details on SonarCloud

@wajda
Copy link
Contributor

wajda commented Mar 12, 2024

@rycowhi Thanks a million for your effort.
Guys let me take a look at those event pairs more closely and I'll get back to you later. I'm a bit out of capacity at the moment, but will try to find some time during the next day or two.

@devsaik
Copy link

devsaik commented Mar 18, 2024

Just checking on status of this PR. We are waiting on this feature. @wajda any updates.

@wajda
Copy link
Contributor

wajda commented Mar 18, 2024

Just checking on status of this PR. We are waiting on this feature. @wajda any updates.

Not yet, but it's high on my todo list for this week.

@luke3481
Copy link

@wajda thanks for taking a look at @rycowhi's PR. Let me know if you'll be able to get to it this week - we're also waiting on this feature.

@kaimalsreeja
Copy link

@wajda Looks like this feature is in high demand. Our team is also watching the PR merge, highly appreciate if you could review this as early as possible. Thank you in advance.

@wajda
Copy link
Contributor

wajda commented Apr 4, 2024

Sorry folks, I've still been buried in work on another high-priority project, and have yet to obtain time allocation approval for this one. The Spline project was essentially put on hold by the company last year, that's the reason why it received almost no support for the past X months. I'm not giving up, but I can't really spend my free time on this any more, so trying to get some official allocation from the employer.

@ramonje5
Copy link

ramonje5 commented May 24, 2024

Sorry folks, I've still been buried in work on another high-priority project, and have yet to obtain time allocation approval for this one. The Spline project was essentially put on hold by the company last year, that's the reason why it received almost no support for the past X months. I'm not giving up, but I can't really spend my free time on this any more, so trying to get some official allocation from the employer.

Hello! Is there any news regarding this Spark 3.4 support? I've been using Spline since a long time and the fact that there is no support for Scala 2.13 is going to affect a lot of projects and maybe the need to find a new listener.

If there is any way I could contribute to this feature, or to speed things up, please let me know!

Thanks!

Edit: I've just seen Upgrade org.scala-lang:scala-library from 2.12.17 to 2.13.14 #806

@wajda
Copy link
Contributor

wajda commented May 24, 2024

@ramonje5 thank you for the message. Unfortunately our company has yet to sort out its plans and book of work for Spline. Currently the team has zero capacity to work on it.

If there is any way I could contribute to this feature, or to speed things up, please let me know!

Thanks!

Edit: I've just seen #806

The PR #806 is automatically created by Snyk and won't work. Please ignore it. Upgrading to Scala 2.13 cannot be a simple change like that, it would require adding another vertical into the build matrix, so to speak. Basically the agent is built for every supported Sclala+Spark version combination.

If you want to help that would be awesome if you take this current PR, test it, address @rycowhi's points in the related issue #705. When testing succeeds and those points are addressed we can merge and release this PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

7 participants