Skip to content

Commit 7548a88

Browse files
HeartSaVioRsrowen
authored andcommitted
[SPARK-28199][SS] Move Trigger implementations to Triggers.scala and avoid exposing these to the end users
## What changes were proposed in this pull request? This patch proposes moving all Trigger implementations to `Triggers.scala`, to avoid exposing these implementations to the end users and let end users only deal with `Trigger.xxx` static methods. This fits the intention of deprecation of `ProcessingTIme`, and we agree to move others without deprecation as this patch will be shipped in major version (Spark 3.0.0). ## How was this patch tested? UTs modified to work with newly introduced class. Closes #24996 from HeartSaVioR/SPARK-28199. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
1 parent 76079fa commit 7548a88

14 files changed

Lines changed: 114 additions & 217 deletions

File tree

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import org.apache.kafka.clients.producer.ProducerRecord
2121

2222
import org.apache.spark.sql.Dataset
2323
import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec
24-
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
24+
import org.apache.spark.sql.execution.streaming.ContinuousTrigger
2525
import org.apache.spark.sql.streaming.Trigger
2626

2727
// Run tests in KafkaSourceSuiteBase in continuous execution mode.

project/MimaExcludes.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -372,7 +372,11 @@ object MimaExcludes {
372372

373373
// [SPARK-26616][MLlib] Expose document frequency in IDFModel
374374
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"),
375-
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf")
375+
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"),
376+
377+
// [SPARK-28199][SS] Remove deprecated ProcessingTime
378+
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime"),
379+
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.ProcessingTime$")
376380
)
377381

378382
// Exclude rules for 2.4.x

sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,10 @@
2020
import java.util.concurrent.TimeUnit;
2121

2222
import org.apache.spark.annotation.Evolving;
23+
import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger;
2324
import scala.concurrent.duration.Duration;
2425

25-
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger;
26+
import org.apache.spark.sql.execution.streaming.ContinuousTrigger;
2627
import org.apache.spark.sql.execution.streaming.OneTimeTrigger$;
2728

2829
/**
@@ -40,7 +41,7 @@ public class Trigger {
4041
* @since 2.2.0
4142
*/
4243
public static Trigger ProcessingTime(long intervalMs) {
43-
return ProcessingTime.create(intervalMs, TimeUnit.MILLISECONDS);
44+
return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS);
4445
}
4546

4647
/**
@@ -56,7 +57,7 @@ public static Trigger ProcessingTime(long intervalMs) {
5657
* @since 2.2.0
5758
*/
5859
public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) {
59-
return ProcessingTime.create(interval, timeUnit);
60+
return ProcessingTimeTrigger.create(interval, timeUnit);
6061
}
6162

6263
/**
@@ -71,7 +72,7 @@ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) {
7172
* @since 2.2.0
7273
*/
7374
public static Trigger ProcessingTime(Duration interval) {
74-
return ProcessingTime.apply(interval);
75+
return ProcessingTimeTrigger.apply(interval);
7576
}
7677

7778
/**
@@ -84,7 +85,7 @@ public static Trigger ProcessingTime(Duration interval) {
8485
* @since 2.2.0
8586
*/
8687
public static Trigger ProcessingTime(String interval) {
87-
return ProcessingTime.apply(interval);
88+
return ProcessingTimeTrigger.apply(interval);
8889
}
8990

