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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala:
##########
@@ -319,3 +319,46 @@ case class RDDScanExec(
 
   override def getStream: Option[SparkDataStream] = stream
 }
+
+/**
+ * A physical plan node for `OneRowRelation` for scans with no 'FROM' clause.
+ *
+ * We do not extend `RDDScanExec` in order to avoid complexity due to 
`TreeNode.makeCopy` and
+ * `TreeNode`'s general use of reflection.
+ */
+case class OneRowRelationExec() extends LeafExecNode
+  with InputRDDCodegen {
+
+  override val nodeName: String = s"Scan OneRowRelation"
+
+  override val output: Seq[Attribute] = Nil
+
+  private val emptyRow: InternalRow = InternalRow.empty
+
+  private val rdd = session.sparkContext.parallelize(Seq(emptyRow), 1)
+
+  override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output 
rows"))
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val numOutputRows = longMetric("numOutputRows")
+    val proj = UnsafeProjection.create(schema)
+    val outputRow = proj(emptyRow)
+    rdd.mapPartitionsWithIndexInternal { (index, iter) =>
+      iter.map { r =>
+        numOutputRows += 1
+        outputRow
+      }
+    }
+  }
+
+  override def simpleString(maxFields: Int): String = s"$nodeName[]"
+
+  override def inputRDD: RDD[InternalRow] = rdd
+
+  override protected val createUnsafeProjection: Boolean = true

Review Comment:
   If we do https://github.com/apache/spark/pull/50849/files#r2097521541 , then 
this can be false.



-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to