Skip to content

[SPARK-28939][SQL][2.4] Propagate SQLConf for plans executed by toRdd #25734

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

Closed
wants to merge 5 commits into from

Conversation

mgaido91
Copy link
Contributor

@mgaido91 mgaido91 commented Sep 9, 2019

What changes were proposed in this pull request?

The PR proposes to create a custom `RDD` which enables to propagate `SQLConf` also in cases not tracked by SQL execution, as it happens when a `Dataset` is converted to and RDD either using `.rdd` or `.queryExecution.toRdd` and then the returned RDD is used to invoke actions on it.

In this way, SQL configs are effective also in these cases, while earlier they were ignored.

Why are the changes needed?

Without this patch, all the times .rdd or .queryExecution.toRdd are used, all the SQL configs set are ignored. An example of a reproducer can be:

      withSQLConf(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") {
        val df = spark.range(2).selectExpr((0 to 5000).map(i => s"id as field_$i"): _*)
        df.createOrReplaceTempView("spark64kb")
        val data = spark.sql("select * from spark64kb limit 10")
        // Subexpression elimination is used here, despite it should have been disabled
        data.describe()
      }

Why are the changes needed?

Without this patch, all the times .rdd or .queryExecution.toRdd are used, all the SQL configs set are ignored. An example of a reproducer can be:

      withSQLConf(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") {
        val df = spark.range(2).selectExpr((0 to 5000).map(i => s"id as field_$i"): _*)
        df.createOrReplaceTempView("spark64kb")
        val data = spark.sql("select * from spark64kb limit 10")
        // Subexpression elimination is used here, despite it should have been disabled
        data.describe()
      }

Does this PR introduce any user-facing change?

When a user calls .queryExecution.toRdd, a SQLExecutionRDD is returned wrapping the RDD of the execute. When .rdd is used, an additional SQLExecutionRDD is present in the hierarchy.

How was this patch tested?

added UT

@mgaido91
Copy link
Contributor Author

mgaido91 commented Sep 9, 2019

cc @cloud-fan @hvanhovell @maropu @viirya

I haven't yet added the flag requested by @hvanhovell as I have some doubts about it as I expressed in the other PR. I'll add if you think it is needed.

@SparkQA
Copy link

SparkQA commented Sep 9, 2019

Test build #110354 has finished for PR 25734 at commit e9d22e1.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class SQLExecutionRDD(

@SparkQA
Copy link

SparkQA commented Sep 9, 2019

Test build #110356 has finished for PR 25734 at commit d145b14.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member

Hi, All.
Shall we hold on this? The original PR seems to break JDK11 build.

@dongjoon-hyun
Copy link
Member

#25738 is ready.

private val sqlConfigs = conf.getAllConfs
private lazy val sqlConfExecutorSide = {
val props = new Properties()
props.putAll(sqlConfigs.asJava)
Copy link
Member

@dongjoon-hyun dongjoon-hyun Sep 10, 2019

Choose a reason for hiding this comment

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

Hi, @mgaido91 .
Although Apache Spark branch-2.4 doesn't support JDK11 officially, some down-streams support JDK11 on branch-2.4. Could you include #25738 to match master branch?

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-28939][SQL][BACKPORT-2.4] Propagate SQLConf for plans executed… [SPARK-28939][SQL][2.4] Propagate SQLConf for plans executed… Sep 10, 2019
@dongjoon-hyun dongjoon-hyun changed the title [SPARK-28939][SQL][2.4] Propagate SQLConf for plans executed… [SPARK-28939][SQL][2.4] Propagate SQLConf for plans executed by toRdd Sep 10, 2019
@mgaido91
Copy link
Contributor Author

hi all, I am going to update this once the followups are done, meanwhile do you all agree adding the flag proposed by @hvanhovell ? If so, I'll add it when I update this PR. Thanks.

@dongjoon-hyun
Copy link
Member

Yes. I'm +1 for @hvanhovell 's advice for the flag. Please add a document for that flag, too.
cc @gatorsmile

@maropu
Copy link
Member

maropu commented Sep 10, 2019

