diff --git a/docs/configs.md b/docs/configs.md index f18f55a2bff..c173ed55bfb 100644 --- a/docs/configs.md +++ b/docs/configs.md @@ -77,6 +77,7 @@ Name | Description | Default Value spark.rapids.sql.improvedTimeOps.enabled|When set to true, some operators will avoid overflowing by converting epoch days directly to seconds without first converting to microseconds|false spark.rapids.sql.incompatibleDateFormats.enabled|When parsing strings as dates and timestamps in functions like unix_timestamp, setting this to true will force all parsing onto GPU even for formats that can result in incorrect results when parsing invalid inputs.|false spark.rapids.sql.incompatibleOps.enabled|For operations that work, but are not 100% compatible with the Spark equivalent set if they should be enabled by default or disabled by default.|false +spark.rapids.sql.metrics.level|GPU plans can produce a lot more metrics than CPU plans do. In very large queries this can sometimes result in going over the max result size limit for the driver. Supported values include DEBUG which will enable all metrics supported and typically only needs to be enabled when debugging the plugin. MODERATE which should output enough metrics to understand how long each part of the query is taking and how much data is going to each part of the query. ESSENTIAL which disables most metrics except those Apache Spark CPU plans will also report or their equivalents.|MODERATE spark.rapids.sql.python.gpu.enabled|This is an experimental feature and is likely to change in the future. Enable (true) or disable (false) support for scheduling Python Pandas UDFs with GPU resources. When enabled, pandas UDFs are assumed to share the same GPU that the RAPIDs accelerator uses and will honor the python GPU configs|false spark.rapids.sql.reader.batchSizeBytes|Soft limit on the maximum number of bytes the reader reads per batch. The readers will read chunks of data until this limit is met or exceeded. Note that the reader may estimate the number of bytes that will be used on the GPU in some cases based on the schema and number of rows in each batch.|2147483647 spark.rapids.sql.reader.batchSizeRows|Soft limit on the maximum number of rows the reader will read per batch. The orc and parquet readers will read row groups until this limit is met or exceeded. The limit is respected by the csv reader.|2147483647 diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala index 300c8f64c29..492198a8519 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuBroadcastHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids.shims.spark300 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -28,9 +27,7 @@ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -96,12 +93,15 @@ case class GpuBroadcastHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + import GpuMetric._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -125,17 +125,17 @@ case class GpuBroadcastHashJoinExec( throw new IllegalStateException( "GpuBroadcastHashJoin does not support row-based processing") - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala index 24497948c01..43540feef2e 100644 --- a/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala +++ b/shims/spark300/src/main/scala/com/nvidia/spark/rapids/shims/spark300/GpuShuffledHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids.shims.spark300 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -27,9 +26,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide, ShuffledHashJoinExec} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.{GpuHashJoin, GpuShuffledHashJoinBase} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object GpuJoinUtils { @@ -94,14 +91,17 @@ case class GpuShuffledHashJoinExec( right, isSkewJoin) with GpuHashJoin { - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + import GpuMetric._ + + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + BUILD_DATA_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_DATA_SIZE), + BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME), + STREAM_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -117,15 +117,15 @@ case class GpuShuffledHashJoinExec( } override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val buildDataSize = longMetric("buildDataSize") - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val buildTime = longMetric("buildTime") - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val buildTime = gpuLongMetric(BUILD_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala index 32d65843899..d67d5585caa 100644 --- a/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala +++ b/shims/spark301/src/main/scala/com/nvidia/spark/rapids/shims/spark301/GpuBroadcastHashJoinExec.scala @@ -1,11 +1,11 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed 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 + * 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, @@ -13,10 +13,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.nvidia.spark.rapids.shims.spark301 -import com.nvidia.spark.rapids.{BaseExprMeta, DataFromReplacementRule, GpuBindReferences, GpuBroadcastJoinMeta, GpuBuildLeft, GpuBuildRight, GpuBuildSide, GpuColumnVector, GpuExec, GpuOverrides, GpuProjectExec, RapidsConf, RapidsMeta} -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.shims.spark300.GpuJoinUtils import org.apache.spark.rdd.RDD @@ -27,10 +27,8 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, HashedRelationBroadcastMode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -93,12 +91,15 @@ case class GpuBroadcastHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + import GpuMetric._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -119,19 +120,20 @@ case class GpuBroadcastHashJoinExec( } override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + throw new IllegalStateException( + "GpuBroadcastHashJoin does not support row-based processing") + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuBroadcastHashJoinExec.scala b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuBroadcastHashJoinExec.scala index fc5fff0daba..4fbfd9713db 100644 --- a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuBroadcastHashJoinExec.scala +++ b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuBroadcastHashJoinExec.scala @@ -1,11 +1,11 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed 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 + * 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, @@ -16,7 +16,6 @@ package com.nvidia.spark.rapids.shims.spark301db import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,8 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch class GpuBroadcastHashJoinMeta( @@ -93,12 +90,15 @@ case class GpuBroadcastHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + import GpuMetric._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -119,19 +119,20 @@ case class GpuBroadcastHashJoinExec( } override def doExecute(): RDD[InternalRow] = - throw new IllegalStateException("GpuBroadcastHashJoin does not support row-based processing") - - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + throw new IllegalStateException( + "GpuBroadcastHashJoin does not support row-based processing") + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuShuffledHashJoinExec.scala b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuShuffledHashJoinExec.scala index 29b63f2c6c7..21f76b2f29f 100644 --- a/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuShuffledHashJoinExec.scala +++ b/shims/spark301db/src/main/scala/com/nvidia/spark/rapids/shims/spark301db/GpuShuffledHashJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids.shims.spark301db import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -28,9 +27,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.GpuHashJoin -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object GpuJoinUtils { @@ -84,14 +81,17 @@ case class GpuShuffledHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + import GpuMetric._ + + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + BUILD_DATA_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_DATA_SIZE), + BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME), + STREAM_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -107,15 +107,15 @@ case class GpuShuffledHashJoinExec( } override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val buildDataSize = longMetric("buildDataSize") - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val buildTime = longMetric("buildTime") - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val buildTime = gpuLongMetric(BUILD_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuBroadcastHashJoinExec.scala b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuBroadcastHashJoinExec.scala index 1ac848db516..3a36fb2c7f2 100644 --- a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuBroadcastHashJoinExec.scala +++ b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuBroadcastHashJoinExec.scala @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids.shims.spark311 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import com.nvidia.spark.rapids.shims.spark301.GpuBroadcastExchangeExec import org.apache.spark.rdd.RDD @@ -29,10 +28,8 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.rapids.execution.{GpuHashJoin, SerializeConcatHostBuffersDeserializeBatch} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -98,12 +95,15 @@ case class GpuBroadcastHashJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan) extends BinaryExecNode with GpuHashJoin { + import GpuMetric._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + STREAM_TIME -> createNanoTimingMetric(DEBUG_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -127,17 +127,17 @@ case class GpuBroadcastHashJoinExec( throw new IllegalStateException( "GpuBroadcastHashJoin does not support row-based processing") - override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val broadcastRelation = broadcastExchange - .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() + .executeColumnarBroadcast[SerializeConcatHostBuffersDeserializeBatch]() val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuShuffledHashJoinExec.scala b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuShuffledHashJoinExec.scala index 63f4cf790d4..30c50804c86 100644 --- a/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuShuffledHashJoinExec.scala +++ b/shims/spark311/src/main/scala/com/nvidia/spark/rapids/shims/spark311/GpuShuffledHashJoinExec.scala @@ -17,7 +17,6 @@ package com.nvidia.spark.rapids.shims.spark311 import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -28,9 +27,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.{GpuHashJoin, GpuShuffledHashJoinBase} -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object GpuJoinUtils { @@ -95,14 +92,17 @@ case class GpuShuffledHashJoinExec( right, isSkewJoin) with GpuHashJoin { - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "streamTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "stream time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + import GpuMetric._ + + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + BUILD_DATA_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_DATA_SIZE), + BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME), + STREAM_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_STREAM_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -118,15 +118,15 @@ case class GpuShuffledHashJoinExec( } override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val buildDataSize = longMetric("buildDataSize") - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val buildTime = longMetric("buildTime") - val streamTime = longMetric("streamTime") - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val buildTime = gpuLongMetric(BUILD_TIME) + val streamTime = gpuLongMetric(STREAM_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val boundCondition = condition.map(GpuBindReferences.bindReference(_, output)) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/UnsafeRowToColumnarBatchIterator.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/UnsafeRowToColumnarBatchIterator.java index a887a904150..6e0a45eed77 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/UnsafeRowToColumnarBatchIterator.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/UnsafeRowToColumnarBatchIterator.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.CudfUnsafeRow; import org.apache.spark.sql.catalyst.expressions.Attribute; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.execution.metric.SQLMetric; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.vectorized.ColumnarBatch; import scala.collection.Iterator; @@ -50,19 +49,19 @@ public abstract class UnsafeRowToColumnarBatchIterator implements Iterator input, Attribute[] schema, CoalesceGoal goal, - SQLMetric totalTime, - SQLMetric numInputRows, - SQLMetric numOutputRows, - SQLMetric numOutputBatches) { + GpuMetric totalTime, + GpuMetric numInputRows, + GpuMetric numOutputRows, + GpuMetric numOutputBatches) { this.input = input; int sizePerRowEstimate = CudfUnsafeRow.getRowSizeEstimate(schema); numRowsEstimate = (int)Math.max(1, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala index 0b096ddf781..e8be113c6d8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuBatchScanExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ import scala.math.max import ai.rapids.cudf import ai.rapids.cudf.{HostMemoryBuffer, NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.compress.CompressionCodecFactory @@ -43,7 +42,6 @@ import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, Partit import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DateType, DecimalType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -54,15 +52,20 @@ import org.apache.spark.util.SerializableConfiguration case class GpuBatchScanExec( output: Seq[AttributeReference], @transient scan: Scan) extends DataSourceV2ScanExecBase with GpuExec { - + import GpuMetric._ @transient lazy val batch: Batch = scan.toBatch override def supportsColumnar = true - override lazy val additionalMetrics = GpuMetricNames.buildGpuScanMetrics(sparkContext) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics = Map( + GPU_DECODE_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_DECODE_TIME), + BUFFER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUFFER_TIME), + PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY)) scan match { - case s: ScanWithMetrics => s.metrics = metrics ++ additionalMetrics + case s: ScanWithMetrics => s.metrics = allMetrics ++ additionalMetrics case _ => } @@ -81,7 +84,7 @@ case class GpuBatchScanExec( trait ScanWithMetrics { //this is initialized by the exec post creation - var metrics : Map[String, SQLMetric] = Map.empty + var metrics : Map[String, GpuMetric] = Map.empty } object GpuCSVScan { @@ -305,7 +308,7 @@ case class GpuCSVPartitionReaderFactory( parsedOptions: CSVOptions, maxReaderBatchSizeRows: Integer, maxReaderBatchSizeBytes: Long, - metrics: Map[String, SQLMetric]) extends FilePartitionReaderFactory { + metrics: Map[String, GpuMetric]) extends FilePartitionReaderFactory { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { throw new IllegalStateException("ROW BASED PARSING IS NOT SUPPORTED ON THE GPU...") @@ -328,8 +331,9 @@ class CSVPartitionReader( parsedOptions: CSVOptions, maxRowsPerChunk: Integer, maxBytesPerChunk: Long, - execMetrics: Map[String, SQLMetric]) + execMetrics: Map[String, GpuMetric]) extends PartitionReader[ColumnarBatch] with ScanWithMetrics with Arm { + import GpuMetric._ private var batch: Option[ColumnarBatch] = None private val lineReader = new HadoopFileLinesReader(partFile, parsedOptions.lineSeparatorInRead, @@ -485,7 +489,7 @@ class CSVPartitionReader( override def next(): Boolean = { batch.foreach(_.close()) batch = if (isExhausted) { - metrics("peakDevMemory").set(maxDeviceMemory) + metrics(PEAK_DEVICE_MEMORY).set(maxDeviceMemory) None } else { readBatch() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala index 28e32a4cd03..22ce1e30c59 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -134,13 +133,13 @@ case class TargetSize(override val targetSizeBytes: Long) extends CoalesceGoal { abstract class AbstractGpuCoalesceIterator( iter: Iterator[ColumnarBatch], goal: CoalesceGoal, - numInputRows: SQLMetric, - numInputBatches: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - collectTime: SQLMetric, - concatTime: SQLMetric, - totalTime: SQLMetric, + numInputRows: GpuMetric, + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + collectTime: GpuMetric, + concatTime: GpuMetric, + totalTime: GpuMetric, opName: String) extends Iterator[ColumnarBatch] with Arm with Logging { private var batchInitialized: Boolean = false @@ -343,14 +342,14 @@ class GpuCoalesceIteratorNoSpill(iter: Iterator[ColumnarBatch], schema: StructType, goal: CoalesceGoal, maxDecompressBatchMemory: Long, - numInputRows: SQLMetric, - numInputBatches: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - collectTime: SQLMetric, - concatTime: SQLMetric, - totalTime: SQLMetric, - peakDevMemory: SQLMetric, + numInputRows: GpuMetric, + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + collectTime: GpuMetric, + concatTime: GpuMetric, + totalTime: GpuMetric, + peakDevMemory: GpuMetric, opName: String) extends AbstractGpuCoalesceIterator(iter, goal, @@ -461,14 +460,14 @@ class GpuCoalesceIterator(iter: Iterator[ColumnarBatch], schema: StructType, goal: CoalesceGoal, maxDecompressBatchMemory: Long, - numInputRows: SQLMetric, - numInputBatches: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - collectTime: SQLMetric, - concatTime: SQLMetric, - totalTime: SQLMetric, - peakDevMemory: SQLMetric, + numInputRows: GpuMetric, + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + collectTime: GpuMetric, + concatTime: GpuMetric, + totalTime: GpuMetric, + peakDevMemory: GpuMetric, opName: String) extends AbstractGpuCoalesceIterator(iter, goal, @@ -571,17 +570,19 @@ class GpuCoalesceIterator(iter: Iterator[ColumnarBatch], case class GpuCoalesceBatches(child: SparkPlan, goal: CoalesceGoal) extends UnaryExecNode with GpuExec { + import GpuMetric._ private[this] val maxDecompressBatchMemory = new RapidsConf(child.conf).shuffleCompressionMaxBatchMemory - import GpuMetricNames._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES), - "collectTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "collect batch time"), - "concatTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "concat batch time"), - PEAK_DEVICE_MEMORY -> SQLMetrics.createSizeMetric(sparkContext, DESCRIPTION_PEAK_DEVICE_MEMORY) + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES), + COLLECT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_COLLECT_TIME), + CONCAT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_CONCAT_TIME), + PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY) ) override protected def doExecute(): RDD[InternalRow] = { @@ -595,14 +596,14 @@ case class GpuCoalesceBatches(child: SparkPlan, goal: CoalesceGoal) override def outputBatching: CoalesceGoal = goal override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numInputRows = longMetric(NUM_INPUT_ROWS) - val numInputBatches = longMetric(NUM_INPUT_BATCHES) - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val collectTime = longMetric("collectTime") - val concatTime = longMetric("concatTime") - val totalTime = longMetric(TOTAL_TIME) - val peakDevMemory = longMetric("peakDevMemory") + val numInputRows = gpuLongMetric(NUM_INPUT_ROWS) + val numInputBatches = gpuLongMetric(NUM_INPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val collectTime = gpuLongMetric(COLLECT_TIME) + val concatTime = gpuLongMetric(CONCAT_TIME) + val totalTime = gpuLongMetric(TOTAL_TIME) + val peakDevMemory = gpuLongMetric(PEAK_DEVICE_MEMORY) // cache in local vars to avoid serializing the plan val outputSchema = schema diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala index 9241ab7e778..4795866b807 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarBatchSerializer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -28,7 +28,6 @@ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.TaskContext import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer, SerializerInstance} -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.NullType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -48,14 +47,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * * @note The RAPIDS shuffle does not use this code. */ -class GpuColumnarBatchSerializer(dataSize: SQLMetric = null) extends Serializer with Serializable { +class GpuColumnarBatchSerializer(dataSize: GpuMetric) + extends Serializer with Serializable { override def newInstance(): SerializerInstance = new GpuColumnarBatchSerializerInstance(dataSize) override def supportsRelocationOfSerializedObjects: Boolean = true } -private class GpuColumnarBatchSerializerInstance( - dataSize: SQLMetric) extends SerializerInstance { +private class GpuColumnarBatchSerializerInstance(dataSize: GpuMetric) extends SerializerInstance { override def serializeStream(out: OutputStream): SerializationStream = new SerializationStream { private[this] val dOut: DataOutputStream = @@ -91,9 +90,7 @@ private class GpuColumnarBatchSerializerInstance( } } - if (dataSize != null) { - dataSize.add(JCudfSerialization.getSerializedSizeInBytes(columns, startRow, numRows)) - } + dataSize += JCudfSerialization.getSerializedSizeInBytes(columns, startRow, numRows) val range = new NvtxRange("Serialize Batch", NvtxColor.YELLOW) try { JCudfSerialization.writeToStream(columns, dOut, startRow, numRows) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala index a58525ef2c6..8028525e594 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,8 +18,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable.Queue -import ai.rapids.cudf.{HostColumnVector, NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.GpuMetricNames._ +import ai.rapids.cudf.{HostColumnVector, NvtxColor, Table} import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -27,7 +26,6 @@ import org.apache.spark.sql.catalyst.{CudfUnsafeRow, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.GpuColumnToRowMapPartitionsRDD import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -38,9 +36,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class AcceleratedColumnarToRowIterator( schema: Seq[Attribute], batches: Iterator[ColumnarBatch], - numInputBatches: SQLMetric = null, - numOutputRows: SQLMetric = null, - totalTime: SQLMetric = null) extends Iterator[InternalRow] with Arm with Serializable { + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + totalTime: GpuMetric) extends Iterator[InternalRow] with Arm with Serializable { @transient private var pendingCvs: Queue[HostColumnVector] = Queue.empty // GPU batches read in must be closed by the receiver (us) @transient private var currentCv: Option[HostColumnVector] = None @@ -97,23 +95,14 @@ class AcceleratedColumnarToRowIterator( } private[this] def setupBatch(cb: ColumnarBatch): Boolean = { - if (numInputBatches != null) { - numInputBatches += 1 - } + numInputBatches += 1 // In order to match the numOutputRows metric in the generated code we update // numOutputRows for each batch. This is less accurate than doing it at output // because it will over count the number of rows output in the case of a limit, // but it is more efficient. - if (numOutputRows != null) { - numOutputRows += cb.numRows() - } + numOutputRows += cb.numRows() if (cb.numRows() > 0) { - val nvtxRange = if (totalTime != null) { - new NvtxWithMetrics("ColumnarToRow: batch", NvtxColor.RED, totalTime) - } else { - new NvtxRange("ColumnarToRow: batch", NvtxColor.RED) - } - withResource(nvtxRange) { _ => + withResource(new NvtxWithMetrics("ColumnarToRow: batch", NvtxColor.RED, totalTime)) { _ => withResource(rearrangeRows(cb)) { table => withResource(table.convertToRows()) { rowsCvList => rowsCvList.foreach { rowsCv => @@ -168,8 +157,10 @@ class AcceleratedColumnarToRowIterator( } } -class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], numInputBatches: SQLMetric = null, - numOutputRows: SQLMetric = null, totalTime: SQLMetric = null) extends Iterator[InternalRow] { +class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + totalTime: GpuMetric) extends Iterator[InternalRow] { // GPU batches read in must be closed by the receiver (us) @transient var cb: ColumnarBatch = null var it: java.util.Iterator[InternalRow] = null @@ -190,26 +181,17 @@ class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], numInputBatches: S } if (batches.hasNext) { val devCb = batches.next() - val nvtxRange = if (totalTime != null) { - new NvtxWithMetrics("ColumnarToRow: batch", NvtxColor.RED, totalTime) - } else { - new NvtxRange("ColumnarToRow: batch", NvtxColor.RED) - } - + val nvtxRange = new NvtxWithMetrics("ColumnarToRow: batch", NvtxColor.RED, totalTime) try { cb = new ColumnarBatch(GpuColumnVector.extractColumns(devCb).map(_.copyToHost()), devCb.numRows()) it = cb.rowIterator() - if (numInputBatches != null) { - numInputBatches += 1 - } + numInputBatches += 1 // In order to match the numOutputRows metric in the generated code we update // numOutputRows for each batch. This is less accurate than doing it at output // because it will over count the number of rows output in the case of a limit, // but it is more efficient. - if (numOutputRows != null) { - numOutputRows += cb.numRows() - } + numOutputRows += cb.numRows() } finally { devCb.close() // Leaving the GPU for a while @@ -254,6 +236,7 @@ object CudfRowTransitions { abstract class GpuColumnarToRowExecParent(child: SparkPlan, val exportColumnarRdd: Boolean) extends UnaryExecNode with GpuExec { + import GpuMetric._ // We need to do this so the assertions don't fail override def supportsColumnar = false @@ -264,15 +247,15 @@ abstract class GpuColumnarToRowExecParent(child: SparkPlan, val exportColumnarRd override def outputOrdering: Seq[SortOrder] = child.outputOrdering // Override the original metrics to remove NUM_OUTPUT_BATCHES, which makes no sense. - override lazy val metrics: Map[String, SQLMetric] = Map( - NUM_OUTPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_ROWS), - TOTAL_TIME -> SQLMetrics.createNanoTimingMetric(sparkContext, DESCRIPTION_TOTAL_TIME), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES)) + override lazy val allMetrics: Map[String, GpuMetric] = Map( + NUM_OUTPUT_ROWS -> createMetric(outputRowsLevel, DESCRIPTION_NUM_OUTPUT_ROWS), + TOTAL_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_TOTAL_TIME), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES)) override def doExecute(): RDD[InternalRow] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numInputBatches = longMetric(NUM_INPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numInputBatches = gpuLongMetric(NUM_INPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) // This avoids calling `output` in the RDD closure, so that we don't need to include the entire // plan (this) in the closure. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala index b8477a0c7de..b213722bedb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuDataWritingCommandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019,2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -55,7 +55,7 @@ trait GpuDataWritingCommand extends DataWritingCommand { case class GpuDataWritingCommandExec(cmd: GpuDataWritingCommand, child: SparkPlan) extends UnaryExecNode with GpuExec { - override lazy val metrics: Map[String, SQLMetric] = cmd.metrics + override lazy val allMetrics: Map[String, GpuMetric] = GpuMetric.wrap(cmd.metrics) private lazy val sideEffectResult: Seq[ColumnarBatch] = cmd.runColumnar(sqlContext.sparkSession, child) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala index c6bac2f3de8..9b333e2f649 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,16 +16,26 @@ package com.nvidia.spark.rapids -import com.nvidia.spark.rapids.GpuMetricNames._ - import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -object GpuMetricNames { +sealed class MetricsLevel(val num: Integer) extends Serializable { + def >=(other: MetricsLevel): Boolean = + num >= other.num +} +object MetricsLevel { + def apply(str: String): MetricsLevel = str match { + case "ESSENTIAL" => GpuMetric.ESSENTIAL_LEVEL + case "MODERATE" => GpuMetric.MODERATE_LEVEL + case _ => GpuMetric.DEBUG_LEVEL + } +} + +object GpuMetric { // Metric names. val BUFFER_TIME = "bufferTime" val GPU_DECODE_TIME = "gpuDecodeTime" @@ -37,25 +47,83 @@ object GpuMetricNames { val NUM_PARTITIONS = "numPartitions" val TOTAL_TIME = "totalTime" val PEAK_DEVICE_MEMORY = "peakDevMemory" + val COLLECT_TIME = "collectTime" + val CONCAT_TIME = "concatTime" + val SORT_TIME = "sortTime" + val AGG_TIME = "computeAggTime" + val JOIN_TIME = "joinTime" + val JOIN_OUTPUT_ROWS = "joinOutputRows" + val FILTER_TIME = "filterTime" + val BUILD_DATA_SIZE = "buildDataSize" + val BUILD_TIME = "buildTime" + val STREAM_TIME = "streamTime" // Metric Descriptions. - val DESCRIPTION_NUM_INPUT_ROWS = "number of input rows" - val DESCRIPTION_NUM_INPUT_BATCHES = "number of input columnar batches" - val DESCRIPTION_NUM_OUTPUT_ROWS = "number of output rows" - val DESCRIPTION_NUM_OUTPUT_BATCHES = "number of output columnar batches" + val DESCRIPTION_BUFFER_TIME = "buffer time" + val DESCRIPTION_GPU_DECODE_TIME = "GPU decode time" + val DESCRIPTION_NUM_INPUT_ROWS = "input rows" + val DESCRIPTION_NUM_INPUT_BATCHES = "input columnar batches" + val DESCRIPTION_NUM_OUTPUT_ROWS = "output rows" + val DESCRIPTION_NUM_OUTPUT_BATCHES = "output columnar batches" val DESCRIPTION_PARTITION_SIZE = "partition data size" - val DESCRIPTION_NUM_PARTITIONS = "number of partitions" + val DESCRIPTION_NUM_PARTITIONS = "partitions" val DESCRIPTION_TOTAL_TIME = "total time" val DESCRIPTION_PEAK_DEVICE_MEMORY = "peak device memory" - - def buildGpuScanMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = { - Map( - NUM_OUTPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_BATCHES), - GPU_DECODE_TIME -> SQLMetrics.createNanoTimingMetric(sparkContext, "GPU decode time"), - BUFFER_TIME -> SQLMetrics.createNanoTimingMetric(sparkContext, "buffer time"), - PEAK_DEVICE_MEMORY -> SQLMetrics.createSizeMetric(sparkContext, - DESCRIPTION_PEAK_DEVICE_MEMORY)) + val DESCRIPTION_COLLECT_TIME = "collect batch time" + val DESCRIPTION_CONCAT_TIME = "concat batch time" + val DESCRIPTION_SORT_TIME = "sort time" + val DESCRIPTION_AGG_TIME = "aggregation time" + val DESCRIPTION_JOIN_TIME = "join time" + val DESCRIPTION_JOIN_OUTPUT_ROWS = "join output rows" + val DESCRIPTION_FILTER_TIME = "filter time" + val DESCRIPTION_BUILD_DATA_SIZE = "build side size" + val DESCRIPTION_BUILD_TIME = "build time" + val DESCRIPTION_STREAM_TIME = "stream time" + + def unwrap(input: GpuMetric): SQLMetric = input match { + case w :WrappedGpuMetric => w.sqlMetric + case i => throw new IllegalArgumentException(s"found unsupported GpuMetric ${i.getClass}") } + + def unwrap(input: Map[String, GpuMetric]): Map[String, SQLMetric] = input.filter { + // remove the metrics that are not registered + case (_, NoopMetric) => false + case _ => true + // sadly mapValues produces a non-serializable result, so we have to hack it a bit to force + // it to be materialized + }.mapValues(unwrap).toArray.toMap + + def wrap(input: SQLMetric): GpuMetric = WrappedGpuMetric(input) + + def wrap(input: Map[String, SQLMetric]): Map[String, GpuMetric] = + // sadly mapValues produces a non-serializable result, so we have to hack it a bit to force + // it to be materialized + input.mapValues(wrap).toArray.toMap + + object DEBUG_LEVEL extends MetricsLevel(0) + object MODERATE_LEVEL extends MetricsLevel(1) + object ESSENTIAL_LEVEL extends MetricsLevel(2) +} + +sealed abstract class GpuMetric extends Serializable { + def value: Long + def set(v: Long): Unit + def +=(v: Long): Unit + def add(v: Long): Unit +} + +object NoopMetric extends GpuMetric { + override def +=(v: Long): Unit = () + override def add(v: Long): Unit = () + override def set(v: Long): Unit = () + override def value: Long = 0 +} + +case class WrappedGpuMetric(sqlMetric: SQLMetric) extends GpuMetric { + def +=(v: Long): Unit = sqlMetric.add(v) + def add(v: Long): Unit = sqlMetric.add(v) + override def set(v: Long): Unit = sqlMetric.set(v) + override def value: Long = sqlMetric.value } object GpuExec { @@ -66,6 +134,7 @@ object GpuExec { } trait GpuExec extends SparkPlan with Arm { + import GpuMetric._ /** * If true is returned batches after this will be coalesced. This should * really be used in cases where it is known that the size of a batch may @@ -88,15 +157,47 @@ trait GpuExec extends SparkPlan with Arm { */ def outputBatching: CoalesceGoal = null + private [this] lazy val metricsConf = MetricsLevel(RapidsConf.METRICS_LEVEL.get(conf)) + + private [this] def createMetricInternal(level: MetricsLevel, f: => SQLMetric): GpuMetric = { + if (level >= metricsConf) { + WrappedGpuMetric(f) + } else { + NoopMetric + } + } + + protected def createMetric(level: MetricsLevel, name: String): GpuMetric = + createMetricInternal(level, SQLMetrics.createMetric(sparkContext, name)) + + protected def createNanoTimingMetric(level: MetricsLevel, name: String): GpuMetric = + createMetricInternal(level, SQLMetrics.createNanoTimingMetric(sparkContext, name)) + + protected def createSizeMetric(level: MetricsLevel, name: String): GpuMetric = + createMetricInternal(level, SQLMetrics.createSizeMetric(sparkContext, name)) + + protected def createAverageMetric(level: MetricsLevel, name: String): GpuMetric = + createMetricInternal(level, SQLMetrics.createAverageMetric(sparkContext, name)) + + protected def createTimingMetric(level: MetricsLevel, name: String): GpuMetric = + createMetricInternal(level, SQLMetrics.createTimingMetric(sparkContext, name)) + override def supportsColumnar = true - override lazy val metrics: Map[String, SQLMetric] = Map( - NUM_OUTPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_ROWS), - NUM_OUTPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_BATCHES), - TOTAL_TIME -> SQLMetrics.createNanoTimingMetric(sparkContext, - DESCRIPTION_TOTAL_TIME)) ++ additionalMetrics + protected val outputRowsLevel: MetricsLevel = DEBUG_LEVEL + protected val outputBatchesLevel: MetricsLevel = DEBUG_LEVEL + + lazy val allMetrics: Map[String, GpuMetric] = Map( + NUM_OUTPUT_ROWS -> createMetric(outputRowsLevel, DESCRIPTION_NUM_OUTPUT_ROWS), + NUM_OUTPUT_BATCHES -> createMetric(outputBatchesLevel, DESCRIPTION_NUM_OUTPUT_BATCHES), + TOTAL_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_TOTAL_TIME)) ++ + additionalMetrics + + def gpuLongMetric(name: String): GpuMetric = allMetrics(name) + + final override lazy val metrics: Map[String, SQLMetric] = unwrap(allMetrics) - lazy val additionalMetrics: Map[String, SQLMetric] = Map.empty + lazy val additionalMetrics: Map[String, GpuMetric] = Map.empty /** * Returns true if there is something in the exec that cannot work when batches between diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala index 19c09923715..1a9d30292f6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpandExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids import scala.collection.mutable import ai.rapids.cudf.{NvtxColor, Scalar} -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.TaskContext @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.{ExpandExec, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -59,24 +58,24 @@ class GpuExpandExecMeta( /** * Apply all of the GroupExpressions to every input row, hence we will get * multiple output rows for an input row. + * * @param projections The group of expressions, all of the group expressions should * output the same schema specified bye the parameter `output` - * @param output Attribute references to Output + * @param output Attribute references to Output * @param child Child operator */ case class GpuExpandExec( projections: Seq[Seq[Expression]], output: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode with GpuExec { + extends UnaryExecNode with GpuExec { - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, - DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, - DESCRIPTION_NUM_INPUT_BATCHES), - PEAK_DEVICE_MEMORY -> SQLMetrics.createSizeMetric(sparkContext, - DESCRIPTION_PEAK_DEVICE_MEMORY)) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES), + PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY)) // The GroupExpressions can output data with arbitrary partitioning, so set it // as UNKNOWN partitioning @@ -91,7 +90,7 @@ case class GpuExpandExec( projections.map(GpuBindReferences.bindGpuReferences(_, child.output)) // cache in a local to avoid serializing the plan - val metricsMap = metrics + val metricsMap = allMetrics child.executeColumnar().mapPartitions { it => new GpuExpandIterator(boundProjections, metricsMap, it) @@ -106,7 +105,7 @@ case class GpuExpandExec( class GpuExpandIterator( boundProjections: Seq[Seq[GpuExpression]], - metrics: Map[String, SQLMetric], + metrics: Map[String, GpuMetric], it: Iterator[ColumnarBatch]) extends Iterator[ColumnarBatch] with Arm { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala index 5598048938e..439993d7b21 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuGenerateExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ColumnVector, NvtxColor, Table} -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetric.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.TaskContext @@ -116,9 +116,9 @@ case class GpuGenerateExec( throw new IllegalStateException(s"Row-based execution should not occur for $this") override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) val boundArrayProjectList = GpuBindReferences.bindGpuReferences(arrayProject, child.output).toArray val numArrayColumns = boundArrayProjectList.length diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index cdb14c06aa4..ef24236b860 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,7 +30,7 @@ import scala.math.max import ai.rapids.cudf._ import com.google.protobuf.CodedOutputStream -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.GpuOrcPartitionReader.{OrcOutputStripe, OrcPartitionReaderContext} import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration @@ -54,7 +54,6 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory} import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.OrcFilters import org.apache.spark.sql.sources.Filter @@ -130,7 +129,7 @@ case class GpuOrcPartitionReaderFactory( partitionSchema: StructType, pushedFilters: Array[Filter], @transient rapidsConf: RapidsConf, - metrics : Map[String, SQLMetric]) extends FilePartitionReaderFactory with Arm { + metrics : Map[String, GpuMetric]) extends FilePartitionReaderFactory with Arm { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val debugDumpPrefix = rapidsConf.orcDebugDumpPrefix private val maxReadBatchSizeRows: Integer = rapidsConf.maxReadBatchSizeRows @@ -388,7 +387,7 @@ class GpuOrcPartitionReader( debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, - execMetrics : Map[String, SQLMetric]) extends PartitionReader[ColumnarBatch] with Logging + execMetrics : Map[String, GpuMetric]) extends PartitionReader[ColumnarBatch] with Logging with ScanWithMetrics with Arm { private var batch: Option[ColumnarBatch] = None private var maxDeviceMemory: Long = 0 @@ -415,7 +414,7 @@ class GpuOrcPartitionReader( if (ctx.blockIterator.hasNext) { batch = readBatch() } else { - metrics("peakDevMemory") += maxDeviceMemory + metrics(PEAK_DEVICE_MEMORY) += maxDeviceMemory } // This is odd, but some operators return data even when there is no input so we need to // be sure that we grab the GPU diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index 0bb577f4e2c..bd65d862d16 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -31,7 +31,7 @@ import scala.math.max import ai.rapids.cudf._ import com.google.common.util.concurrent.ThreadFactoryBuilder import com.nvidia.spark.RebaseHelper -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.ParquetPartitionReader.CopyRange import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.commons.io.IOUtils @@ -57,7 +57,6 @@ import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFil import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, ParquetReadSupport} import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.rapids.InputFileUtils import org.apache.spark.sql.rapids.execution.TrampolineUtil @@ -359,7 +358,7 @@ case class GpuParquetMultiFilePartitionReaderFactory( partitionSchema: StructType, filters: Array[Filter], @transient rapidsConf: RapidsConf, - metrics: Map[String, SQLMetric], + metrics: Map[String, GpuMetric], queryUsesInputFile: Boolean) extends PartitionReaderFactory with Arm with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix @@ -468,7 +467,7 @@ case class GpuParquetPartitionReaderFactory( partitionSchema: StructType, filters: Array[Filter], @transient rapidsConf: RapidsConf, - metrics: Map[String, SQLMetric]) extends FilePartitionReaderFactory with Arm with Logging { + metrics: Map[String, GpuMetric]) extends FilePartitionReaderFactory with Arm with Logging { private val isCaseSensitive = sqlConf.caseSensitiveAnalysis private val debugDumpPrefix = rapidsConf.parquetDebugDumpPrefix private val maxReadBatchSizeRows = rapidsConf.maxReadBatchSizeRows @@ -507,7 +506,7 @@ abstract class FileParquetPartitionReaderBase( isSchemaCaseSensitive: Boolean, readDataSchema: StructType, debugDumpPrefix: String, - execMetrics: Map[String, SQLMetric]) extends PartitionReader[ColumnarBatch] with Logging + execMetrics: Map[String, GpuMetric]) extends PartitionReader[ColumnarBatch] with Logging with ScanWithMetrics with Arm { protected var isDone: Boolean = false @@ -886,7 +885,7 @@ class MultiFileParquetPartitionReader( debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, - execMetrics: Map[String, SQLMetric], + execMetrics: Map[String, GpuMetric], partitionSchema: StructType, numThreads: Int) extends FileParquetPartitionReaderBase(conf, isSchemaCaseSensitive, readDataSchema, @@ -922,7 +921,7 @@ class MultiFileParquetPartitionReader( if (!isDone) { if (!blockIterator.hasNext) { isDone = true - metrics("peakDevMemory") += maxDeviceMemory + metrics(PEAK_DEVICE_MEMORY) += maxDeviceMemory } else { batch = readBatch() } @@ -1304,7 +1303,7 @@ class MultiFileCloudParquetPartitionReader( debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, - execMetrics: Map[String, SQLMetric], + execMetrics: Map[String, GpuMetric], partitionSchema: StructType, numThreads: Int, maxNumFileProcessed: Int, @@ -1484,7 +1483,7 @@ class MultiFileCloudParquetPartitionReader( } } else { isDone = true - metrics("peakDevMemory") += maxDeviceMemory + metrics(PEAK_DEVICE_MEMORY) += maxDeviceMemory } } } @@ -1625,7 +1624,7 @@ class ParquetPartitionReader( debugDumpPrefix: String, maxReadBatchSizeRows: Integer, maxReadBatchSizeBytes: Long, - execMetrics: Map[String, SQLMetric], + execMetrics: Map[String, GpuMetric], isCorrectedRebaseMode: Boolean) extends FileParquetPartitionReaderBase(conf, isSchemaCaseSensitive, readDataSchema, debugDumpPrefix, execMetrics) { @@ -1638,7 +1637,7 @@ class ParquetPartitionReader( if (!isDone) { if (!blockIterator.hasNext) { isDone = true - metrics("peakDevMemory") += maxDeviceMemory + metrics(PEAK_DEVICE_MEMORY) += maxDeviceMemory } else { batch = readBatch() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala index 9a34055b244..5757bcc3d0a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -41,7 +40,7 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, SQLMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala index 0d0dd20d6a8..7c7d9556c68 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -25,7 +25,6 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType @@ -60,5 +59,5 @@ trait GpuReadFileFormatWithMetrics extends FileFormat { filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, SQLMetric]): PartitionedFile => Iterator[InternalRow] + metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index a8450079afc..be0c58f092d 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -40,7 +39,7 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, SQLMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index 0d0ea9a05f6..7d6893cf124 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -40,7 +39,7 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW filters: Seq[Filter], options: Map[String, String], hadoopConf: Configuration, - metrics: Map[String, SQLMetric]): PartitionedFile => Iterator[InternalRow] = { + metrics: Map[String, GpuMetric]): PartitionedFile => Iterator[InternalRow] = { val sqlConf = sparkSession.sessionState.conf val broadcastedHadoopConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala index 5861c08df31..c4f7dba6ed6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRowToColumnarExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,9 +16,8 @@ package com.nvidia.spark.rapids -import ai.rapids.cudf.{NvtxColor, NvtxRange} +import ai.rapids.cudf.NvtxColor import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder -import com.nvidia.spark.rapids.GpuMetricNames._ import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast @@ -29,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, Speciali import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodegenContext, CodeGenerator} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch @@ -529,10 +527,10 @@ class RowToColumnarIterator( localSchema: StructType, localGoal: CoalesceGoal, converters: GpuRowToColumnConverter, - totalTime: SQLMetric = null, - numInputRows: SQLMetric = null, - numOutputRows: SQLMetric = null, - numOutputBatches: SQLMetric = null) extends Iterator[ColumnarBatch] { + totalTime: GpuMetric = NoopMetric, + numInputRows: GpuMetric = NoopMetric, + numOutputRows: GpuMetric = NoopMetric, + numOutputBatches: GpuMetric = NoopMetric) extends Iterator[ColumnarBatch] with Arm { private val targetSizeBytes = localGoal.targetSizeBytes private var targetRows = 0 @@ -587,26 +585,12 @@ class RowToColumnarIterator( // option here Option(TaskContext.get()).foreach(GpuSemaphore.acquireIfNecessary) - var buildRange: NvtxRange = null - if (totalTime != null) { - buildRange = new NvtxWithMetrics("RowToColumnar", NvtxColor.GREEN, totalTime) - } else { - buildRange = new NvtxRange("RowToColumnar", NvtxColor.GREEN) - } - val ret = try { + val ret = withResource(new NvtxWithMetrics("RowToColumnar", NvtxColor.GREEN, totalTime)) { _=> builders.build(rowCount) - } finally { - buildRange.close() - } - if (numInputRows != null) { - numInputRows += rowCount - } - if (numOutputRows != null) { - numOutputRows += rowCount - } - if (numOutputBatches != null) { - numOutputBatches += 1 } + numInputRows += rowCount + numOutputRows += rowCount + numOutputBatches += 1 // refine the targetRows estimate based on the average of all batches processed so far totalOutputBytes += GpuColumnVector.getTotalDeviceMemoryUsed(ret) @@ -628,19 +612,19 @@ object GeneratedUnsafeRowToCudfRowIterator extends Logging { def apply(input: Iterator[UnsafeRow], schema: Array[Attribute], goal: CoalesceGoal, - totalTime: SQLMetric, - numInputRows: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric): UnsafeRowToColumnarBatchIterator = { + totalTime: GpuMetric, + numInputRows: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric): UnsafeRowToColumnarBatchIterator = { val ctx = new CodegenContext ctx.addReferenceObj("iter", input, classOf[Iterator[UnsafeRow]].getName) ctx.addReferenceObj("schema", schema, classOf[Array[Attribute]].getName) ctx.addReferenceObj("goal", goal, classOf[CoalesceGoal].getName) - ctx.addReferenceObj("totalTime", totalTime, classOf[SQLMetric].getName) - ctx.addReferenceObj("numInputRows", numInputRows, classOf[SQLMetric].getName) - ctx.addReferenceObj("numOutputRows", numOutputRows, classOf[SQLMetric].getName) - ctx.addReferenceObj("numOutputBatches", numOutputBatches, classOf[SQLMetric].getName) + ctx.addReferenceObj("totalTime", totalTime, classOf[GpuMetric].getName) + ctx.addReferenceObj("numInputRows", numInputRows, classOf[GpuMetric].getName) + ctx.addReferenceObj("numOutputRows", numOutputRows, classOf[GpuMetric].getName) + ctx.addReferenceObj("numOutputBatches", numOutputBatches, classOf[GpuMetric].getName) val rowBaseObj = ctx.freshName("rowBaseObj") val rowBaseOffset = ctx.freshName("rowBaseOffset") @@ -705,10 +689,10 @@ object GeneratedUnsafeRowToCudfRowIterator extends Logging { | super((scala.collection.Iterator)references[0], | (org.apache.spark.sql.catalyst.expressions.Attribute[])references[1], | (com.nvidia.spark.rapids.CoalesceGoal)references[2], - | (org.apache.spark.sql.execution.metric.SQLMetric)references[3], - | (org.apache.spark.sql.execution.metric.SQLMetric)references[4], - | (org.apache.spark.sql.execution.metric.SQLMetric)references[5], - | (org.apache.spark.sql.execution.metric.SQLMetric)references[6]); + | (com.nvidia.spark.rapids.GpuMetric)references[3], + | (com.nvidia.spark.rapids.GpuMetric)references[4], + | (com.nvidia.spark.rapids.GpuMetric)references[5], + | (com.nvidia.spark.rapids.GpuMetric)references[6]); | ${ctx.initMutableStates()} | } | @@ -780,6 +764,7 @@ object GeneratedUnsafeRowToCudfRowIterator extends Logging { */ case class GpuRowToColumnarExec(child: SparkPlan, goal: CoalesceGoal) extends UnaryExecNode with GpuExec { + import GpuMetric._ override def output: Seq[Attribute] = child.output @@ -797,17 +782,17 @@ case class GpuRowToColumnarExec(child: SparkPlan, goal: CoalesceGoal) TrampolineUtil.doExecuteBroadcast(child) } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS) + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS) ) override def doExecuteColumnar(): RDD[ColumnarBatch] = { // use local variables instead of class global variables to prevent the entire // object from having to be serialized - val numInputRows = longMetric(NUM_INPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val totalTime = longMetric(TOTAL_TIME) + val numInputRows = gpuLongMetric(NUM_INPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val totalTime = gpuLongMetric(TOTAL_TIME) val localGoal = goal val rowBased = child.execute() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala index 7e82af95e56..ac873ea1d31 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffleCoalesceExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -41,12 +40,15 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long) extends UnaryExecNode with GpuExec { - import GpuMetricNames._ - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES), - "collectTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "collect batch time"), - "concatTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "concat batch time") + import GpuMetric._ + + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES), + COLLECT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_COLLECT_TIME), + CONCAT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_CONCAT_TIME) ) override def output: Seq[Attribute] = child.output @@ -58,7 +60,7 @@ case class GpuShuffleCoalesceExec(child: SparkPlan, targetBatchByteSize: Long) } override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val metricsMap = metrics + val metricsMap = allMetrics val targetSize = targetBatchByteSize val sparkSchema = GpuColumnVector.extractTypes(schema) @@ -78,13 +80,13 @@ class GpuShuffleCoalesceIterator( batchIter: Iterator[ColumnarBatch], targetBatchByteSize: Long, sparkSchema: Array[DataType], - metricsMap: Map[String, SQLMetric]) + metricsMap: Map[String, GpuMetric]) extends Iterator[ColumnarBatch] with Arm with AutoCloseable { - private[this] val totalTimeMetric = metricsMap(GpuMetricNames.TOTAL_TIME) - private[this] val inputBatchesMetric = metricsMap(GpuMetricNames.NUM_INPUT_BATCHES) - private[this] val inputRowsMetric = metricsMap(GpuMetricNames.NUM_INPUT_ROWS) - private[this] val outputBatchesMetric = metricsMap(GpuMetricNames.NUM_OUTPUT_BATCHES) - private[this] val outputRowsMetric = metricsMap(GpuMetricNames.NUM_OUTPUT_ROWS) + private[this] val totalTimeMetric = metricsMap(GpuMetric.TOTAL_TIME) + private[this] val inputBatchesMetric = metricsMap(GpuMetric.NUM_INPUT_BATCHES) + private[this] val inputRowsMetric = metricsMap(GpuMetric.NUM_INPUT_ROWS) + private[this] val outputBatchesMetric = metricsMap(GpuMetric.NUM_OUTPUT_BATCHES) + private[this] val outputRowsMetric = metricsMap(GpuMetric.NUM_OUTPUT_ROWS) private[this] val collectTimeMetric = metricsMap("collectTime") private[this] val concatTimeMetric = metricsMap("concatTime") private[this] val serializedTables = new util.ArrayDeque[SerializedTableColumn] diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala index 54a86af6f4b..ba4300ab236 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuSortExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf import ai.rapids.cudf.{NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, NullsFirst, NullsLast, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{SortExec, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -79,13 +78,12 @@ case class GpuSortExec( throw new IllegalStateException(s"Row-based execution should not occur for $this") override lazy val additionalMetrics = Map( - "sortTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "sort time"), - "peakDevMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak device memory"), - "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) + SORT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_SORT_TIME), + PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY)) override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val sortTime = longMetric("sortTime") - val peakDevMemory = longMetric("peakDevMemory") + val sortTime = gpuLongMetric(SORT_TIME) + val peakDevMemory = gpuLongMetric(PEAK_DEVICE_MEMORY) val crdd = child.executeColumnar() crdd.mapPartitions { cbIter => @@ -113,12 +111,12 @@ class GpuColumnarBatchSorter( private var maxDeviceMemory = 0L private var haveSortedBatch = false private val numSortCols = sortOrder.length - private val totalTimeMetric : Option[SQLMetric] = initMetric(TOTAL_TIME) - private val outputBatchesMetric : Option[SQLMetric] = initMetric(NUM_OUTPUT_BATCHES) - private val outputRowsMetric : Option[SQLMetric] = initMetric(NUM_OUTPUT_ROWS) + private val totalTimeMetric : Option[GpuMetric] = initMetric(TOTAL_TIME) + private val outputBatchesMetric : Option[GpuMetric] = initMetric(NUM_OUTPUT_BATCHES) + private val outputRowsMetric : Option[GpuMetric] = initMetric(NUM_OUTPUT_ROWS) - private def initMetric(metricName: String): Option[SQLMetric] = if (shouldUpdateMetrics) { - Some(exec.longMetric(metricName)) + private def initMetric(metricName: String): Option[GpuMetric] = if (shouldUpdateMetrics) { + Some(exec.gpuLongMetric(metricName)) } else { None } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index 31b7138e626..e644c22ce1b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -130,9 +130,9 @@ case class GpuWindowExec( throw new IllegalStateException(s"Row-based execution should not happen, in $this.") override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputBatches = metrics(GpuMetricNames.NUM_OUTPUT_BATCHES) - val numOutputRows = metrics(GpuMetricNames.NUM_OUTPUT_ROWS) - val totalTime = metrics(GpuMetricNames.TOTAL_TIME) + val numOutputBatches = gpuLongMetric(GpuMetric.NUM_OUTPUT_BATCHES) + val numOutputRows = gpuLongMetric(GpuMetric.NUM_OUTPUT_ROWS) + val totalTime = gpuLongMetric(GpuMetric.TOTAL_TIME) val projectList = if (resultColumnsOnly) { windowExpressionAliases diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala index 85f9ccf20ae..eaff871af99 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostColumnarToGpu.scala @@ -26,7 +26,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.sql.vectorized.rapids.AccessibleArrowColumnVector @@ -230,14 +229,14 @@ object HostColumnarToGpu extends Logging { class HostToGpuCoalesceIterator(iter: Iterator[ColumnarBatch], goal: CoalesceGoal, schema: StructType, - numInputRows: SQLMetric, - numInputBatches: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - collectTime: SQLMetric, - concatTime: SQLMetric, - totalTime: SQLMetric, - peakDevMemory: SQLMetric, + numInputRows: GpuMetric, + numInputBatches: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + collectTime: GpuMetric, + concatTime: GpuMetric, + totalTime: GpuMetric, + peakDevMemory: GpuMetric, opName: String, useArrowCopyOpt: Boolean) extends AbstractGpuCoalesceIterator(iter, @@ -353,17 +352,15 @@ class HostToGpuCoalesceIterator(iter: Iterator[ColumnarBatch], case class HostColumnarToGpu(child: SparkPlan, goal: CoalesceGoal) extends UnaryExecNode with GpuExec { - import GpuMetricNames._ - - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> - SQLMetrics.createMetric(sparkContext, GpuMetricNames.DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> - SQLMetrics.createMetric(sparkContext, GpuMetricNames.DESCRIPTION_NUM_INPUT_BATCHES), - "collectTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "collect batch time"), - "concatTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "concat batch time"), - "peakDevMemory" -> - SQLMetrics.createMetric(sparkContext, GpuMetricNames.DESCRIPTION_PEAK_DEVICE_MEMORY) + import GpuMetric._ + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES), + COLLECT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_COLLECT_TIME), + CONCAT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_CONCAT_TIME), + PEAK_DEVICE_MEMORY -> createMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY) ) override def output: Seq[Attribute] = child.output @@ -387,14 +384,14 @@ case class HostColumnarToGpu(child: SparkPlan, goal: CoalesceGoal) */ override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numInputRows = longMetric(NUM_INPUT_ROWS) - val numInputBatches = longMetric(NUM_INPUT_BATCHES) - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val collectTime = longMetric("collectTime") - val concatTime = longMetric("concatTime") - val totalTime = longMetric(TOTAL_TIME) - val peakDevMemory = longMetric("peakDevMemory") + val numInputRows = gpuLongMetric(NUM_INPUT_ROWS) + val numInputBatches = gpuLongMetric(NUM_INPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val collectTime = gpuLongMetric(COLLECT_TIME) + val concatTime = gpuLongMetric(CONCAT_TIME) + val totalTime = gpuLongMetric(TOTAL_TIME) + val peakDevMemory = gpuLongMetric(PEAK_DEVICE_MEMORY) // cache in a local to avoid serializing the plan val outputSchema = schema diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala index ea942d1339f..633ce75bd41 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/NvtxWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,13 +18,11 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{NvtxColor, NvtxRange} -import org.apache.spark.sql.execution.metric.SQLMetric - /** * NvtxRange with option to pass one or more nano timing metric(s) that are updated upon close * by the amount of time spent in the range */ -class NvtxWithMetrics(name: String, color: NvtxColor, val metric: SQLMetric) +class NvtxWithMetrics(name: String, color: NvtxColor, val metric: GpuMetric) extends NvtxRange(name, color) { private val start = System.nanoTime() @@ -35,7 +33,7 @@ class NvtxWithMetrics(name: String, color: NvtxColor, val metric: SQLMetric) } } -class MetricRange(val metric: SQLMetric) extends AutoCloseable { +class MetricRange(val metric: GpuMetric) extends AutoCloseable { private val start = System.nanoTime() override def close(): Unit = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index c80ba3fb59e..b37f755494c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -117,6 +117,7 @@ abstract class ConfEntry[T](val key: String, val converter: String => T, val doc: String, val isInternal: Boolean) { def get(conf: Map[String, String]): T + def get(conf: SQLConf): T def help(asTable: Boolean = false): Unit override def toString: String = key @@ -130,6 +131,15 @@ class ConfEntryWithDefault[T](key: String, converter: String => T, doc: String, conf.get(key).map(converter).getOrElse(defaultValue) } + override def get(conf: SQLConf): T = { + val tmp = conf.getConfString(key, null) + if (tmp == null) { + defaultValue + } else { + converter(tmp) + } + } + override def help(asTable: Boolean = false): Unit = { if (!isInternal) { if (asTable) { @@ -153,6 +163,15 @@ class OptionalConfEntry[T](key: String, val rawConverter: String => T, doc: Stri conf.get(key).map(rawConverter) } + override def get(conf: SQLConf): Option[T] = { + val tmp = conf.getConfString(key, null) + if (tmp == null) { + None + } else { + Some(rawConverter(tmp)) + } + } + override def help(asTable: Boolean = false): Unit = { if (!isInternal) { if (asTable) { @@ -410,6 +429,23 @@ object RapidsConf { .booleanConf .createWithDefault(false) + // METRICS + + val METRICS_LEVEL = conf("spark.rapids.sql.metrics.level") + .doc("GPU plans can produce a lot more metrics than CPU plans do. In very large " + + "queries this can sometimes result in going over the max result size limit for the " + + "driver. Supported values include " + + "DEBUG which will enable all metrics supported and typically only needs to be enabled " + + "when debugging the plugin. " + + "MODERATE which should output enough metrics to understand how long each part of the " + + "query is taking and how much data is going to each part of the query. " + + "ESSENTIAL which disables most metrics except those Apache Spark CPU plans will also " + + "report or their equivalents.") + .stringConf + .transform(_.toUpperCase(java.util.Locale.ROOT)) + .checkValues(Set("DEBUG", "MODERATE", "ESSENTIAL")) + .createWithDefault("MODERATE") + // ENABLE/DISABLE PROCESSING val IMPROVED_TIMESTAMP_OPS = @@ -992,6 +1028,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val rapidsConfMap: util.Map[String, String] = conf.filterKeys( _.startsWith("spark.rapids.")).asJava + lazy val metricsLevel: String = get(METRICS_LEVEL) + lazy val isSqlEnabled: Boolean = get(SQL_ENABLED) lazy val isUdfCompilerEnabled: Boolean = get(UDF_COMPILER_ENABLED) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala index 66594d10154..e667fedf038 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/aggregate.scala @@ -20,7 +20,7 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf import ai.rapids.cudf.NvtxColor -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.TaskContext @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistrib import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.{ExplainUtils, SortExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.{CudfAggregate, GpuAggregateExpression, GpuDeclarativeAggregate} import org.apache.spark.sql.types.{DoubleType, FloatType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} @@ -338,30 +337,11 @@ case class GpuHashAggregateExec( // aggregation, because it detected a spill // d) sort based aggregation is then the mode forward, and not covered in this. override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val computeAggTime = longMetric("computeAggTime") - val concatTime = longMetric("concatTime") - // These metrics are supported by the cpu hash aggregate - // - // We should eventually have gpu versions of: - // - // This is the peak memory used max of what the hash map has used - // and what the external sorter has used - // val peakMemory = longMetric("peakMemory") - // - // Byte amount spilled. - // val spillSize = longMetric("spillSize") - // - // These don't make a lot of sense for the gpu case: - // - // This the time that has passed while setting up the iterators for tungsten - // val aggTime = longMetric("aggTime") - // - // Avg number of bucket list iterations per lookup in the underlying map - // val avgHashProbe = longMetric("avgHashProbe") - // + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val computeAggTime = gpuLongMetric(AGG_TIME) + val concatTime = gpuLongMetric(CONCAT_TIME) val rdd = child.executeColumnar() // cache in a local variable to avoid serializing the full child plan @@ -620,7 +600,7 @@ case class GpuHashAggregateExec( */ private def concatenateBatches(aggregatedInputCb: ColumnarBatch, aggregatedCb: ColumnarBatch, - concatTime: SQLMetric): Seq[GpuColumnVector] = { + concatTime: GpuMetric): Seq[GpuColumnVector] = { withResource(new NvtxWithMetrics("concatenateBatches", NvtxColor.BLUE, concatTime)) { _ => // get tuples of columns to concatenate @@ -806,7 +786,7 @@ case class GpuHashAggregateExec( groupingExpressions: Seq[Expression], aggModeCudfAggregates : Seq[(AggregateMode, Seq[CudfAggregate])], merge : Boolean, - computeAggTime: SQLMetric): ColumnarBatch = { + computeAggTime: GpuMetric): ColumnarBatch = { val nvtxRange = new NvtxWithMetrics("computeAggregate", NvtxColor.CYAN, computeAggTime) try { if (groupingExpressions.nonEmpty) { @@ -890,15 +870,11 @@ case class GpuHashAggregateExec( } } + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL override lazy val additionalMetrics = Map( - // not supported in GPU - "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), - "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), - "avgHashProbe" -> - SQLMetrics.createAverageMetric(sparkContext, "avg hash probe bucket list iters"), - "computeAggTime"-> - SQLMetrics.createNanoTimingMetric(sparkContext, "time in compute agg"), - "concatTime"-> SQLMetrics.createNanoTimingMetric(sparkContext, "time in batch concat") + AGG_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_AGG_TIME), + CONCAT_TIME-> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_CONCAT_TIME) ) protected def outputExpressions: Seq[NamedExpression] = resultExpressions diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index b5fd142ef7b..fcf55be2316 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf import ai.rapids.cudf.{NvtxColor, Scalar, Table} -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, NamedExpression, NullIntolerant, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, RangePartitioning, SinglePartition, UnknownPartitioning} import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.rapids.GpuPredicateHelper import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.types.{DataType, LongType} @@ -35,7 +34,7 @@ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuProjectExec extends Arm { def projectAndClose[A <: Expression](cb: ColumnarBatch, boundExprs: Seq[A], - totalTime: SQLMetric): ColumnarBatch = { + totalTime: GpuMetric): ColumnarBatch = { val nvtxRange = new NvtxWithMetrics("ProjectExec", NvtxColor.CYAN, totalTime) try { project(cb, boundExprs) @@ -76,9 +75,9 @@ case class GpuProjectExec(projectList: Seq[Expression], child: SparkPlan) throw new IllegalStateException(s"Row-based execution should not occur for $this") override def doExecuteColumnar() : RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) val boundProjectList = GpuBindReferences.bindGpuReferences(projectList, child.output) val rdd = child.executeColumnar() rdd.map { cb => @@ -99,9 +98,9 @@ object GpuFilter extends Arm { def apply( batch: ColumnarBatch, boundCondition: Expression, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - filterTime: SQLMetric): ColumnarBatch = { + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + filterTime: GpuMetric): ColumnarBatch = { withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, filterTime)) { _ => val filteredBatch = GpuFilter(batch, boundCondition) numOutputBatches += 1 @@ -169,10 +168,13 @@ case class GpuFilterExec(condition: Expression, child: SparkPlan) override def doExecute(): RDD[InternalRow] = throw new IllegalStateException(s"Row-based execution should not occur for $this") + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) val boundCondition = GpuBindReferences.bindReference(condition, child.output) val rdd = child.executeColumnar() rdd.map { batch => @@ -220,9 +222,9 @@ case class GpuRangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range } protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) val maxRowCountPerBatch = Math.min(targetSizeBytes/8, Int.MaxValue) if (isEmptyRange) { @@ -336,9 +338,9 @@ case class GpuUnionExec(children: Seq[SparkPlan]) extends SparkPlan with GpuExec throw new IllegalStateException(s"Row-based execution should not occur for $this") override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) sparkContext.union(children.map(_.executeColumnar())).map { batch => withResource(new NvtxWithMetrics("Union", NvtxColor.CYAN, totalTime)) { _ => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala index 17101e1c448..894d8e1735c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/limit.scala @@ -20,7 +20,7 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{NvtxColor, Table} import ai.rapids.cudf -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.rdd.RDD @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, Nu import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.{CollectLimitExec, LimitExec, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} @@ -53,9 +52,9 @@ trait GpuBaseLimitExec extends LimitExec with GpuExec { throw new IllegalStateException(s"Row-based execution should not occur for $this") override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) val crdd = child.executeColumnar() crdd.mapPartitions { cbIter => @@ -177,7 +176,7 @@ object GpuTopN extends Arm { private[this] def sortBatch( sortOrder: Seq[SortOrder], inputBatch: ColumnarBatch, - sortTime: SQLMetric): ColumnarBatch = { + sortTime: GpuMetric): ColumnarBatch = { withResource(new NvtxWithMetrics("sort", NvtxColor.DARK_GREEN, sortTime)) { _ => var outputTypes: Seq[DataType] = Nil var inputTbl: Table = null @@ -204,7 +203,7 @@ object GpuTopN extends Arm { private[this] def concatAndClose(a: SpillableColumnarBatch, b: ColumnarBatch, - concatTime: SQLMetric): ColumnarBatch = { + concatTime: GpuMetric): ColumnarBatch = { withResource(new NvtxWithMetrics("readNConcat", NvtxColor.CYAN, concatTime)) { _ => val dataTypes = GpuColumnVector.extractTypes(b) val aTable = withResource(a) { a => @@ -240,7 +239,7 @@ object GpuTopN extends Arm { def apply(limit: Int, sortOrder: Seq[SortOrder], batch: ColumnarBatch, - sortTime: SQLMetric): ColumnarBatch = { + sortTime: GpuMetric): ColumnarBatch = { withResource(sortBatch(sortOrder, batch, sortTime)) { sorted => takeN(sorted, limit) } @@ -249,14 +248,13 @@ object GpuTopN extends Arm { def apply(limit: Int, sortOrder: Seq[SortOrder], iter: Iterator[ColumnarBatch], - totalTime: SQLMetric, - sortTime: SQLMetric, - concatTime: SQLMetric, - batchTime: SQLMetric, - inputBatches: SQLMetric, - inputRows: SQLMetric, - outputBatches: SQLMetric, - outputRows: SQLMetric): Iterator[ColumnarBatch] = + totalTime: GpuMetric, + sortTime: GpuMetric, + concatTime: GpuMetric, + inputBatches: GpuMetric, + inputRows: GpuMetric, + outputBatches: GpuMetric, + outputRows: GpuMetric): Iterator[ColumnarBatch] = new Iterator[ColumnarBatch]() { override def hasNext: Boolean = iter.hasNext @@ -291,10 +289,8 @@ object GpuTopN extends Arm { apply(limit, sortOrder, concat, sortTime) } } - pending = withResource(new NvtxWithMetrics("make batch", - NvtxColor.RED, batchTime)) { _ => - Some(SpillableColumnarBatch(runningResult, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) - } + pending = + Some(SpillableColumnarBatch(runningResult, SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) } } val ret = pending.get.getColumnarBatch() @@ -324,27 +320,27 @@ case class GpuTopN( projectList.map(_.toAttribute) } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES), - "sortTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "sort time"), - "concatTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "concat time"), - "batchTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "batch time") + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES), + SORT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_SORT_TIME), + CONCAT_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_CONCAT_TIME) ) override def doExecuteColumnar(): RDD[ColumnarBatch] = { val boundSortExprs = GpuBindReferences.bindReferences(sortOrder, child.output) val boundProjectExprs = GpuBindReferences.bindGpuReferences(projectList, child.output) - val totalTime = metrics(TOTAL_TIME) - val inputBatches = metrics(NUM_INPUT_BATCHES) - val inputRows = metrics(NUM_INPUT_ROWS) - val outputBatches = metrics(NUM_OUTPUT_BATCHES) - val outputRows = metrics(NUM_OUTPUT_ROWS) - val sortTime = metrics("sortTime") - val concatTime = metrics("concatTime") - val batchTime = metrics("batchTime") + val totalTime = gpuLongMetric(TOTAL_TIME) + val inputBatches = gpuLongMetric(NUM_INPUT_BATCHES) + val inputRows = gpuLongMetric(NUM_INPUT_ROWS) + val outputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val outputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val sortTime = gpuLongMetric(SORT_TIME) + val concatTime = gpuLongMetric(CONCAT_TIME) child.executeColumnar().mapPartitions { iter => - val topN = GpuTopN(limit, boundSortExprs, iter, totalTime, sortTime, concatTime, batchTime, + val topN = GpuTopN(limit, boundSortExprs, iter, totalTime, sortTime, concatTime, inputBatches, inputRows, outputBatches, outputRows) if (projectList != child.output) { topN.map { batch => diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala index 4cad40679ab..fc6c9b89f28 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuCartesianProductExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,7 +19,7 @@ package org.apache.spark.sql.rapids import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import ai.rapids.cudf.{JCudfSerialization, NvtxColor, NvtxRange, Table} -import com.nvidia.spark.rapids.{Arm, GpuBindReferences, GpuBuildLeft, GpuColumnVector, GpuExec, GpuExpression, GpuSemaphore} +import com.nvidia.spark.rapids.{Arm, GpuBindReferences, GpuBuildLeft, GpuColumnVector, GpuExec, GpuExpression, GpuMetric, GpuSemaphore, MetricsLevel} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.{Dependency, NarrowDependency, Partition, SparkContext, TaskContext} @@ -27,7 +27,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils, SparkPlan} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.rapids.execution.GpuBroadcastNestedLoopJoinExecBase import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} @@ -111,12 +110,12 @@ class GpuCartesianRDD( sc: SparkContext, boundCondition: Option[GpuExpression], outputSchema: Array[DataType], - joinTime: SQLMetric, - joinOutputRows: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - filterTime: SQLMetric, - totalTime: SQLMetric, + joinTime: GpuMetric, + joinOutputRows: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + filterTime: GpuMetric, + totalTime: GpuMetric, var rdd1 : RDD[GpuSerializableBatch], var rdd2 : RDD[GpuSerializableBatch]) extends RDD[ColumnarBatch](sc, Nil) @@ -186,8 +185,8 @@ object GpuNoColumnCrossJoin extends Arm { def divideIntoBatches( rowCounts: RDD[Long], targetSizeBytes: Long, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric): RDD[ColumnarBatch] = { + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric): RDD[ColumnarBatch] = { // Hash aggregate explodes the rows out, so if we go too large // it can blow up. The size of a Long is 8 bytes so we just go with // that as our estimate, no nulls. @@ -215,8 +214,8 @@ object GpuNoColumnCrossJoin extends Arm { table: Table, numTimes: Long, outputSchema: Array[DataType], - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric): Iterator[ColumnarBatch] = { + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric): Iterator[ColumnarBatch] = { // TODO if we hit a point where we need to we can divide the data up into batches // The current use case is likely to be small enough that we are OK without this. assert(numTimes < Int.MaxValue) @@ -233,6 +232,8 @@ case class GpuCartesianProductExec( right: SparkPlan, condition: Option[Expression], targetSizeBytes: Long) extends BinaryExecNode with GpuExec { + import GpuMetric._ + override def output: Seq[Attribute]= left.output ++ right.output override def verboseStringWithOperatorId(): String = { @@ -243,23 +244,23 @@ case class GpuCartesianProductExec( """.stripMargin } - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time"), - "dataSize" -> SQLMetrics.createMetric(sparkContext, "size of shuffled data")) + protected override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + protected override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) protected override def doExecute(): RDD[InternalRow] = throw new IllegalStateException("This should only be called from columnar") protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { - import com.nvidia.spark.rapids.GpuMetricNames._ - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val joinTime = longMetric("joinTime") - val joinOutputRows = longMetric("joinOutputRows") - val filterTime = longMetric("filterTime") - val totalTime = longMetric(TOTAL_TIME) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val joinTime = gpuLongMetric(JOIN_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) + val filterTime = gpuLongMetric(FILTER_TIME) + val totalTime = gpuLongMetric(TOTAL_TIME) val outputSchema = output.map(_.dataType).toArray val boundCondition = condition.map(GpuBindReferences.bindGpuReference(_, output)) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 348f31312c2..f822993ef96 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import scala.collection.mutable.HashMap -import com.nvidia.spark.rapids.{GpuDataSourceRDD, GpuExec, GpuMetricNames, GpuParquetMultiFilePartitionReaderFactory, GpuReadCSVFileFormat, GpuReadFileFormatWithMetrics, GpuReadOrcFileFormat, GpuReadParquetFileFormat, RapidsConf, ShimLoader, SparkPlanMeta} +import com.nvidia.spark.rapids.{GpuDataSourceRDD, GpuExec, GpuMetric, GpuParquetMultiFilePartitionReaderFactory, GpuReadCSVFileFormat, GpuReadFileFormatWithMetrics, GpuReadOrcFileFormat, GpuReadParquetFileFormat, RapidsConf, ShimLoader, SparkPlanMeta} import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.{ExecSubqueryExpression, ExplainUtils, Fil import org.apache.spark.sql.execution.datasources.{BucketingUtils, DataSourceStrategy, DataSourceUtils, FileFormat, FilePartition, HadoopFsRelation, PartitionDirectory, PartitionedFile} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -67,6 +67,7 @@ case class GpuFileSourceScanExec( tableIdentifier: Option[TableIdentifier], queryUsesInputFile: Boolean = false)(@transient val rapidsConf: RapidsConf) extends GpuDataSourceScanExec with GpuExec { + import GpuMetric._ private val isParquetFileFormat: Boolean = relation.fileFormat.isInstanceOf[ParquetFileFormat] private val isPerFileReadEnabled = rapidsConf.isParquetPerFileReadEnabled || !isParquetFileFormat @@ -295,7 +296,7 @@ case class GpuFileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options), - metrics = metrics) + metrics = allMetrics) Some(reader) } else { None @@ -318,10 +319,10 @@ case class GpuFileSourceScanExec( /** SQL metrics generated only for scans using dynamic partition pruning. */ private lazy val staticMetrics = if (partitionFilters.filter(isDynamicPruningFilter).nonEmpty) { - Map("staticFilesNum" -> SQLMetrics.createMetric(sparkContext, "static number of files read"), - "staticFilesSize" -> SQLMetrics.createSizeMetric(sparkContext, "static size of files read")) + Map("staticFilesNum" -> createMetric(ESSENTIAL_LEVEL, "static number of files read"), + "staticFilesSize" -> createSizeMetric(ESSENTIAL_LEVEL, "static size of files read")) } else { - Map.empty[String, SQLMetric] + Map.empty[String, GpuMetric] } /** Helper for computing total number and size of files in selected partitions. */ @@ -342,36 +343,39 @@ case class GpuFileSourceScanExec( } } - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files read"), - "metadataTime" -> SQLMetrics.createTimingMetric(sparkContext, "metadata time"), - "filesSize" -> SQLMetrics.createSizeMetric(sparkContext, "size of files read") + override lazy val allMetrics = Map( + NUM_OUTPUT_ROWS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_OUTPUT_ROWS), + NUM_OUTPUT_BATCHES -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_OUTPUT_BATCHES), + "numFiles" -> createMetric(ESSENTIAL_LEVEL, "number of files read"), + "metadataTime" -> createTimingMetric(ESSENTIAL_LEVEL, "metadata time"), + "filesSize" -> createSizeMetric(ESSENTIAL_LEVEL, "size of files read"), + GPU_DECODE_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_GPU_DECODE_TIME), + BUFFER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUFFER_TIME), + PEAK_DEVICE_MEMORY -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_PEAK_DEVICE_MEMORY) ) ++ { // Tracking scan time has overhead, we can't afford to do it for each row, and can only do // it for each batch. if (supportsColumnar) { - Some("scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + Some("scanTime" -> createTimingMetric(ESSENTIAL_LEVEL, "scan time")) } else { None } } ++ { if (relation.partitionSchemaOption.isDefined) { Map( - "numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions read"), - "pruningTime" -> - SQLMetrics.createTimingMetric(sparkContext, "dynamic partition pruning time")) + NUM_PARTITIONS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_PARTITIONS), + "pruningTime" -> createTimingMetric(ESSENTIAL_LEVEL, "dynamic partition pruning time")) } else { - Map.empty[String, SQLMetric] + Map.empty[String, GpuMetric] } - } ++ staticMetrics ++ GpuMetricNames.buildGpuScanMetrics(sparkContext) + } ++ staticMetrics override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException(s"Row-based execution should not occur for $this") override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric("numOutputRows") - val scanTime = longMetric("scanTime") + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val scanTime = gpuLongMetric("scanTime") inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches => new Iterator[ColumnarBatch] { @@ -465,7 +469,7 @@ case class GpuFileSourceScanExec( relation.partitionSchema, pushedDownFilters.toArray, rapidsConf, - metrics, + allMetrics, queryUsesInputFile) // note we use the v2 DataSourceRDD instead of FileScanRDD so we don't have to copy more code @@ -517,7 +521,7 @@ case class GpuFileSourceScanExec( relation.partitionSchema, pushedDownFilters.toArray, rapidsConf, - metrics, + allMetrics, queryUsesInputFile) // note we use the v2 DataSourceRDD instead of FileScanRDD so we don't have to copy more code diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala index ee92b1327b8..b72b84296c9 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastExchangeExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -26,10 +26,11 @@ import scala.util.control.NonFatal import ai.rapids.cudf.{HostMemoryBuffer, JCudfSerialization, NvtxColor, NvtxRange} import com.google.common.util.concurrent.ThreadFactoryBuilder import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import org.apache.spark.{broadcast, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.broadcast.Broadcast import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -238,11 +239,13 @@ abstract class GpuBroadcastExchangeExecBase( mode: BroadcastMode, child: SparkPlan) extends Exchange with GpuExec { + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL override lazy val additionalMetrics = Map( - "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), - "collectTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to collect"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to build"), - "broadcastTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to broadcast")) + "dataSize" -> createSizeMetric(ESSENTIAL_LEVEL, "data size"), + COLLECT_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_COLLECT_TIME), + BUILD_TIME -> createNanoTimingMetric(ESSENTIAL_LEVEL, DESCRIPTION_BUILD_TIME), + "broadcastTime" -> createNanoTimingMetric(ESSENTIAL_LEVEL, "time to broadcast")) override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) @@ -250,14 +253,14 @@ abstract class GpuBroadcastExchangeExecBase( override def outputBatching: CoalesceGoal = RequireSingleBatch @transient - private lazy val promise = Promise[broadcast.Broadcast[Any]]() + private lazy val promise = Promise[Broadcast[Any]]() /** * For registering callbacks on `relationFuture`. * Note that calling this field will not start the execution of broadcast job. */ @transient - lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = promise.future + lazy val completionFuture: concurrent.Future[Broadcast[Any]] = promise.future @transient private val timeout: Long = SQLConf.get.broadcastTimeout @@ -265,18 +268,18 @@ abstract class GpuBroadcastExchangeExecBase( val _runId: UUID = UUID.randomUUID() @transient - lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + lazy val relationFuture: Future[Broadcast[Any]] = { // relationFuture is used in "doExecute". Therefore we can get the execution id correctly here. val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val totalTime = longMetric(TOTAL_TIME) - val collectTime = longMetric("collectTime") - val buildTime = longMetric("buildTime") - val broadcastTime = longMetric("broadcastTime") - - val task = new Callable[broadcast.Broadcast[Any]]() { - override def call(): broadcast.Broadcast[Any] = { + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val totalTime = gpuLongMetric(TOTAL_TIME) + val collectTime = gpuLongMetric(COLLECT_TIME) + val buildTime = gpuLongMetric(BUILD_TIME) + val broadcastTime = gpuLongMetric("broadcastTime") + + val task = new Callable[Broadcast[Any]]() { + override def call(): Broadcast[Any] = { // This will run in another thread. Set the execution id so that we can connect these jobs // with the correct execution. SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { @@ -313,7 +316,7 @@ abstract class GpuBroadcastExchangeExecBase( // val relation = mode.transform(input, Some(numRows)) val dataSize = batch.dataSize - longMetric("dataSize") += dataSize + gpuLongMetric("dataSize") += dataSize if (dataSize >= (8L << 30)) { throw new SparkException( s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") @@ -339,10 +342,10 @@ abstract class GpuBroadcastExchangeExecBase( case oe: OutOfMemoryError => val ex = new Exception( new OutOfMemoryError("Not enough memory to build and broadcast the table to all " + - "worker nodes. As a workaround, you can either disable broadcast by setting " + - s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark " + - s"driver memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value.") - .initCause(oe.getCause)) + "worker nodes. As a workaround, you can either disable broadcast by setting " + + s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark " + + s"driver memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value.") + .initCause(oe.getCause)) promise.failure(ex) throw ex case e if !NonFatal(e) => @@ -358,7 +361,7 @@ abstract class GpuBroadcastExchangeExecBase( } } } - GpuBroadcastExchangeExec.executionContext.submit[broadcast.Broadcast[Any]](task) + GpuBroadcastExchangeExec.executionContext.submit[Broadcast[Any]](task) } override protected def doPrepare(): Unit = { @@ -371,9 +374,9 @@ abstract class GpuBroadcastExchangeExecBase( "GpuBroadcastExchange does not support the execute() code path.") } - override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + override protected[sql] def doExecuteBroadcast[T](): Broadcast[T] = { try { - relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[broadcast.Broadcast[T]] + relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[Broadcast[T]] } catch { case ex: TimeoutException => logError(s"Could not execute broadcast in $timeout secs.", ex) @@ -388,12 +391,12 @@ abstract class GpuBroadcastExchangeExecBase( } } - final def executeColumnarBroadcast[T](): broadcast.Broadcast[T] = { + final def executeColumnarBroadcast[T](): Broadcast[T] = { if (isCanonicalizedPlan) { throw new IllegalStateException("A canonicalized plan is not supposed to be executed.") } try { - relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[broadcast.Broadcast[T]] + relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[Broadcast[T]] } catch { case ex: TimeoutException => logError(s"Could not execute broadcast in $timeout secs.", ex) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala index 796ea9231e7..c931b2847ac 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastNestedLoopJoinExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} +import com.nvidia.spark.rapids.GpuMetric.{NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, TOTAL_TIME} import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast @@ -96,12 +96,12 @@ object GpuBroadcastNestedLoopJoinExecBase extends Arm { buildSide: GpuBuildSide, boundCondition: Option[GpuExpression], outputSchema: Array[DataType], - joinTime: SQLMetric, - joinOutputRows: SQLMetric, - numOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - filterTime: SQLMetric, - totalTime: SQLMetric): Iterator[ColumnarBatch] = { + joinTime: GpuMetric, + joinOutputRows: GpuMetric, + numOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + filterTime: GpuMetric, + totalTime: GpuMetric): Iterator[ColumnarBatch] = { streamedIter.map { cb => val startTime = System.nanoTime() val streamTable = withResource(cb) { cb => @@ -141,6 +141,8 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( condition: Option[Expression], targetSizeBytes: Long) extends BinaryExecNode with GpuExec { + import GpuMetric._ + // Spark BuildSide, BuildRight, BuildLeft changed packages between Spark versions // so return a GPU version that is agnostic to the Spark version. def getGpuBuildSide: GpuBuildSide @@ -148,12 +150,14 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException("This should only be called from columnar") - override lazy val additionalMetrics: Map[String, SQLMetric] = Map( - "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "build side size"), - "buildTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "build time"), - "joinTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "join time"), - "joinOutputRows" -> SQLMetrics.createMetric(sparkContext, "join output rows"), - "filterTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "filter time")) + override val outputRowsLevel: MetricsLevel = ESSENTIAL_LEVEL + override val outputBatchesLevel: MetricsLevel = MODERATE_LEVEL + override lazy val additionalMetrics: Map[String, GpuMetric] = Map( + BUILD_DATA_SIZE -> createSizeMetric(MODERATE_LEVEL, DESCRIPTION_BUILD_DATA_SIZE), + BUILD_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_BUILD_TIME), + JOIN_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_TIME), + JOIN_OUTPUT_ROWS -> createMetric(MODERATE_LEVEL, DESCRIPTION_JOIN_OUTPUT_ROWS), + FILTER_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_FILTER_TIME)) /** BuildRight means the right relation <=> the broadcast relation. */ private val (streamed, broadcast) = getGpuBuildSide match { @@ -198,8 +202,8 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( private[this] def makeBuiltTable( broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], - buildTime: SQLMetric, - buildDataSize: SQLMetric): Table = { + buildTime: GpuMetric, + buildDataSize: GpuMetric): Table = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => val ret = GpuColumnVector.from(broadcastRelation.value.batch) // Don't warn for a leak, because we cannot control when we are done with this @@ -214,8 +218,8 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( private[this] def computeBuildRowCount( broadcastRelation: Broadcast[SerializeConcatHostBuffersDeserializeBatch], - buildTime: SQLMetric, - buildDataSize: SQLMetric): Int = { + buildTime: GpuMetric, + buildDataSize: GpuMetric): Int = { withResource(new NvtxWithMetrics("build join table", NvtxColor.GREEN, buildTime)) { _ => buildDataSize += 0 broadcastRelation.value.batch.numRows() @@ -223,17 +227,17 @@ abstract class GpuBroadcastNestedLoopJoinExecBase( } override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val totalTime = longMetric(TOTAL_TIME) - val joinTime = longMetric("joinTime") - val filterTime = longMetric("filterTime") - val joinOutputRows = longMetric("joinOutputRows") + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val totalTime = gpuLongMetric(TOTAL_TIME) + val joinTime = gpuLongMetric(JOIN_TIME) + val filterTime = gpuLongMetric(FILTER_TIME) + val joinOutputRows = gpuLongMetric(JOIN_OUTPUT_ROWS) val boundCondition = condition.map(GpuBindReferences.bindGpuReference(_, output)) - val buildTime = longMetric("buildTime") - val buildDataSize = longMetric("buildDataSize") + val buildTime = gpuLongMetric(BUILD_TIME) + val buildDataSize = gpuLongMetric(BUILD_DATA_SIZE) val outputSchema = output.map(_.dataType).toArray diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuCustomShuffleReaderExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuCustomShuffleReaderExec.scala index f8673078534..807a76867a3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuCustomShuffleReaderExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuCustomShuffleReaderExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. All rights reserved. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. All rights reserved. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -15,8 +15,7 @@ */ package org.apache.spark.sql.rapids.execution -import com.nvidia.spark.rapids.{CoalesceGoal, GpuExec, ShimLoader} -import com.nvidia.spark.rapids.GpuMetricNames.{DESCRIPTION_NUM_PARTITIONS, DESCRIPTION_PARTITION_SIZE, DESCRIPTION_TOTAL_TIME, NUM_PARTITIONS, PARTITION_SIZE, TOTAL_TIME} +import com.nvidia.spark.rapids.{CoalesceGoal, GpuExec, GpuMetric, ShimLoader} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -25,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.execution.{CoalescedPartitionSpec, PartialMapperPartitionSpec, PartialReducerPartitionSpec, ShufflePartitionSpec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.adaptive.ShuffleQueryStageExec import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -38,6 +36,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuCustomShuffleReaderExec( child: SparkPlan, partitionSpecs: Seq[ShufflePartitionSpec]) extends UnaryExecNode with GpuExec { + import GpuMetric._ /** * We intentionally override metrics in this case rather than overriding additionalMetrics so @@ -46,12 +45,10 @@ case class GpuCustomShuffleReaderExec( * * The Spark version of this operator does not output any metrics. */ - override lazy val metrics: Map[String, SQLMetric] = Map( - PARTITION_SIZE -> - SQLMetrics.createSizeMetric(sparkContext, DESCRIPTION_PARTITION_SIZE), - NUM_PARTITIONS -> - SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_PARTITIONS), - TOTAL_TIME -> SQLMetrics.createNanoTimingMetric(sparkContext, DESCRIPTION_TOTAL_TIME) + override lazy val allMetrics: Map[String, GpuMetric] = Map( + PARTITION_SIZE -> createSizeMetric(ESSENTIAL_LEVEL, DESCRIPTION_PARTITION_SIZE), + NUM_PARTITIONS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_PARTITIONS), + TOTAL_TIME -> createNanoTimingMetric(MODERATE_LEVEL, DESCRIPTION_TOTAL_TIME) ) override def output: Seq[Attribute] = child.output diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala index 69395063083..3e37d77f988 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuHashJoin.scala @@ -16,13 +16,12 @@ package org.apache.spark.sql.rapids.execution import ai.rapids.cudf.{NvtxColor, Table} -import com.nvidia.spark.rapids.{CoalesceGoal, GpuBindReferences, GpuBuildLeft, GpuBuildRight, GpuBuildSide, GpuColumnVector, GpuExec, GpuExpression, GpuFilter, GpuIsNotNull, GpuProjectExec, NvtxWithMetrics, RapidsMeta, RequireSingleBatch} +import com.nvidia.spark.rapids._ import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.{ExistenceJoin, FullOuter, InnerLike, JoinType, LeftAnti, LeftExistence, LeftOuter, LeftSemi, RightOuter} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} object GpuHashJoin { @@ -157,13 +156,13 @@ trait GpuHashJoin extends GpuExec { def doJoin(builtTable: Table, stream: Iterator[ColumnarBatch], boundCondition: Option[Expression], - numOutputRows: SQLMetric, - joinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - streamTime: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric, - totalTime: SQLMetric): Iterator[ColumnarBatch] = { + numOutputRows: GpuMetric, + joinOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + streamTime: GpuMetric, + joinTime: GpuMetric, + filterTime: GpuMetric, + totalTime: GpuMetric): Iterator[ColumnarBatch] = { new Iterator[ColumnarBatch] { import scala.collection.JavaConverters._ var nextCb: Option[ColumnarBatch] = None @@ -215,11 +214,11 @@ trait GpuHashJoin extends GpuExec { private[this] def doJoin(builtTable: Table, streamedBatch: ColumnarBatch, boundCondition: Option[Expression], - numOutputRows: SQLMetric, - numJoinOutputRows: SQLMetric, - numOutputBatches: SQLMetric, - joinTime: SQLMetric, - filterTime: SQLMetric): Option[ColumnarBatch] = { + numOutputRows: GpuMetric, + numJoinOutputRows: GpuMetric, + numOutputBatches: GpuMetric, + joinTime: GpuMetric, + filterTime: GpuMetric): Option[ColumnarBatch] = { val combined = withResource(streamedBatch) { streamedBatch => withResource(GpuProjectExec.project(streamedBatch, gpuStreamedKeys)) { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExec.scala index 118a7eeadf3..e9fedbde46d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuShuffleExchangeExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2019-2021, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,7 +20,6 @@ import scala.collection.AbstractIterator import scala.concurrent.Future import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames.{DESCRIPTION_NUM_OUTPUT_BATCHES, DESCRIPTION_NUM_OUTPUT_ROWS, DESCRIPTION_NUM_PARTITIONS, DESCRIPTION_PARTITION_SIZE, NUM_OUTPUT_BATCHES, NUM_OUTPUT_ROWS, NUM_PARTITIONS, PARTITION_SIZE} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import org.apache.spark.{MapOutputStatistics, ShuffleDependency} @@ -70,6 +69,7 @@ class GpuShuffleMeta( abstract class GpuShuffleExchangeExecBase( override val outputPartitioning: Partitioning, child: SparkPlan) extends Exchange with GpuExec { + import GpuMetric._ // Shuffle produces a lot of small output batches that should be coalesced together. // This coalesce occurs on the GPU and should always be done when using RAPIDS shuffle. @@ -80,18 +80,16 @@ abstract class GpuShuffleExchangeExecBase( SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) lazy val readMetrics = SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) - override lazy val additionalMetrics : Map[String, SQLMetric] = Map( - "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") - ) ++ readMetrics ++ writeMetrics + override lazy val additionalMetrics : Map[String, GpuMetric] = Map( + "dataSize" -> createSizeMetric(ESSENTIAL_LEVEL,"data size") + ) ++ GpuMetric.wrap(readMetrics) ++ GpuMetric.wrap(writeMetrics) // Spark doesn't report totalTime for this operator so we override metrics - override lazy val metrics: Map[String, SQLMetric] = Map( - PARTITION_SIZE -> - SQLMetrics.createMetric(sparkContext, DESCRIPTION_PARTITION_SIZE), - NUM_PARTITIONS -> - SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_PARTITIONS), - NUM_OUTPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_ROWS), - NUM_OUTPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_BATCHES) + override lazy val allMetrics: Map[String, GpuMetric] = Map( + PARTITION_SIZE -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_PARTITION_SIZE), + NUM_PARTITIONS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_PARTITIONS), + NUM_OUTPUT_ROWS -> createMetric(ESSENTIAL_LEVEL, DESCRIPTION_NUM_OUTPUT_ROWS), + NUM_OUTPUT_BATCHES -> createMetric(MODERATE_LEVEL, DESCRIPTION_NUM_OUTPUT_BATCHES) ) ++ additionalMetrics override def nodeName: String = "GpuColumnarExchange" @@ -114,7 +112,7 @@ abstract class GpuShuffleExchangeExecBase( // This value must be lazy because the child's output may not have been resolved // yet in all cases. private lazy val serializer: Serializer = new GpuColumnarBatchSerializer( - longMetric("dataSize")) + gpuLongMetric("dataSize")) @transient lazy val inputBatchRDD: RDD[ColumnarBatch] = child.executeColumnar() @@ -131,7 +129,7 @@ abstract class GpuShuffleExchangeExecBase( outputPartitioning, sparkTypes, serializer, - metrics, + allMetrics, writeMetrics, additionalMetrics) } @@ -160,9 +158,9 @@ object GpuShuffleExchangeExec { newPartitioning: Partitioning, sparkTypes: Array[DataType], serializer: Serializer, - metrics: Map[String, SQLMetric], + metrics: Map[String, GpuMetric], writeMetrics: Map[String, SQLMetric], - additionalMetrics: Map[String, SQLMetric]) + additionalMetrics: Map[String, GpuMetric]) : ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { val isRoundRobin = newPartitioning match { case _: GpuRoundRobinPartitioning => true @@ -212,9 +210,9 @@ object GpuShuffleExchangeExec { } partitioned = getParts(batch).asInstanceOf[Array[(ColumnarBatch, Int)]] partitioned.foreach(batches => { - metrics(GpuMetricNames.NUM_OUTPUT_ROWS) += batches._1.numRows() + metrics(GpuMetric.NUM_OUTPUT_ROWS) += batches._1.numRows() }) - metrics(GpuMetricNames.NUM_OUTPUT_BATCHES) += partitioned.length + metrics(GpuMetric.NUM_OUTPUT_BATCHES) += partitioned.length at = 0 } } @@ -255,7 +253,7 @@ object GpuShuffleExchangeExec { sparkTypes, serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics), - metrics = additionalMetrics) + metrics = GpuMetric.unwrap(additionalMetrics)) dependency } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala index c42c3bf813d..130eef3a5fc 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/ShuffledBatchRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.rapids.execution import java.util import ai.rapids.cudf.NvtxColor -import com.nvidia.spark.rapids.{GpuMetricNames, NvtxWithMetrics, ShimLoader} +import com.nvidia.spark.rapids.{GpuMetric, NvtxWithMetrics, ShimLoader} import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -209,8 +209,8 @@ class ShuffledBatchRDD( .map(_._2).sum (reader, partitionSize) } - metrics(GpuMetricNames.NUM_PARTITIONS).add(1) - metrics(GpuMetricNames.PARTITION_SIZE).add(partitionSize) + metrics(GpuMetric.NUM_PARTITIONS).add(1) + metrics(GpuMetric.PARTITION_SIZE).add(partitionSize) reader.read().asInstanceOf[Iterator[Product2[Int, ColumnarBatch]]].map(_._2) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala index 8c39ae2aea6..c1175617ac0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuArrowEvalPythonExec.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020, NVIDIA CORPORATION. + * Copyright (c) 2020-2021, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -27,8 +27,8 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf.{AggregationOnColumn, ArrowIPCOptions, ArrowIPCWriterOptions, ColumnVector, HostBufferConsumer, HostBufferProvider, HostMemoryBuffer, NvtxColor, NvtxRange, StreamedTableReader, Table} -import com.nvidia.spark.rapids.{Arm, ConcatAndConsumeAll, GpuAggregateWindowFunction, GpuBindReferences, GpuColumnVector, GpuColumnVectorFromBuffer, GpuExec, GpuProjectExec, GpuSemaphore, GpuUnevaluable, SpillableColumnarBatch, SpillPriorities} -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.{Arm, ConcatAndConsumeAll, GpuAggregateWindowFunction, GpuBindReferences, GpuColumnVector, GpuColumnVectorFromBuffer, GpuExec, GpuMetric, GpuProjectExec, GpuSemaphore, GpuUnevaluable, SpillableColumnarBatch, SpillPriorities} +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.python.PythonWorkerSemaphore @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.python.PythonUDFRunner import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructField, StructType} @@ -59,8 +58,8 @@ class RebatchingRoundoffIterator( wrapped: Iterator[ColumnarBatch], schema: StructType, targetRoundoff: Int, - inputRows: SQLMetric, - inputBatches: SQLMetric) + inputRows: GpuMetric, + inputBatches: GpuMetric) extends Iterator[ColumnarBatch] with Arm { var pending: Option[SpillableColumnarBatch] = None @@ -539,18 +538,18 @@ case class GpuArrowEvalPythonExec( override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException(s"Row-based execution should not occur for $this") - override lazy val metrics: Map[String, SQLMetric] = Map( - NUM_OUTPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_ROWS), - NUM_OUTPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_BATCHES), - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES) + override lazy val allMetrics: Map[String, GpuMetric] = Map( + NUM_OUTPUT_ROWS -> createMetric(outputRowsLevel, DESCRIPTION_NUM_OUTPUT_ROWS), + NUM_OUTPUT_BATCHES -> createMetric(outputBatchesLevel, DESCRIPTION_NUM_OUTPUT_BATCHES), + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES) ) override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) - val numInputRows = longMetric(NUM_INPUT_ROWS) - val numInputBatches = longMetric(NUM_INPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) + val numInputRows = gpuLongMetric(NUM_INPUT_ROWS) + val numInputBatches = gpuLongMetric(NUM_INPUT_BATCHES) lazy val isPythonOnGpuEnabled = GpuPythonHelper.isPythonOnGpuEnabled(conf) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala index 58ae0f8ebfd..b63c3009c2a 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuWindowInPandasExecBase.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.rapids.execution.python import ai.rapids.cudf import ai.rapids.cudf.{Aggregation, Table} import com.nvidia.spark.rapids._ -import com.nvidia.spark.rapids.GpuMetricNames._ +import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.python.PythonWorkerSemaphore import scala.collection.mutable @@ -32,10 +32,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.UnaryExecNode -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.rapids.GpuAggregateExpression -import org.apache.spark.sql.types.{DataType, IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.ColumnarBatch @@ -88,8 +87,8 @@ abstract class GpuWindowInPandasExecMetaBase( class GroupingIterator( wrapped: Iterator[ColumnarBatch], partitionSpec: Seq[Expression], - inputRows: SQLMetric, - inputBatches: SQLMetric) extends Iterator[ColumnarBatch] with Arm { + inputRows: GpuMetric, + inputBatches: GpuMetric) extends Iterator[ColumnarBatch] with Arm { // Currently do it in a somewhat ugly way. In the future cuDF will provide a dedicated API. // Current solution assumes one group data exists in only one batch, so just split the @@ -388,21 +387,21 @@ trait GpuWindowInPandasExecBase extends UnaryExecNode with GpuExec { new ColumnarBatch(boundsCVs ++ dataCVs.map(_.incRefCount()), numRows) } - override lazy val metrics: Map[String, SQLMetric] = Map( - NUM_OUTPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_ROWS), - NUM_OUTPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_OUTPUT_BATCHES), - NUM_INPUT_ROWS -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_ROWS), - NUM_INPUT_BATCHES -> SQLMetrics.createMetric(sparkContext, DESCRIPTION_NUM_INPUT_BATCHES) + override lazy val allMetrics: Map[String, GpuMetric] = Map( + NUM_OUTPUT_ROWS -> createMetric(outputRowsLevel, DESCRIPTION_NUM_OUTPUT_ROWS), + NUM_OUTPUT_BATCHES -> createMetric(outputBatchesLevel, DESCRIPTION_NUM_OUTPUT_BATCHES), + NUM_INPUT_ROWS -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_ROWS), + NUM_INPUT_BATCHES -> createMetric(DEBUG_LEVEL, DESCRIPTION_NUM_INPUT_BATCHES) ) override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException(s"Row-based execution should not occur for $this") override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - val numInputRows = longMetric(NUM_INPUT_ROWS) - val numInputBatches = longMetric(NUM_INPUT_BATCHES) - val numOutputRows = longMetric(NUM_OUTPUT_ROWS) - val numOutputBatches = longMetric(NUM_OUTPUT_BATCHES) + val numInputRows = gpuLongMetric(NUM_INPUT_ROWS) + val numInputBatches = gpuLongMetric(NUM_INPUT_BATCHES) + val numOutputRows = gpuLongMetric(NUM_OUTPUT_ROWS) + val numOutputBatches = gpuLongMetric(NUM_OUTPUT_BATCHES) val sessionLocalTimeZone = conf.sessionLocalTimeZone // 1) Unwrap the expressions and build some info data: diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala index b812555125a..eb11ecd7a4a 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala @@ -57,12 +57,8 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { batch.close() // assert metrics are correct - assert(gpuRowToColumnarExec.metrics(GpuMetricNames.NUM_OUTPUT_ROWS).value == 14) - assert(gpuRowToColumnarExec.metrics(GpuMetricNames.NUM_OUTPUT_BATCHES).value == 14) - assert(gpuCoalesceBatches.metrics(GpuMetricNames.NUM_INPUT_ROWS).value == 14) - assert(gpuCoalesceBatches.metrics(GpuMetricNames.NUM_INPUT_BATCHES).value == 14) - assert(gpuCoalesceBatches.metrics(GpuMetricNames.NUM_OUTPUT_ROWS).value == 14) - assert(gpuCoalesceBatches.metrics(GpuMetricNames.NUM_OUTPUT_BATCHES).value == 1) + assert(gpuCoalesceBatches.metrics(GpuMetric.NUM_OUTPUT_ROWS).value == 14) + assert(gpuCoalesceBatches.metrics(GpuMetric.NUM_OUTPUT_BATCHES).value == 1) }) } @@ -96,8 +92,7 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { assert(coalesce.goal == RequireSingleBatch) assert(coalesce.goal.targetSizeBytes == Long.MaxValue) - assert(coalesce.additionalMetrics("numInputBatches").value == 7) - assert(coalesce.longMetric(GpuMetricNames.NUM_OUTPUT_BATCHES).value == 1) + assert(coalesce.longMetric(GpuMetric.NUM_OUTPUT_BATCHES).value == 1) }, conf) } @@ -212,9 +207,14 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { val hostToGpuCoalesceIterator = new HostToGpuCoalesceIterator(iter, TargetSize(1024), schema: StructType, - new SQLMetric("t1", 0), new SQLMetric("t2", 0), new SQLMetric("t3", 0), - new SQLMetric("t4", 0), new SQLMetric("t5", 0), new SQLMetric("t6", 0), - new SQLMetric("t7", 0), new SQLMetric("t8", 0), + WrappedGpuMetric(new SQLMetric("t1", 0)), + WrappedGpuMetric(new SQLMetric("t2", 0)), + WrappedGpuMetric(new SQLMetric("t3", 0)), + WrappedGpuMetric(new SQLMetric("t4", 0)), + WrappedGpuMetric(new SQLMetric("t5", 0)), + WrappedGpuMetric(new SQLMetric("t6", 0)), + WrappedGpuMetric(new SQLMetric("t7", 0)), + WrappedGpuMetric(new SQLMetric("t8", 0)), "testcoalesce", useArrowCopyOpt = true) @@ -230,9 +230,14 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { val hostToGpuCoalesceIterator = new HostToGpuCoalesceIterator(iter, TargetSize(1024), schema: StructType, - new SQLMetric("t1", 0), new SQLMetric("t2", 0), new SQLMetric("t3", 0), - new SQLMetric("t4", 0), new SQLMetric("t5", 0), new SQLMetric("t6", 0), - new SQLMetric("t7", 0), new SQLMetric("t8", 0), + WrappedGpuMetric(new SQLMetric("t1", 0)), + WrappedGpuMetric(new SQLMetric("t2", 0)), + WrappedGpuMetric(new SQLMetric("t3", 0)), + WrappedGpuMetric(new SQLMetric("t4", 0)), + WrappedGpuMetric(new SQLMetric("t5", 0)), + WrappedGpuMetric(new SQLMetric("t6", 0)), + WrappedGpuMetric(new SQLMetric("t7", 0)), + WrappedGpuMetric(new SQLMetric("t8", 0)), "testcoalesce", useArrowCopyOpt = true) @@ -249,9 +254,14 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { val hostToGpuCoalesceIterator = new HostToGpuCoalesceIterator(iter, TargetSize(1024), schema: StructType, - new SQLMetric("t1", 0), new SQLMetric("t2", 0), new SQLMetric("t3", 0), - new SQLMetric("t4", 0), new SQLMetric("t5", 0), new SQLMetric("t6", 0), - new SQLMetric("t7", 0), new SQLMetric("t8", 0), + WrappedGpuMetric(new SQLMetric("t1", 0)), + WrappedGpuMetric(new SQLMetric("t2", 0)), + WrappedGpuMetric(new SQLMetric("t3", 0)), + WrappedGpuMetric(new SQLMetric("t4", 0)), + WrappedGpuMetric(new SQLMetric("t5", 0)), + WrappedGpuMetric(new SQLMetric("t6", 0)), + WrappedGpuMetric(new SQLMetric("t7", 0)), + WrappedGpuMetric(new SQLMetric("t8", 0)), "testcoalesce", useArrowCopyOpt = false) @@ -344,14 +354,13 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { // assert the metrics start out at zero assert(coalesce.additionalMetrics("numInputBatches").value == 0) - assert(coalesce.longMetric(GpuMetricNames.NUM_OUTPUT_BATCHES).value == 0) + assert(coalesce.longMetric(GpuMetric.NUM_OUTPUT_BATCHES).value == 0) // execute the plan df2.collect() // assert the metrics are correct - assert(coalesce.additionalMetrics("numInputBatches").value == 14) - assert(coalesce.longMetric(GpuMetricNames.NUM_OUTPUT_BATCHES).value == 11) + assert(coalesce.longMetric(GpuMetric.NUM_OUTPUT_BATCHES).value == 11) }, conf) } @@ -379,7 +388,7 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { val schema = new StructType().add("i", LongType) .add("j", DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 3)) - val dummyMetric = new SQLMetric("ignored") + val dummyMetric = WrappedGpuMetric(new SQLMetric("ignored")) val coalesceIter = new GpuCoalesceIterator( batchIter, schema, @@ -460,7 +469,7 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { val schema = new StructType().add("i", LongType) .add("j", DecimalType(ai.rapids.cudf.DType.DECIMAL64_MAX_PRECISION, 3)) - val dummyMetric = new SQLMetric("ignored") + val dummyMetric = WrappedGpuMetric(new SQLMetric("ignored")) val coalesceIter = new GpuCoalesceIterator( batchIter, schema,