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

[SPARK-45357][CONNECT][TESTS][3.5] Normalize dataframeId when comparing CollectMetrics in SparkConnectProtoSuite #45141

Closed
wants to merge 1 commit into from

Conversation

LuciferYang
Copy link
Contributor

What changes were proposed in this pull request?

This PR add a new function normalizeDataframeId to sets the dataframeId to the constant 0 of CollectMetrics before comparing LogicalPlan in the test case of SparkConnectProtoSuite.

Why are the changes needed?

The test scenario in SparkConnectProtoSuite does not need to compare the dataframeId in CollectMetrics

Does this PR introduce any user-facing change?

No

How was this patch tested?

  • Manually check

run

build/mvn clean install -pl connector/connect/server -am -DskipTests
build/mvn test -pl connector/connect/server 

Before

- Test observe *** FAILED ***
  == FAIL: Plans do not match ===
  !CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 0   CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 53
   +- LocalRelation <empty>, [id#0, name#0]                                                                 +- LocalRelation <empty>, [id#0, name#0] (PlanTest.scala:179) 

After

Run completed in 41 seconds, 631 milliseconds.
Total number of tests run: 882
Suites: completed 24, aborted 0
Tests: succeeded 882, failed 0, canceled 0, ignored 0, pending 0
All tests passed.

Was this patch authored or co-authored using generative AI tooling?

No

…`CollectMetrics` in `SparkConnectProtoSuite`

### What changes were proposed in this pull request?
This PR add a new function `normalizeDataframeId` to sets the `dataframeId` to the constant 0 of `CollectMetrics`  before comparing `LogicalPlan` in the test case of `SparkConnectProtoSuite`.

### Why are the changes needed?
The test scenario in `SparkConnectProtoSuite` does not need to compare the `dataframeId` in `CollectMetrics`

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
- Manually check

run

```
build/mvn clean install -pl connector/connect/server -am -DskipTests
build/mvn test -pl connector/connect/server
```

**Before**

```
- Test observe *** FAILED ***
  == FAIL: Plans do not match ===
  !CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 0   CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 53
   +- LocalRelation <empty>, [id#0, name#0]                                                                 +- LocalRelation <empty>, [id#0, name#0] (PlanTest.scala:179)
```

**After**

```
Run completed in 41 seconds, 631 milliseconds.
Total number of tests run: 882
Suites: completed 24, aborted 0
Tests: succeeded 882, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes apache#43155 from LuciferYang/SPARK-45357.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Ruifeng Zheng <ruifengz@apache.org>
@LuciferYang
Copy link
Contributor Author

cc @HeartSaVioR @srowen @amaliujia

Actually, I didn't reproduce this issue locally because when I use Maven to test the branch-3.5 connect module, the order of test case execution is SparkConnectStreamingQueryCacheSuite, ExecuteEventsManagerSuite, SparkConnectProtoSuite... and there are no DataFrame instances in SparkConnectStreamingQueryCacheSuite and ExecuteEventsManagerSuite. Therefore, sparkTestRelation in SparkConnectProtoSuite is still the first DataFrame to be initialized, its id is 0, which bypasses this issue.

However, we can use some tricky methods to reproduce the failure. For example, change

test("Basic select") {
val connectPlan = connectTestRelation.select("id".protoAttr)
val sparkPlan = sparkTestRelation.select("id")
comparePlans(connectPlan, sparkPlan)
}

to

test("Basic select") {
  val connectPlan = connectTestRelation.select("id".protoAttr)
  val sparkPlan = sparkTestRelation2.select("id")
  comparePlans(connectPlan, sparkPlan)
}

In this way, sparkTestRelation will definitely not be the first DataFrame to be initialized, and the test failure can be reproduced, but Basic select will still pass the test.

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.

Do we need this to branch-3.4, @LuciferYang ?

dongjoon-hyun pushed a commit that referenced this pull request Feb 16, 2024
…ring `CollectMetrics` in `SparkConnectProtoSuite`

### What changes were proposed in this pull request?
This PR add a new function `normalizeDataframeId` to sets the `dataframeId` to the constant 0 of `CollectMetrics`  before comparing `LogicalPlan` in the test case of `SparkConnectProtoSuite`.

### Why are the changes needed?
The test scenario in `SparkConnectProtoSuite` does not need to compare the `dataframeId` in `CollectMetrics`

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
- Manually check

run

```
build/mvn clean install -pl connector/connect/server -am -DskipTests
build/mvn test -pl connector/connect/server
```

**Before**

```
- Test observe *** FAILED ***
  == FAIL: Plans do not match ===
  !CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 0   CollectMetrics my_metric, [min(id#0) AS min_val#0, max(id#0) AS max_val#0, sum(id#0) AS sum(id)#0L], 53
   +- LocalRelation <empty>, [id#0, name#0]                                                                 +- LocalRelation <empty>, [id#0, name#0] (PlanTest.scala:179)
```

**After**

```
Run completed in 41 seconds, 631 milliseconds.
Total number of tests run: 882
Suites: completed 24, aborted 0
Tests: succeeded 882, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #45141 from LuciferYang/SPARK-45357-35.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
@dongjoon-hyun
Copy link
Member

Merged to branch-3.5.

@HeartSaVioR
Copy link
Contributor

Late +1.

@LuciferYang
Copy link
Contributor Author

LuciferYang commented Feb 19, 2024

+1, LGTM.

Do we need this to branch-3.4, @LuciferYang ?

branch-3.4 does not need this patch

@LuciferYang
Copy link
Contributor Author

Thanks @srowen @dongjoon-hyun @HeartSaVioR

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