+1 for the flag to keep the legacy behaivour. Is that document @dongjoon-hyun suggested the migration guide? Yea, we need to update that, too.

@mgaido91
Copy link
Contributor Author

ok thanks, but then shouldn't we add a note in the migration guide for 3.0 too?

@SparkQA
Copy link

SparkQA commented Sep 11, 2019

Test build #110471 has finished for PR 25734 at commit 43bd021.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

buildConf("spark.sql.legacy.rdd.applyConf")
.internal()
.doc("When false, SQL configurations are disregarded when operations on a RDD derived from" +
" a dataframe are executed. This is the (buggy) behavior up to 2.4.3.")
Copy link
Member

Choose a reason for hiding this comment

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

up to 2.4.4 ?


- Starting from 2.4.5, SQL configurations are effective also when a Dataset is converted to an RDD and its
plan is executed due to action on the derived RDD. The previous buggy behavior can be restored setting
`spark.sql.legacy.rdd.applyConf` to `false`.
Copy link
Member

Choose a reason for hiding this comment

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

Hi, @hvanhovell , @cloud-fan and @gatorsmile . As @mgaido91 asked here, this PR will add this flag only at branch-2.4. In this case, is it okay this config will be added and deprecated at 2.4.5 and will be removed at 3.0.0?

For me, we don't need to add this configuration to master. Did I understand correctly?

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think we need to add the flag to master. We are allowed to break behavior there.

## Upgrading from Spark SQL 2.4 to 2.4.5

- Starting from 2.4.5, SQL configurations are effective also when a Dataset is converted to an RDD and its
plan is executed due to action on the derived RDD. The previous buggy behavior can be restored setting
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd personally refrain from using the term buggy. Please explain what the previous behavior was.

@SparkQA
Copy link

SparkQA commented Sep 12, 2019

Test build #110506 has finished for PR 25734 at commit a5eb604.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 12, 2019

Test build #110512 has finished for PR 25734 at commit 1b145e2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA
Copy link

SparkQA commented Nov 10, 2019

Test build #113545 has finished for PR 25734 at commit 1b145e2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. Merged to branch-2.4.
Sorry for being late, @mgaido91 .

cc @gatorsmile

dongjoon-hyun pushed a commit that referenced this pull request Nov 10, 2019
### What changes were proposed in this pull request?
    The PR proposes to create a custom `RDD` which enables to propagate `SQLConf` also in cases not tracked by SQL execution, as it happens when a `Dataset` is converted to and RDD either using `.rdd` or `.queryExecution.toRdd` and then the returned RDD is used to invoke actions on it.

In this way, SQL configs are effective also in these cases, while earlier they were ignored.

### Why are the changes needed?

Without this patch, all the times `.rdd` or `.queryExecution.toRdd` are used, all the SQL configs set are ignored. An example of a reproducer can be:
```
      withSQLConf(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") {
        val df = spark.range(2).selectExpr((0 to 5000).map(i => s"id as field_$i"): _*)
        df.createOrReplaceTempView("spark64kb")
        val data = spark.sql("select * from spark64kb limit 10")
        // Subexpression elimination is used here, despite it should have been disabled
        data.describe()
      }
```

### Why are the changes needed?
Without this patch, all the times `.rdd` or `.queryExecution.toRdd` are used, all the SQL configs set are ignored. An example of a reproducer can be:
```
      withSQLConf(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") {
        val df = spark.range(2).selectExpr((0 to 5000).map(i => s"id as field_$i"): _*)
        df.createOrReplaceTempView("spark64kb")
        val data = spark.sql("select * from spark64kb limit 10")
        // Subexpression elimination is used here, despite it should have been disabled
        data.describe()
      }
```

### Does this PR introduce any user-facing change?
When a user calls `.queryExecution.toRdd`, a `SQLExecutionRDD` is returned wrapping the `RDD` of the execute. When `.rdd` is used, an additional `SQLExecutionRDD` is present in the hierarchy.

### How was this patch tested?
added UT

Closes #25734 from mgaido91/SPARK-28939_2.4.

Authored-by: Marco Gaido <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants