-
Notifications
You must be signed in to change notification settings - Fork 139
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[BUG] Fix incorrect spark metrics (#324)
### What changes were proposed in this pull request? Fix incorrect spark metrics ### Why are the changes needed? 1. The corresponding shuffle-read records number and shuffle-write records number is not consistent in our internal cluster 2. Log wont show the correct fetch bytes, always return 0 like `22/11/15 13:54:53 INFO RssShuffleDataIterator: Fetch 0 bytes cost 30791 ms and 53 ms to serialize, 347 ms to decompress with unCompressionLength[274815736] ` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? 1. UTs 2. Online spark3 jobs test
- Loading branch information
Showing
4 changed files
with
130 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
...tion-test/spark-common/src/test/java/org/apache/uniffle/test/WriteAndReadMetricsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* 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.uniffle.test; | ||
|
||
import java.lang.reflect.InvocationTargetException; | ||
import java.util.ArrayList; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
import org.apache.spark.sql.Dataset; | ||
import org.apache.spark.sql.Row; | ||
import org.apache.spark.sql.SparkSession; | ||
import org.apache.spark.sql.functions; | ||
import org.apache.spark.status.AppStatusStore; | ||
import org.apache.spark.status.api.v1.StageData; | ||
import org.junit.jupiter.api.Test; | ||
import scala.collection.Seq; | ||
|
||
public class WriteAndReadMetricsTest extends SimpleTestBase { | ||
|
||
@Test | ||
public void test() throws Exception { | ||
run(); | ||
} | ||
|
||
@Override | ||
public Map runTest(SparkSession spark, String fileName) throws Exception { | ||
// take a rest to make sure shuffle server is registered | ||
Thread.sleep(3000); | ||
|
||
Dataset<Row> df1 = spark.range(0, 100, 1, 10) | ||
.select(functions.when(functions.col("id").$less$eq(50), 1) | ||
.otherwise(functions.col("id")).as("key1"), functions.col("id").as("value1")); | ||
df1.createOrReplaceTempView("table1"); | ||
|
||
List list = spark.sql("select count(value1) from table1 group by key1").collectAsList(); | ||
Map<String, Long> result = new HashMap<>(); | ||
result.put("size", Long.valueOf(list.size())); | ||
|
||
for (int stageId : spark.sparkContext().statusTracker().getJobInfo(0).get().stageIds()) { | ||
long writeRecords = getFirstStageData(spark, stageId).shuffleWriteRecords(); | ||
long readRecords = getFirstStageData(spark, stageId).shuffleReadRecords(); | ||
result.put(stageId + "-write-records", writeRecords); | ||
result.put(stageId + "-read-records", readRecords); | ||
} | ||
|
||
return result; | ||
} | ||
|
||
private StageData getFirstStageData(SparkSession spark, int stageId) | ||
throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { | ||
AppStatusStore statestore = spark.sparkContext().statusStore(); | ||
try { | ||
return ((Seq<StageData>)statestore | ||
.getClass() | ||
.getDeclaredMethod( | ||
"stageData", | ||
int.class, | ||
boolean.class | ||
).invoke(statestore, stageId, false)).toList().head(); | ||
} catch (Exception e) { | ||
return ((Seq<StageData>)statestore | ||
.getClass() | ||
.getDeclaredMethod( | ||
"stageData", | ||
int.class, | ||
boolean.class, | ||
List.class, | ||
boolean.class, | ||
double[].class | ||
).invoke( | ||
statestore, stageId, false, new ArrayList<>(), true, new double[]{})).toList().head(); | ||
} | ||
} | ||
} |