viirya commented on code in PR #50015:
URL: https://github.com/apache/spark/pull/50015#discussion_r1963014815


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -128,11 +130,39 @@ class QueryExecution(
     case _ => "command"
   }
 
+  private def assertNoStreamSourceMarkerNode(p: LogicalPlan): Unit = {
+    // In UnsupportedOperationChecker.checkForBatch, we check if the plan has 
any streaming node.
+    // That is more aggressive than just checking the marker node for 
streaming source which is
+    // yet to be materialized. We'd like to be a bit conservative here since 
this is the exact
+    // problematic case we figured out.
+    p.foreach {
+      case _: StreamingRelation | _: StreamingRelationV2 |
+           _: StreamingExecutionRelation | _: 
StreamingDataSourceV2ScanRelation =>
+        val msg = "Queries with streaming sources must be executed with 
writeStream.start()"
+        // This is exactly the same with 
UnsupportedOperationChecker.checkForBatch.
+        // TODO: Classify and issue a new error class, along with 
UnsupportedOperationChecker.
+        throw new ExtendedAnalysisException(
+          new AnalysisException(
+            errorClass = "_LEGACY_ERROR_TEMP_3102",
+            messageParameters = Map("msg" -> msg)),
+          plan = p)
+
+      case _ =>
+    }
+  }
+
   private def eagerlyExecuteCommands(p: LogicalPlan) = {
     def eagerlyExecute(
         p: LogicalPlan,
         name: String,
         mode: CommandExecutionMode.Value): LogicalPlan = {
+      // Since we are about to execute the plan, the plan shouldn't have a 
marker node to be
+      // materialized during microbatch planning. If the plan has a marker 
node, it is highly
+      // likely that users put streaming sources in a batch query.
+      // This case brings problem before reaching the check in 
UnsupportedOperationChecker,

Review Comment:
   Because `assertSupported` is not called for eagerly executed command?



-- 
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