Skip to content
This repository has been archived by the owner on Sep 18, 2023. It is now read-only.

[NSE-326] update scala unit tests to spark-3.1.1 #327

Merged
merged 5 commits into from
May 25, 2021
Merged
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
move back tests on columnar shuffle
rui-mo committed May 21, 2021
commit 0098a9a7be12eee28a21b169c29dc19ae277dca2
Original file line number Diff line number Diff line change
@@ -185,23 +185,6 @@ object InMemoryRelation {
relation
}

def apply(
storageLevel: StorageLevel,
qe: QueryExecution,
tableName: Option[String]): InMemoryRelation = {
val optimizedPlan = qe.optimizedPlan
val serializer = getSerializer(optimizedPlan.conf)
val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {
convertToColumnarIfPossible(qe.executedPlan)
} else {
qe.executedPlan
}
val cacheBuilder = CachedRDDBuilder(serializer, storageLevel, child, tableName)
val relation = new InMemoryRelation(child.output, cacheBuilder, optimizedPlan.outputOrdering)
relation.statsOfPlanToCache = optimizedPlan.stats
relation
}

/**
* This API is intended only to be used for testing.
*/
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,196 @@
/*
* 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.spark.sql

import com.intel.oap.execution.ColumnarHashAggregateExec
import com.intel.oap.datasource.parquet.ParquetReader
import org.apache.spark.SparkConf
import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
import org.apache.spark.sql.execution.{
ColumnarShuffleExchangeExec,
ColumnarToRowExec,
RowToColumnarExec
}
import org.apache.spark.sql.test.SharedSparkSession

class RepartitionSuite extends QueryTest with SharedSparkSession {
import testImplicits._

override def sparkConf: SparkConf =
super.sparkConf
.setAppName("test repartition")
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")

def checkCoulumnarExec(data: DataFrame) = {
val found = data.queryExecution.executedPlan
.collect {
case r2c: RowToColumnarExec => 1
case c2r: ColumnarToRowExec => 10
case exc: ColumnarShuffleExchangeExec => 100
}
.distinct
.sum
assert(found == 110)
}

def withInput(input: DataFrame)(
transformation: Option[DataFrame => DataFrame],
repartition: DataFrame => DataFrame): Unit = {
val expected = transformation.getOrElse(identity[DataFrame](_))(input)
val data = repartition(expected)
checkCoulumnarExec(data)
checkAnswer(data, expected)
}

lazy val input: DataFrame = Seq((1, "1"), (2, "20"), (3, "300")).toDF("id", "val")

def withTransformationAndRepartition(
transformation: DataFrame => DataFrame,
repartition: DataFrame => DataFrame): Unit =
withInput(input)(Some(transformation), repartition)

def withRepartition: (DataFrame => DataFrame) => Unit = withInput(input)(None, _)
}

class TPCHTableRepartitionSuite extends RepartitionSuite {
import testImplicits._

val filePath = getTestResourcePath(
"test-data/part-00000-d648dd34-c9d2-4fe9-87f2-770ef3551442-c000.snappy.parquet")

override lazy val input = spark.read.format("arrow").load(filePath)

ignore("tpch table round robin partitioning") {
withRepartition(df => df.repartition(2))
}

ignore("tpch table hash partitioning") {
withRepartition(df => df.repartition('n_nationkey))
}

ignore("tpch table range partitioning") {
withRepartition(df => df.repartitionByRange('n_name))
}

ignore("tpch table hash partitioning with expression") {
withRepartition(df => df.repartition('n_nationkey + 'n_regionkey))
}

ignore("tpch table sum after repartition") {
withTransformationAndRepartition(
df => df.groupBy("n_regionkey").agg(Map("n_nationkey" -> "sum")),
df => df.repartition(2))
}
}

class DisableColumnarShuffleSuite extends RepartitionSuite {
import testImplicits._

override def sparkConf: SparkConf = {
super.sparkConf
.set("spark.shuffle.manager", "sort")
.set("spark.sql.codegen.wholeStage", "true")
}

override def checkCoulumnarExec(data: DataFrame) = {
val found = data.queryExecution.executedPlan
.collectFirst {
case exc: ColumnarShuffleExchangeExec => exc
}
data.explain
assert(found.isEmpty)
}

test("round robin partitioning") {
withRepartition(df => df.repartition(2))
}

test("hash partitioning") {
withRepartition(df => df.repartition('id))
}

test("range partitioning") {
withRepartition(df => df.repartitionByRange('id))
}
}

class AdaptiveQueryExecRepartitionSuite extends TPCHTableRepartitionSuite {
override def sparkConf: SparkConf = {
super.sparkConf
.set("spark.sql.adaptive.enabled", "true")
}

def checkBefore(data: DataFrame) = {
val planBefore = data.queryExecution.executedPlan
assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false"))
}

def checkAfter(data: DataFrame) = {
val planAfter = data.queryExecution.executedPlan
assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true"))
val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan

val found = adaptivePlan
.collect {
case c2r: ColumnarToRowExec => 1
case row: ShuffleExchangeExec => 10
case col: ColumnarShuffleExchangeExec => 100
}
.distinct
.sum
assert(found == 1, "The final plan should not contain any Exchange node.")
}

override def withInput(input: DataFrame)(
transformation: Option[DataFrame => DataFrame],
repartition: DataFrame => DataFrame): Unit = {
val expected = transformation.getOrElse(identity[DataFrame](_))(input)
val data = repartition(expected)
checkBefore(data)
checkAnswer(data, expected)
checkAfter(data)
}

}

class ReuseExchangeSuite extends RepartitionSuite {
val filePath = getTestResourcePath(
"test-data/part-00000-d648dd34-c9d2-4fe9-87f2-770ef3551442-c000.snappy.parquet")

override lazy val input = spark.read.parquet(filePath)

ignore("columnar exchange same result") {
val df1 = input.groupBy("n_regionkey").agg(Map("n_nationkey" -> "sum"))
val hashAgg1 = df1.queryExecution.executedPlan.collectFirst {
case agg: ColumnarHashAggregateExec => agg
}.get

val df2 = input.groupBy("n_regionkey").agg(Map("n_nationkey" -> "sum"))
val hashAgg2 = df2.queryExecution.executedPlan.collectFirst {
case agg: ColumnarHashAggregateExec => agg
}.get

assert(hashAgg1.sameResult(hashAgg2))

val exchange1 = new ColumnarShuffleExchangeExec(UnknownPartitioning(1), hashAgg1)
val exchange2 = new ColumnarShuffleExchangeExec(UnknownPartitioning(1), hashAgg2)
assert(exchange1.sameResult(exchange2))
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
/*
* 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.spark.sql.execution.adaptive

import java.io.File
import java.net.URI

import org.apache.log4j.Level
import org.apache.spark.SparkConf
import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart}
import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy}
import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight}
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
import org.apache.spark.sql.execution.{PartialReducerPartitionSpec, QueryExecution, ReusedSubqueryExec, ShuffledRowRDD, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.command.DataWritingCommandExec
import org.apache.spark.sql.execution.datasources.noop.NoopDataSource
import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, REPARTITION, REPARTITION_WITH_NUM, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike}
import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, SortMergeJoinExec}
import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter
import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, StructType}
import org.apache.spark.sql.util.QueryExecutionListener
import org.apache.spark.util.Utils

class ColumnarAdaptiveQueryExecSuite
extends QueryTest
with SharedSparkSession
with AdaptiveSparkPlanHelper {

import testImplicits._

override def sparkConf: SparkConf =
super.sparkConf
.setAppName("test repartition")
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")

setupTestData()

private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = {
var finalPlanCnt = 0
val listener = new SparkListener {
override def onOtherEvent(event: SparkListenerEvent): Unit = {
event match {
case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) =>
if (sparkPlanInfo.simpleString.startsWith(
"AdaptiveSparkPlan isFinalPlan=true")) {
finalPlanCnt += 1
}
case _ => // ignore other events
}
}
}
spark.sparkContext.addSparkListener(listener)

val dfAdaptive = sql(query)
val planBefore = dfAdaptive.queryExecution.executedPlan
assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false"))
val result = dfAdaptive.collect()
withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") {
val df = sql(query)
checkAnswer(df, result)
}
val planAfter = dfAdaptive.queryExecution.executedPlan
assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true"))
val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan

spark.sparkContext.listenerBus.waitUntilEmpty()
// AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that
// exist out of query stages.
val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1)
assert(finalPlanCnt == expectedFinalPlanCnt)
spark.sparkContext.removeSparkListener(listener)

val exchanges = adaptivePlan.collect {
case e: Exchange => e
}
assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.")
(dfAdaptive.queryExecution.sparkPlan, adaptivePlan)
}

private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = {
collect(plan) {
case j: BroadcastHashJoinExec => j
}
}

private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = {
collect(plan) {
case j: SortMergeJoinExec => j
}
}

private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = {
collect(plan) {
case j: BaseJoinExec => j
}
}

private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = {
collectWithSubqueries(plan) {
case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e
case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e
}
}

private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = {
collectWithSubqueries(plan) {
case e: ReusedSubqueryExec => e
}
}

private def checkNumLocalShuffleReaders(
plan: SparkPlan, numShufflesWithoutLocalReader: Int = 0): Unit = {
val numShuffles = collect(plan) {
case s: ShuffleQueryStageExec => s
}.length

val numLocalReaders = collect(plan) {
case reader: CustomShuffleReaderExec if reader.isLocalReader => reader
}
numLocalReaders.foreach { r =>
val rdd = r.execute()
val parts = rdd.partitions
assert(parts.forall(rdd.preferredLocations(_).nonEmpty))
}
assert(numShuffles === (numLocalReaders.length + numShufflesWithoutLocalReader))
}

private def checkInitialPartitionNum(df: Dataset[_], numPartition: Int): Unit = {
// repartition obeys initialPartitionNum when adaptiveExecutionEnabled
val plan = df.queryExecution.executedPlan
assert(plan.isInstanceOf[AdaptiveSparkPlanExec])
val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect {
case s: ShuffleExchangeExec => s
}
assert(shuffle.size == 1)
assert(shuffle(0).outputPartitioning.numPartitions == numPartition)
}

test("Columnar exchange reuse") {
withSQLConf(
SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true",
SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") {
val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(
"SELECT value FROM testData join testData2 ON key = a " +
"join (SELECT value v from testData join testData3 ON key = a) on value = v")
val ex = findReusedExchange(adaptivePlan)
assert(ex.size == 1)
}
}
}