9091
/**

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchSt
3030
import org.apache.spark.sql.internal.SQLConf
3131
import org.apache.spark.sql.sources.v2._
3232
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream}
33-
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
33+
import org.apache.spark.sql.streaming.{OutputMode, Trigger}
3434
import org.apache.spark.util.Clock
3535

3636
class MicroBatchExecution(
@@ -51,7 +51,7 @@ class MicroBatchExecution(
5151
@volatile protected var sources: Seq[SparkDataStream] = Seq.empty
5252

5353
private val triggerExecutor = trigger match {
54-
case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock)
54+
case t: ProcessingTimeTrigger => ProcessingTimeExecutor(t, triggerClock)
5555
case OneTimeTrigger => OneTimeExecutor()
5656
case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger")
5757
}

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.spark.sql.execution.streaming
1919

2020
import org.apache.spark.internal.Logging
21-
import org.apache.spark.sql.streaming.ProcessingTime
2221
import org.apache.spark.util.{Clock, SystemClock}
2322

2423
trait TriggerExecutor {
@@ -43,10 +42,12 @@ case class OneTimeExecutor() extends TriggerExecutor {
4342
/**
4443
* A trigger executor that runs a batch every `intervalMs` milliseconds.
4544
*/
46-
case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = new SystemClock())
45+
case class ProcessingTimeExecutor(
46+
processingTimeTrigger: ProcessingTimeTrigger,
47+
clock: Clock = new SystemClock())
4748
extends TriggerExecutor with Logging {
4849

49-
private val intervalMs = processingTime.intervalMs
50+
private val intervalMs = processingTimeTrigger.intervalMs
5051
require(intervalMs >= 0)
5152

5253
override def execute(triggerHandler: () => Boolean): Unit = {

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala

Lines changed: 82 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,94 @@
1717

1818
package org.apache.spark.sql.execution.streaming
1919

20+
import java.util.concurrent.TimeUnit
21+
22+
import scala.concurrent.duration.Duration
23+
2024
import org.apache.spark.annotation.{Evolving, Experimental}
2125
import org.apache.spark.sql.streaming.Trigger
26+
import org.apache.spark.unsafe.types.CalendarInterval
27+
28+
private object Triggers {
29+
def validate(intervalMs: Long): Unit = {
30+
require(intervalMs >= 0, "the interval of trigger should not be negative")
31+
}
32+
33+
def convert(interval: String): Long = {
34+
val cal = CalendarInterval.fromCaseInsensitiveString(interval)
35+
if (cal.months > 0) {
36+
throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval")
37+
}
38+
TimeUnit.MICROSECONDS.toMillis(cal.microseconds)
39+
}
40+
41+
def convert(interval: Duration): Long = interval.toMillis
42+
43+
def convert(interval: Long, unit: TimeUnit): Long = unit.toMillis(interval)
44+
}
2245

2346
/**
2447
* A [[Trigger]] that processes only one batch of data in a streaming query then terminates
2548
* the query.
2649
*/
2750
@Experimental
2851
@Evolving
29-
case object OneTimeTrigger extends Trigger
52+
private[sql] case object OneTimeTrigger extends Trigger
53+
54+
/**
55+
* A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0,
56+
* the query will run as fast as possible.
57+
*/
58+
@Evolving
59+
private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger {
60+
Triggers.validate(intervalMs)
61+
}
62+
63+
private[sql] object ProcessingTimeTrigger {
64+
import Triggers._
65+
66+
def apply(interval: String): ProcessingTimeTrigger = {
67+
ProcessingTimeTrigger(convert(interval))
68+
}
69+
70+
def apply(interval: Duration): ProcessingTimeTrigger = {
71+
ProcessingTimeTrigger(convert(interval))
72+
}
73+
74+
def create(interval: String): ProcessingTimeTrigger = {
75+
apply(interval)
76+
}
77+
78+
def create(interval: Long, unit: TimeUnit): ProcessingTimeTrigger = {
79+
ProcessingTimeTrigger(convert(interval, unit))
80+
}
81+
}
82+
83+
/**
84+
* A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at
85+
* the specified interval.
86+
*/
87+
@Evolving
88+
private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger {
89+
Triggers.validate(intervalMs)
90+
}
91+
92+
private[sql] object ContinuousTrigger {
93+
import Triggers._
94+
95+
def apply(interval: String): ContinuousTrigger = {
96+
ContinuousTrigger(convert(interval))
97+
}
98+
99+
def apply(interval: Duration): ContinuousTrigger = {
100+
ContinuousTrigger(convert(interval))
101+
}
102+
103+
def create(interval: String): ContinuousTrigger = {
104+
apply(interval)
105+
}
106+
107+
def create(interval: Long, unit: TimeUnit): ContinuousTrigger = {
108+
ContinuousTrigger(convert(interval, unit))
109+
}
110+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _}
3434
import org.apache.spark.sql.sources.v2
3535
import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability}
3636
import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset}
37-
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger}
37+
import org.apache.spark.sql.streaming.{OutputMode, Trigger}
3838
import org.apache.spark.util.Clock
3939

4040
class ContinuousExecution(
@@ -93,7 +93,7 @@ class ContinuousExecution(
9393
}
9494

9595
private val triggerExecutor = trigger match {
96-
case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTime(t), triggerClock)
96+
case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTimeTrigger(t), triggerClock)
9797
case _ => throw new IllegalStateException(s"Unsupported type of trigger: $trigger")
9898
}
9999

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala

Lines changed: 0 additions & 57 deletions
This file was deleted.

sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.command.DDLUtils
2929
import org.apache.spark.sql.execution.datasources.DataSource
3030
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
3131
import org.apache.spark.sql.execution.streaming._
32-
import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger
3332
import org.apache.spark.sql.execution.streaming.sources._
3433
import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider}
3534
import org.apache.spark.sql.sources.v2.TableCapability._

0 commit comments

Comments
 (0)