cloud-fan commented on code in PR #44375:
URL: https://github.com/apache/spark/pull/44375#discussion_r1429445056


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonDataSource.scala:
##########
@@ -124,21 +128,78 @@ class PythonPartitionReaderFactory(
 
   override def createReader(partition: InputPartition): 
PartitionReader[InternalRow] = {
     new PartitionReader[InternalRow] {
-      private val outputIter = source.createPartitionReadIteratorInPython(
-        partition.asInstanceOf[PythonInputPartition],
-        pickledReadFunc,
-        outputSchema,
-        jobArtifactUUID)
+      // Dummy SQLMetrics. The result is manually reported via DSv2 interface
+      // via passing the value to `CustomTaskMetric`. Note that 
`pythonOtherMetricsDesc`
+      // is not used when it is reported. It is to reuse existing Python 
runner.
+      // See also `UserDefinedPythonDataSource.createPythonMetrics`.
+      private[this] val metrics: Map[String, SQLMetric] = {
+        PythonSQLMetrics.pythonSizeMetricsDesc.keys
+          .map(_ -> new SQLMetric("size", -1)).toMap ++

Review Comment:
   The code is a little tricky. In the DS v2 framework, the reader (runs at the 
executor side) needs to update and report the current value of its metrics. To 
reuse existing code, here we use `SQLMetrics` and its value will be updated 
within `createMapInBatchEvaluatorFactory` (which calls 
`MapInBatchEvaluatorFactory`). Then we take the value from `SQLMetric` and 
report it via the DS v2 framework in `currentMetricsValues`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to