Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
54 changes: 54 additions & 0 deletions sql/api/src/main/java/org/apache/spark/sql/streaming/Trigger.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,12 @@
import scala.concurrent.duration.Duration;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.annotation.Experimental;
import org.apache.spark.sql.execution.streaming.AvailableNowTrigger$;
import org.apache.spark.sql.execution.streaming.ContinuousTrigger;
import org.apache.spark.sql.execution.streaming.OneTimeTrigger$;
import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger;
import org.apache.spark.sql.execution.streaming.RealTimeTrigger;

/**
* Policy used to indicate how often results should be produced by a [[StreamingQuery]].
Expand Down Expand Up @@ -176,4 +178,56 @@ public static Trigger Continuous(Duration interval) {
public static Trigger Continuous(String interval) {
return ContinuousTrigger.apply(interval);
}

/**
* A trigger for real time mode, with batch at the specified duration.
*
*/
@Experimental
public static Trigger RealTime(long batchDurationMs) {
return RealTimeTrigger.apply(batchDurationMs);
}

/**
* A trigger for real time mode, with batch at the specified duration.
*
*/
@Experimental
public static Trigger RealTime(long batchDuration, TimeUnit timeUnit) {
return RealTimeTrigger.create(batchDuration, timeUnit);
}

/**
* A trigger for real time mode, with batch at the specified duration.
*
* {{{
* import scala.concurrent.duration._
* df.writeStream.trigger(Trigger.RealTime(10.seconds))
* }}}
*/
@Experimental
public static Trigger RealTime(Duration batchDuration) {
return RealTimeTrigger.apply(batchDuration);
}

/**
* A trigger for real time mode, with batch at the specified duration.
*
* {{{
* df.writeStream.trigger(Trigger.RealTime("10 seconds"))
* }}}
*/
@Experimental
public static Trigger RealTime(String batchDuration) {
return RealTimeTrigger.apply(batchDuration);
}

/**
* A trigger for real time mode, with batch at the specified duration. The default duration is 5
* minutes.
*/
@Experimental
public static Trigger RealTime() {
return RealTimeTrigger.apply();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,12 @@ package org.apache.spark.sql.execution.streaming

import java.util.concurrent.TimeUnit

import scala.concurrent.duration.Duration
import scala.concurrent.duration.{Duration, MINUTES}

import org.json4s.DefaultFormats

import org.apache.spark.SparkIllegalArgumentException
import org.apache.spark.annotation.Experimental
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY
import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.microsToMillis
import org.apache.spark.sql.catalyst.util.SparkIntervalUtils
Expand Down Expand Up @@ -114,3 +117,40 @@ object ContinuousTrigger {
ContinuousTrigger(convert(interval, unit))
}
}

/**
* A [[Trigger]] that runs a query in real time mode.
* @param batchDurationMs
* The duration of each batch in milliseconds. This must be strictly positive.
*/
@Experimental
case class RealTimeTrigger(batchDurationMs: Long) extends Trigger {
require(batchDurationMs > 0, "the batch duration should not be negative")

implicit val defaultFormats: DefaultFormats = DefaultFormats
}

@Experimental
object RealTimeTrigger {
import Triggers._

def apply(): RealTimeTrigger = {
RealTimeTrigger(Duration(5, MINUTES))
}

def apply(batchDuration: String): RealTimeTrigger = {
RealTimeTrigger(convert(batchDuration))
}

def apply(batchDuration: Duration): RealTimeTrigger = {
RealTimeTrigger(convert(batchDuration))
}

def create(batchDuration: String): RealTimeTrigger = {
apply(batchDuration)
}

def create(batchDuration: Long, unit: TimeUnit): RealTimeTrigger = {
RealTimeTrigger(convert(batchDuration, unit))
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.streaming

import java.util.concurrent.TimeUnit

import scala.concurrent.duration.Duration

import org.apache.spark.sql.execution.streaming.RealTimeTrigger

class StreamRealTimeModeSuite extends StreamTest {

test("test trigger") {
def testTrigger(trigger: Trigger, actual: Long): Unit = {
val realTimeTrigger = trigger.asInstanceOf[RealTimeTrigger]
assert(
realTimeTrigger.batchDurationMs == actual,
s"Real time trigger duration should be ${actual} ms" +
s" but got ${realTimeTrigger.batchDurationMs} ms"
)
}

// test default
testTrigger(Trigger.RealTime(), 300000)

testTrigger(Trigger.RealTime("1 second"), 1000)
testTrigger(Trigger.RealTime("1 minute"), 60000)
testTrigger(Trigger.RealTime("1 hour"), 3600000)
testTrigger(Trigger.RealTime("1 day"), 86400000)
testTrigger(Trigger.RealTime("1 week"), 604800000)

testTrigger(Trigger.RealTime(1000), 1000)
testTrigger(Trigger.RealTime(60000), 60000)
testTrigger(Trigger.RealTime(3600000), 3600000)
testTrigger(Trigger.RealTime(86400000), 86400000)
testTrigger(Trigger.RealTime(604800000), 604800000)

testTrigger(Trigger.RealTime(Duration.apply(1000, "ms")), 1000)
testTrigger(Trigger.RealTime(Duration.apply(60, "s")), 60000)
testTrigger(Trigger.RealTime(Duration.apply(1, "h")), 3600000)
testTrigger(Trigger.RealTime(Duration.apply(1, "d")), 86400000)

testTrigger(Trigger.RealTime(1000, TimeUnit.MILLISECONDS), 1000)
testTrigger(Trigger.RealTime(60, TimeUnit.SECONDS), 60000)
testTrigger(Trigger.RealTime(1, TimeUnit.HOURS), 3600000)
testTrigger(Trigger.RealTime(1, TimeUnit.DAYS), 86400000)
}
}