You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/18 13:06:13 UTC

[GitHub] [spark] Ngone51 commented on a change in pull request #31842: [SPARK-34748][SS] Create a rule of the analysis logic for streaming write

Ngone51 commented on a change in pull request #31842:
URL: https://github.com/apache/spark/pull/31842#discussion_r596834842



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ResolveWriteToStream.scala
##########
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.util.UUID
+
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.streaming.{WriteToStream, WriteToStreamStatement}
+import org.apache.spark.sql.connector.catalog.SupportsWrite
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.Utils
+
+/**
+ * Replaces logical [[WriteToStreamStatement]] operator with an [[WriteToStream]] operator.
+ */
+object ResolveWriteToStream extends Rule[LogicalPlan] with SQLConfHelper {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
+    case WriteToStreamStatement(userSpecifiedName, userSpecifiedCheckpointLocation,
+        useTempCheckpointLocation, recoverFromCheckpointLocation, sink, outputMode, hadoopConf,
+        isContinuousTrigger, queryPlan) =>

Review comment:
       It recommends matching the type only instead of expanding all parameters according to [pattern-matching](https://github.com/databricks/scala-style-guide#pattern-matching).

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/WriteToStreamStatement.scala
##########
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.streaming
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.streaming.OutputMode
+
+/**
+ * A statement for Stream writing. It contains all neccessary param and will be address in the
+ * rule [[ResolveStreamWrite]].
+ *
+ * @param userSpecifiedName  Query name optionally specified by the user.
+ * @param userSpecifiedCheckpointLocation  Checkpoint location optionally specified by the user.
+ * @param useTempCheckpointLocation  Whether to use a temporary checkpoint location when the user
+ *                                   has not specified one. If false, then error will be thrown.
+ * @param recoverFromCheckpointLocation  Whether to recover query from the checkpoint location.
+ *                                       If false and the checkpoint location exists, then error
+ *                                       will be thrown.
+ * @param sink  Sink to write the streaming outputs.
+ * @param outputMode  Output mode for the sink.
+ * @param hadoopConf  The Hadoop Configuration to get a FileSystem instance
+ * @param isContinuousTrigger  Whether the statement is triggered by a continous query or not.
+ * @param queryPlan  The analyzed query plan from the streaming DataFrame.
+ */
+case class WriteToStreamStatement(
+    userSpecifiedName: Option[String],
+    userSpecifiedCheckpointLocation: Option[String],
+    useTempCheckpointLocation: Boolean,
+    recoverFromCheckpointLocation: Boolean,
+    sink: Table,
+    outputMode: OutputMode,
+    hadoopConf: Configuration,
+    isContinuousTrigger: Boolean,
+    queryPlan: LogicalPlan) extends LogicalPlan {
+
+  override def isStreaming: Boolean = true
+
+  override def output: Seq[Attribute] = Nil
+
+  override def children: Seq[LogicalPlan] = Seq.empty

Review comment:
       Why not extend the `LeafNode` for simplicity?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
##########
@@ -239,82 +237,39 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo
       recoverFromCheckpointLocation: Boolean,
       trigger: Trigger,
       triggerClock: Clock): StreamingQueryWrapper = {
-    var deleteCheckpointOnStop = false
-    val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
-      new Path(userSpecified).toString
-    }.orElse {
-      df.sparkSession.sessionState.conf.checkpointLocation.map { location =>
-        new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toString
-      }
-    }.getOrElse {
-      if (useTempCheckpointLocation) {
-        deleteCheckpointOnStop = true
-        val tempDir = Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
-        logWarning("Temporary checkpoint location created which is deleted normally when" +
-          s" the query didn't fail: $tempDir. If it's required to delete it under any" +
-          s" circumstances, please set ${SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key} to" +
-          s" true. Important to know deleting temp checkpoint folder is best effort.")
-        tempDir
-      } else {
-        throw new AnalysisException(
-          "checkpointLocation must be specified either " +
-            """through option("checkpointLocation", ...) or """ +
-            s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""")
-      }
-    }
-
-    // If offsets have already been created, we trying to resume a query.
-    if (!recoverFromCheckpointLocation) {
-      val checkpointPath = new Path(checkpointLocation, "offsets")
-      val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf())
-      if (fs.exists(checkpointPath)) {
-        throw new AnalysisException(
-          s"This query does not support recovering from checkpoint location. " +
-            s"Delete $checkpointPath to start over.")
-      }
-    }
-
     val analyzedPlan = df.queryExecution.analyzed
     df.queryExecution.assertAnalyzed()
 
-    val operationCheckEnabled = sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled
+    val dataStreamWritePlan = WriteToStreamStatement(
+      userSpecifiedName,
+      userSpecifiedCheckpointLocation,
+      useTempCheckpointLocation,
+      recoverFromCheckpointLocation,
+      sink,
+      outputMode,
+      df.sparkSession.sessionState.newHadoopConf(),
+      trigger.isInstanceOf[ContinuousTrigger],
+      analyzedPlan)
 
-    if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) {
-      logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " +
-          "is not supported in streaming DataFrames/Datasets and will be disabled.")
-    }
+    val analyzedStreamWritePlan =
+      sparkSession.sessionState.executePlan(dataStreamWritePlan).analyzed

Review comment:
       This seems to introduce the analysis overhead here. Do we have a plan to add more rules? Or, maybe, something like: 
   ```scala
   val analyzedStreamWritePlan  = rules.foldLeft(dataStreamWritePlan) { case (p, r) => r.apply(p) }
   ```
   should be enough?
   
   




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

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



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