Github user joseph-torres commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19984#discussion_r158159270
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala
 ---
    @@ -0,0 +1,343 @@
    +/*
    + * 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.continuous
    +
    +import java.util.concurrent.TimeUnit
    +
    +import scala.collection.mutable.{ArrayBuffer, Map => MutableMap}
    +
    +import org.apache.spark.SparkEnv
    +import org.apache.spark.sql.{AnalysisException, SparkSession}
    +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, 
CurrentBatchTimestamp, CurrentDate, CurrentTimestamp}
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.sql.execution.SQLExecution
    +import 
org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, 
WriteToDataSourceV2}
    +import 
org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, 
StreamingRelationV2, _}
    +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, 
ContinuousWriteSupport, DataSourceV2Options}
    +import org.apache.spark.sql.sources.v2.reader.{ContinuousReader, Offset, 
PartitionOffset}
    +import org.apache.spark.sql.sources.v2.writer.ContinuousWriter
    +import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
    +import org.apache.spark.sql.types.StructType
    +import org.apache.spark.util.{Clock, Utils}
    +
    +class ContinuousExecution(
    +    sparkSession: SparkSession,
    +    name: String,
    +    checkpointRoot: String,
    +    analyzedPlan: LogicalPlan,
    +    sink: ContinuousWriteSupport,
    +    trigger: Trigger,
    +    triggerClock: Clock,
    +    outputMode: OutputMode,
    +    extraOptions: Map[String, String],
    +    deleteCheckpointOnStop: Boolean)
    +  extends StreamExecution(
    +    sparkSession, name, checkpointRoot, analyzedPlan, sink,
    +    trigger, triggerClock, outputMode, deleteCheckpointOnStop) {
    +
    +  @volatile protected var continuousSources: Seq[ContinuousReader] = 
Seq.empty
    +  override protected def sources: Seq[BaseStreamingSource] = 
continuousSources
    +
    +  override lazy val logicalPlan: LogicalPlan = {
    +    assert(queryExecutionThread eq Thread.currentThread,
    +      "logicalPlan must be initialized in StreamExecutionThread " +
    +        s"but the current thread was ${Thread.currentThread}")
    +    var nextSourceId = 0L
    +    val toExecutionRelationMap = MutableMap[StreamingRelationV2, 
ContinuousExecutionRelation]()
    +    analyzedPlan.transform {
    +      case r @ StreamingRelationV2(
    +          source: ContinuousReadSupport, _, extraReaderOptions, output, _) 
=>
    +        toExecutionRelationMap.getOrElseUpdate(r, {
    +          ContinuousExecutionRelation(source, extraReaderOptions, 
output)(sparkSession)
    +        })
    +      case StreamingRelationV2(_, sourceName, _, _, _) =>
    +        throw new AnalysisException(
    +          s"Data source $sourceName does not support continuous 
processing.")
    +    }
    +  }
    +
    +  private val triggerExecutor = trigger match {
    +    case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTime(t), 
triggerClock)
    --- End diff --
    
    I think the coupling is correct here. ProcessingTime represents the rate of 
progress through the query's fenceposts, which applies here as well as it does 
in the microbatch case.


---

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

Reply via email to