wForget commented on code in PR #4642: URL: https://github.com/apache/kyuubi/pull/4642#discussion_r1170898483
########## extensions/spark/kyuubi-extension-spark-common/src/main/scala/org/apache/kyuubi/sql/watchdog/MaxScanStrategy.scala: ########## @@ -0,0 +1,293 @@ +/* + * 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.kyuubi.sql.watchdog + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{PruneFileSourcePartitionHelper, SparkSession, Strategy} +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HiveTableRelation} +import org.apache.spark.sql.catalyst.planning.ScanOperation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, InMemoryFileIndex, LogicalRelation} +import org.apache.spark.sql.types.StructType + +import org.apache.kyuubi.sql.KyuubiSQLConf + +/** + * Add MaxScanStrategy to avoid scan excessive partitions or files + * 1. Check if scan exceed maxPartition of partitioned table + * 2. Check if scan exceed maxFileSize + * This Strategy Add Planner Strategy after LogicalOptimizer + * @param session + */ +case class MaxScanStrategy(session: SparkSession) + extends Strategy + with SQLConfHelper + with PruneFileSourcePartitionHelper { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = { + val maxScanPartitions = conf.getConf(KyuubiSQLConf.WATCHDOG_MAX_PARTITIONS).getOrElse(0) + val maxFileSize = conf.getConf(KyuubiSQLConf.WATCHDOG_MAX_FILE_SIZE).getOrElse(0L) + if (maxScanPartitions > 0 || maxFileSize > 0) { + checkScan(plan, maxScanPartitions, maxFileSize) + } + Nil + } + + private def checkScan(plan: LogicalPlan, maxScanPartitions: Int, maxFileSize: Long): Unit = { + plan match { + case ScanOperation(_, _, relation: HiveTableRelation) => + if (relation.isPartitioned) { + relation.prunedPartitions match { + case Some(prunedPartitions) => + if (maxScanPartitions > 0 && prunedPartitions.size > maxScanPartitions) { + throw new MaxPartitionExceedException( + s""" + |SQL job scan hive partition: ${prunedPartitions.size} + |exceed restrict of hive scan maxPartition $maxScanPartitions + |You should optimize your SQL logical according partition structure + |or shorten query scope such as p_date, detail as below: + |Table: ${relation.tableMeta.qualifiedName} + |Owner: ${relation.tableMeta.owner} + |Partition Structure: ${relation.partitionCols.map(_.name).mkString(", ")} + |""".stripMargin) + } + lazy val scanFileSize = prunedPartitions.flatMap(_.stats).map(_.sizeInBytes).sum + if (maxFileSize > 0 && scanFileSize > maxFileSize) { + throw partTableMaxFileExceedError( + scanFileSize, + maxFileSize, + Some(relation.tableMeta), + prunedPartitions.flatMap(_.storage.locationUri).map(_.toString), + relation.partitionCols.map(_.name)) + } + case _ => + val totalPartitions = session + .sessionState.catalog.externalCatalog.listPartitions( + relation.tableMeta.database, + relation.tableMeta.identifier.table) + if (maxScanPartitions > 0 && totalPartitions.size > maxScanPartitions) { + throw new MaxPartitionExceedException( + s""" + |Your SQL job scan a whole huge table without any partition filter, + |You should optimize your SQL logical according partition structure + |or shorten query scope such as p_date, detail as below: + |Table: ${relation.tableMeta.qualifiedName} + |Owner: ${relation.tableMeta.owner} + |Partition Structure: ${relation.partitionCols.map(_.name).mkString(", ")} + |""".stripMargin) + } + lazy val scanFileSize = totalPartitions.flatMap(_.stats).map(_.sizeInBytes).sum + if (maxFileSize > 0 && scanFileSize > maxFileSize) { + throw new MaxFileSizeExceedException( + s""" + |Your SQL job scan a whole huge table without any partition filter, Review Comment: > user may have a valid partition filter, but the file size is still large enough Here matches the table without partition filtering. ``` relation.prunedPartitions match { case Some(prunedPartitions) => ...... case _ > // This means the prunedPartitions is None ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
