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

[KYUUBI #5377][FOLLOWUP] Get limit from more spark plan and regard result max rows #5963

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@ import org.apache.spark.network.util.{ByteUnit, JavaUtils}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.plans.logical.GlobalLimit
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
import org.apache.spark.sql.execution.{CollectLimitExec, LocalTableScanExec, SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.HiveResult
import org.apache.spark.sql.execution.{CollectLimitExec, HiveResult, LocalTableScanExec, QueryExecution, SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
import org.apache.spark.sql.execution.arrow.KyuubiArrowConverters
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
Expand Down Expand Up @@ -295,16 +295,23 @@ object SparkDatasetHelper extends Logging {
SQLMetrics.postDriverMetricUpdates(sc, executionId, metrics.values.toSeq)
}

private[kyuubi] def optimizedPlanLimit(queryExecution: QueryExecution): Option[Long] =
queryExecution.optimizedPlan match {
case globalLimit: GlobalLimit => globalLimit.maxRows
case _ => None
}

def shouldSaveResultToFs(resultMaxRows: Int, minSize: Long, result: DataFrame): Boolean = {
if (isCommandExec(result.queryExecution.executedPlan.nodeName)) {
return false
}
lazy val limit = result.queryExecution.executedPlan match {
case collectLimit: CollectLimitExec => collectLimit.limit
case _ => resultMaxRows
val finalLimit = optimizedPlanLimit(result.queryExecution) match {
case Some(limit) if resultMaxRows > 0 => math.min(limit, resultMaxRows)
case Some(limit) => limit
case None => resultMaxRows
}
lazy val stats = if (limit > 0) {
limit * EstimationUtils.getSizePerRow(
lazy val stats = if (finalLimit > 0) {
finalLimit * EstimationUtils.getSizePerRow(
result.queryExecution.executedPlan.output)
} else {
result.queryExecution.optimizedPlan.stats.sizeInBytes
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.kyuubi

import org.apache.spark.sql.internal.SQLConf

import org.apache.kyuubi.engine.spark.WithSparkSQLEngine

class SparkDatasetHelperSuite extends WithSparkSQLEngine {
override def withKyuubiConf: Map[String, String] = Map.empty

test("get limit from spark plan") {
Seq(true, false).foreach { aqe =>
val topKThreshold = 3
spark.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, aqe)
spark.sessionState.conf.setConf(SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD, topKThreshold)
spark.sql("CREATE OR REPLACE TEMPORARY VIEW tv AS" +
" SELECT * FROM VALUES(1),(2),(3),(4) AS t(id)")

val topKStatement = s"SELECT * FROM(SELECT * FROM tv ORDER BY id LIMIT ${topKThreshold - 1})"
assert(SparkDatasetHelper.optimizedPlanLimit(
spark.sql(topKStatement).queryExecution) === Option(topKThreshold - 1))

val collectLimitStatement =
s"SELECT * FROM (SELECT * FROM tv ORDER BY id LIMIT $topKThreshold)"
assert(SparkDatasetHelper.optimizedPlanLimit(
spark.sql(collectLimitStatement).queryExecution) === Option(topKThreshold))
}
}
}
Loading