spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tdas <...@git.apache.org>
Subject [GitHub] spark pull request #16220: [SPARK-18796][SS]StreamingQueryManager should not...
Date Fri, 09 Dec 2016 02:02:59 GMT
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16220#discussion_r91646897
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala
---
    @@ -206,90 +287,70 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession)
{
           recoverFromCheckpointLocation: Boolean = true,
           trigger: Trigger = ProcessingTime(0),
           triggerClock: Clock = new SystemClock()): StreamingQuery = {
    -    activeQueriesLock.synchronized {
    -      val name = userSpecifiedName match {
    -        case Some(n) =>
    -          if (activeQueries.values.exists(_.name == userSpecifiedName.get)) {
    +    // `queryName` and `queryId` will be set when we add them into `pendingQueryNames`
and
    +    // `pendingQueryIds` so that we can remove them at the end of this method when an
error happens.
    +    var queryName: String = null
    +    var queryId: UUID = null
    +    try {
    +      userSpecifiedName.foreach { name =>
    +        activeQueriesLock.synchronized {
    +          if (activeQueries.values.exists(_.name == name) || pendingQueryNames.contains(name))
{
                 throw new IllegalArgumentException(
    -              s"Cannot start query with name $n as a query with that name is already
active")
    +              s"Cannot start query with name $name as a query with that name is already
active")
               }
    -          n
    -        case None => null
    -      }
    -      val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified =>
    -        new Path(userSpecified).toUri.toString
    -      }.orElse {
    -        df.sparkSession.sessionState.conf.checkpointLocation.map { location =>
    -          new Path(location, name).toUri.toString
    +          pendingQueryNames += name
    +          queryName = name
             }
    -      }.getOrElse {
    -        if (useTempCheckpointLocation) {
    -          Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath
    -        } 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()
    -
    -      if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) {
    -        UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode)
           }
    -
    -      var nextSourceId = 0L
    -
    -      val logicalPlan = analyzedPlan.transform {
    -        case StreamingRelation(dataSource, _, output) =>
    -          // Materialize source to avoid creating it in every batch
    -          val metadataPath = s"$checkpointLocation/sources/$nextSourceId"
    -          val source = dataSource.createSource(metadataPath)
    -          nextSourceId += 1
    -          // We still need to use the previous `output` instead of `source.schema` as
attributes in
    -          // "df.logicalPlan" has already used attributes of the previous `output`.
    -          StreamingExecutionRelation(source, output)
    -      }
    -      val query = new StreamExecution(
    -        sparkSession,
    -        name,
    -        checkpointLocation,
    -        logicalPlan,
    +      val query = prepareAndCreateQuery(
    +        userSpecifiedName,
    +        userSpecifiedCheckpointLocation,
    +        df,
             sink,
    +        outputMode,
    +        useTempCheckpointLocation,
    +        recoverFromCheckpointLocation,
             trigger,
    -        triggerClock,
    -        outputMode)
    -
    -      if (activeQueries.values.exists(_.id == query.id)) {
    -        throw new IllegalStateException(
    -          s"Cannot start query with id ${query.id} as another query with same id is "
+
    -            s"already active. Perhaps you are attempting to restart a query from checkpoint"
+
    -            s"that is already active.")
    +        triggerClock)
    +      activeQueriesLock.synchronized {
    +        if (activeQueries.values.exists(_.id == query.id) || pendingQueryIds.contains(query.id))
{
    +          throw new IllegalStateException(
    +            s"Cannot start query with id ${query.id} as another query with same id is
" +
    +              s"already active. Perhaps you are attempting to restart a query from checkpoint
" +
    +              s"that is already active.")
    +        }
    +        pendingQueryIds += query.id
    +        queryId = query.id
           }
    -
           query.start()
    --- End diff --
    
    add comment explaining why this should not be inside a lock. so that this is less likely
to regress in future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


Mime
View raw message