-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-14176][SQL]Add DataFrameWriter.trigger to set the stream batch period #11976
Changes from all commits
bf5d675
ab5dbd3
6f5c6ed
92d204c
a7355ed
f3526d0
7c4bc42
6c1b382
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* 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 | ||
|
||
import java.util.concurrent.TimeUnit | ||
|
||
import scala.concurrent.duration.Duration | ||
|
||
import org.apache.commons.lang3.StringUtils | ||
|
||
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.unsafe.types.CalendarInterval | ||
|
||
/** | ||
* :: Experimental :: | ||
* Used to indicate how often results should be produced by a [[ContinuousQuery]]. | ||
*/ | ||
@Experimental | ||
sealed trait Trigger {} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. everything here should be @experimental |
||
|
||
/** | ||
* :: Experimental :: | ||
* A trigger that runs a query periodically based on the processing time. If `intervalMs` is 0, | ||
* the query will run as fast as possible. | ||
* | ||
* Scala Example: | ||
* {{{ | ||
* def.writer.trigger(ProcessingTime("10 seconds")) | ||
* | ||
* import scala.concurrent.duration._ | ||
* def.writer.trigger(ProcessingTime(10.seconds)) | ||
* }}} | ||
* | ||
* Java Example: | ||
* {{{ | ||
* def.writer.trigger(ProcessingTime.create("10 seconds")) | ||
* | ||
* import java.util.concurrent.TimeUnit | ||
* def.writer.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) | ||
* }}} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nice documentation! Maybe put the typesafe one second and include the |
||
*/ | ||
@Experimental | ||
case class ProcessingTime(intervalMs: Long) extends Trigger { | ||
require(intervalMs >= 0, "the interval of trigger should not be negative") | ||
} | ||
|
||
/** | ||
* :: Experimental :: | ||
* Used to create [[ProcessingTime]] triggers for [[ContinuousQuery]]s. | ||
*/ | ||
@Experimental | ||
object ProcessingTime { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Used to create [[ProcessingTime]] triggers for [[ContinuousQueries]]. Or something. |
||
|
||
/** | ||
* Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. | ||
* | ||
* Example: | ||
* {{{ | ||
* def.writer.trigger(ProcessingTime("10 seconds")) | ||
* }}} | ||
*/ | ||
def apply(interval: String): ProcessingTime = { | ||
if (StringUtils.isBlank(interval)) { | ||
throw new IllegalArgumentException( | ||
"interval cannot be null or blank.") | ||
} | ||
val cal = if (interval.startsWith("interval")) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this logic duplicated elsewhere? Should There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
SQL also uses There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The presence of |
||
CalendarInterval.fromString(interval) | ||
} else { | ||
CalendarInterval.fromString("interval " + interval) | ||
} | ||
if (cal == null) { | ||
throw new IllegalArgumentException(s"Invalid interval: $interval") | ||
} | ||
if (cal.months > 0) { | ||
throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") | ||
} | ||
new ProcessingTime(cal.microseconds / 1000) | ||
} | ||
|
||
/** | ||
* Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. | ||
* | ||
* Example: | ||
* {{{ | ||
* import scala.concurrent.duration._ | ||
* def.writer.trigger(ProcessingTime(10.seconds)) | ||
* }}} | ||
*/ | ||
def apply(interval: Duration): ProcessingTime = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Needs docs with examples |
||
new ProcessingTime(interval.toMillis) | ||
} | ||
|
||
/** | ||
* Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. | ||
* | ||
* Example: | ||
* {{{ | ||
* def.writer.trigger(ProcessingTime.create("10 seconds")) | ||
* }}} | ||
*/ | ||
def create(interval: String): ProcessingTime = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Needs scala docs with examples |
||
apply(interval) | ||
} | ||
|
||
/** | ||
* Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. | ||
* | ||
* Example: | ||
* {{{ | ||
* import java.util.concurrent.TimeUnit | ||
* def.writer.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) | ||
* }}} | ||
*/ | ||
def create(interval: Long, unit: TimeUnit): ProcessingTime = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Needs scala docs with examples |
||
new ProcessingTime(unit.toMillis(interval)) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* 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 org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.ProcessingTime | ||
import org.apache.spark.util.{Clock, SystemClock} | ||
|
||
trait TriggerExecutor { | ||
|
||
/** | ||
* Execute batches using `batchRunner`. If `batchRunner` runs `false`, terminate the execution. | ||
*/ | ||
def execute(batchRunner: () => Boolean): Unit | ||
} | ||
|
||
/** | ||
* A trigger executor that runs a batch every `intervalMs` milliseconds. | ||
*/ | ||
case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = new SystemClock()) | ||
extends TriggerExecutor with Logging { | ||
|
||
private val intervalMs = processingTime.intervalMs | ||
|
||
override def execute(batchRunner: () => Boolean): Unit = { | ||
while (true) { | ||
val batchStartTimeMs = clock.getTimeMillis() | ||
val terminated = !batchRunner() | ||
if (intervalMs > 0) { | ||
val batchEndTimeMs = clock.getTimeMillis() | ||
val batchElapsedTimeMs = batchEndTimeMs - batchStartTimeMs | ||
if (batchElapsedTimeMs > intervalMs) { | ||
notifyBatchFallingBehind(batchElapsedTimeMs) | ||
} | ||
if (terminated) { | ||
return | ||
} | ||
clock.waitTillTime(nextBatchTime(batchEndTimeMs)) | ||
} else { | ||
if (terminated) { | ||
return | ||
} | ||
} | ||
} | ||
} | ||
|
||
/** Called when a batch falls behind. Expose for test only */ | ||
def notifyBatchFallingBehind(realElapsedTimeMs: Long): Unit = { | ||
logWarning("Current batch is falling behind. The trigger interval is " + | ||
s"${intervalMs} milliseconds, but spent ${realElapsedTimeMs} milliseconds") | ||
} | ||
|
||
/** Return the next multiple of intervalMs */ | ||
def nextBatchTime(now: Long): Long = { | ||
(now - 1) / intervalMs * intervalMs + intervalMs | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* 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 | ||
|
||
import java.util.concurrent.TimeUnit | ||
|
||
import scala.concurrent.duration._ | ||
|
||
import org.apache.spark.SparkFunSuite | ||
|
||
class ProcessingTimeSuite extends SparkFunSuite { | ||
|
||
test("create") { | ||
assert(ProcessingTime(10.seconds).intervalMs === 10 * 1000) | ||
assert(ProcessingTime.create(10, TimeUnit.SECONDS).intervalMs === 10 * 1000) | ||
assert(ProcessingTime("1 minute").intervalMs === 60 * 1000) | ||
assert(ProcessingTime("interval 1 minute").intervalMs === 60 * 1000) | ||
|
||
intercept[IllegalArgumentException] { ProcessingTime(null: String) } | ||
intercept[IllegalArgumentException] { ProcessingTime("") } | ||
intercept[IllegalArgumentException] { ProcessingTime("invalid") } | ||
intercept[IllegalArgumentException] { ProcessingTime("1 month") } | ||
intercept[IllegalArgumentException] { ProcessingTime("1 year") } | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This scala doc should have an example right here.