From 168568f9e231827dd1a00eaf2f470592a5aa7ded Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sat, 26 Mar 2016 14:02:30 -0700 Subject: [PATCH 01/21] time windowing --- .../sql/catalyst/analysis/Analyzer.scala | 33 ++++++++++++++ .../sql/catalyst/expressions/TimeWindow.scala | 44 +++++++++++++++++++ .../org/apache/spark/sql/functions.scala | 16 +++++++ 3 files changed, 93 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3b83e680184a5..3fce7691b6aa2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1585,3 +1585,36 @@ object ResolveUpCast extends Rule[LogicalPlan] { } } } + +/** + * Replace the `UpCast` expression by `Cast`, and throw exceptions if the cast may truncate. + */ +object TimeWindowing extends Rule[LogicalPlan] { + + private def generateWindows(p: LogicalPlan): (LogicalPlan, AttributeReference) = { + val windowExpr = p.expressions.collect { case window: TimeWindow => window }.head + val expandedWindow = AttributeReference("window", StructType(Seq( + StructField("start", TimestampType), StructField("end", TimestampType))))() + val projections = Seq.tabulate(windowExpr.numOverlapping) { i => + + ??? + } + (Expand(projections, expandedWindow :: Nil, p.children.head), expandedWindow) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p: LogicalPlan if p.expressions.exists(_.isInstanceOf[TimeWindow]) && + p.children.length == 1 => + val (windowed, columnRef) = generateWindows(p) + val rewritten = p transformExpressions { + case windowExpr: TimeWindow => + windowExpr.validate() match { + case Some(e) => throw new AnalysisException(e) + case _ => // valid expression + } + columnRef + } + rewritten.withNewChildren(windowed :: Nil) + } + +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala new file mode 100644 index 0000000000000..0724a698167fe --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -0,0 +1,44 @@ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +case class TimeWindow( + timeColumn: Expression, + private val _windowDuration: String, + private val _slideDuration: String, + private val _startTime: String) extends UnaryExpression + with ExpectsInputTypes + with Unevaluable + with NonSQLExpression { + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def child: Expression = timeColumn + override def dataType: DataType = StructType(Seq( + StructField("start", TimestampType), StructField("end", TimestampType))) + + private def getIntervalInMillis(interval: CalendarInterval): Long = { + (interval.months * 4 * CalendarInterval.MICROS_PER_WEEK + interval.microseconds) / 1000 + } + + val windowDuration = getIntervalInMillis(CalendarInterval.fromString(_windowDuration)) + val slideDuration = getIntervalInMillis(CalendarInterval.fromString(_slideDuration)) + val startTime = getIntervalInMillis(CalendarInterval.fromString(_startTime)) + + def validate(): Option[String] = { + if (slideDuration > windowDuration) { + return Some(s"The slide duration ($slideDuration) must be less than or equal to the " + + s"windowDuration ($windowDuration).") + } + if (startTime >= windowDuration) { + return Some(s"The start time ($startTime) must be less than the " + + s"windowDuration ($windowDuration).") + } + None + } + + /** + * Returns number of overlapping windows we will have with the given window and slide durations. + */ + def numOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 8abb9d7e4a1f0..750eac7034360 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1153,6 +1153,22 @@ object functions { CaseWhen(Seq((condition.expr, lit(value).expr))) } + def window( + timeColumn: Column, + windowDuration: String, + slideDuration: String, + startTime: String): Column = withExpr { + ??? + } + + def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { + window(timeColumn, windowDuration, slideDuration, "0s") + } + + def window(timeColumn: Column, windowDuration: String): Column = { + window(timeColumn, windowDuration, windowDuration, "0s") + } + /** * Computes bitwise NOT. * From df7bce039b02de75fbc4dcff521dc21ee3f86d18 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sat, 26 Mar 2016 16:31:47 -0700 Subject: [PATCH 02/21] finished adding time windowing --- .../sql/catalyst/analysis/Analyzer.scala | 30 +++++++++++++++---- .../sql/catalyst/expressions/TimeWindow.scala | 15 +++++++--- .../org/apache/spark/sql/functions.scala | 2 +- 3 files changed, 37 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3fce7691b6aa2..5f785b6e3b40f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -77,6 +77,7 @@ class Analyzer( Batch("Substitution", fixedPoint, CTESubstitution, WindowsSubstitution, + TimeWindowing, EliminateUnions), Batch("Resolution", fixedPoint, ResolveRelations :: @@ -1587,19 +1588,38 @@ object ResolveUpCast extends Rule[LogicalPlan] { } /** - * Replace the `UpCast` expression by `Cast`, and throw exceptions if the cast may truncate. + * Maps a time column to multiple time windows using the Expand operator. Since it's non-trivial to + * figure out how many windows a time column can map to, we over-estimate the number of windows and + * filter out the rows where the time column is not inside the time window. */ object TimeWindowing extends Rule[LogicalPlan] { + private def addSlideDurationAndOffset( + expr: Expression, + windowExpr: TimeWindow, + window: Int): Expression = { + Add(Add(expr, Multiply(Literal(window), Literal(windowExpr.slideDuration))), + Literal(windowExpr.startTime)) + } + private def generateWindows(p: LogicalPlan): (LogicalPlan, AttributeReference) = { val windowExpr = p.expressions.collect { case window: TimeWindow => window }.head val expandedWindow = AttributeReference("window", StructType(Seq( StructField("start", TimestampType), StructField("end", TimestampType))))() - val projections = Seq.tabulate(windowExpr.numOverlapping) { i => - - ??? + val projections = Seq.tabulate(windowExpr.maxNumOverlapping + 1) { i => + val division = Divide(windowExpr.timeColumn, Literal(windowExpr.windowDuration)) + val windowStart = addSlideDurationAndOffset(Multiply(Floor(division), + Literal(windowExpr.windowDuration)), windowExpr, i - 1) + val windowEnd = addSlideDurationAndOffset(Multiply(Ceil(division), + Literal(windowExpr.windowDuration)), windowExpr, i - 1) + CreateStruct(windowStart :: windowEnd :: Nil) :: Nil } - (Expand(projections, expandedWindow :: Nil, p.children.head), expandedWindow) + val windowStartCol = expandedWindow.children.head + val windowEndCol = expandedWindow.children.last + val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowStartCol), + LessThan(windowExpr.timeColumn, windowEndCol)) + (Filter(filterExpr, + Expand(projections, expandedWindow :: Nil, p.children.head)), expandedWindow) } def apply(plan: LogicalPlan): LogicalPlan = plan transform { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 0724a698167fe..2cce4830aecb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -26,19 +26,26 @@ case class TimeWindow( val startTime = getIntervalInMillis(CalendarInterval.fromString(_startTime)) def validate(): Option[String] = { + if (windowDuration <= 0) { + return Some(s"The window duration ($windowDuration) must be greater than 0.") + } + if (slideDuration <= 0) { + return Some(s"The slide duration ($slideDuration) must be greater than 0.") + } if (slideDuration > windowDuration) { return Some(s"The slide duration ($slideDuration) must be less than or equal to the " + s"windowDuration ($windowDuration).") } - if (startTime >= windowDuration) { + if (startTime >= slideDuration) { return Some(s"The start time ($startTime) must be less than the " + - s"windowDuration ($windowDuration).") + s"slideDuration ($slideDuration).") } None } /** - * Returns number of overlapping windows we will have with the given window and slide durations. + * Returns the maximum possible number of overlapping windows we will have with the given + * window and slide durations. */ - def numOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt + def maxNumOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 750eac7034360..022249e6d4c2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1158,7 +1158,7 @@ object functions { windowDuration: String, slideDuration: String, startTime: String): Column = withExpr { - ??? + TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) } def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { From 03c14563786496276ecadb2951f818add4190396 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 27 Mar 2016 12:56:33 -0700 Subject: [PATCH 03/21] works --- .../sql/catalyst/analysis/Analyzer.scala | 75 +++++++++++-------- .../sql/catalyst/expressions/TimeWindow.scala | 36 +++++---- .../org/apache/spark/sql/functions.scala | 4 +- .../spark/sql/DataFrameFunctionsSuite.scala | 9 +++ 4 files changed, 77 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5f785b6e3b40f..8a4d24b1c8802 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -77,7 +77,6 @@ class Analyzer( Batch("Substitution", fixedPoint, CTESubstitution, WindowsSubstitution, - TimeWindowing, EliminateUnions), Batch("Resolution", fixedPoint, ResolveRelations :: @@ -94,6 +93,7 @@ class Analyzer( ResolveWindowOrder :: ResolveWindowFrame :: ResolveNaturalAndUsingJoin :: + TimeWindowing :: ExtractWindowExpressions :: GlobalAggregates :: ResolveAggregateFunctions :: @@ -1492,7 +1492,7 @@ object EliminateUnions extends Rule[LogicalPlan] { * Window(window expressions). */ object CleanupAliases extends Rule[LogicalPlan] { - private def trimAliases(e: Expression): Expression = { + private[catalyst] def trimAliases(e: Expression): Expression = { var stop = false e.transformDown { // CreateStruct is a special case, we need to retain its top level Aliases as they decide the @@ -1598,43 +1598,54 @@ object TimeWindowing extends Rule[LogicalPlan] { expr: Expression, windowExpr: TimeWindow, window: Int): Expression = { - Add(Add(expr, Multiply(Literal(window), Literal(windowExpr.slideDuration))), - Literal(windowExpr.startTime)) + Multiply(Add(Add(expr, Multiply(Literal(window), Literal(windowExpr.slideDuration))), + Literal(windowExpr.startTime)), Literal(1000000)) } - private def generateWindows(p: LogicalPlan): (LogicalPlan, AttributeReference) = { - val windowExpr = p.expressions.collect { case window: TimeWindow => window }.head - val expandedWindow = AttributeReference("window", StructType(Seq( - StructField("start", TimestampType), StructField("end", TimestampType))))() - val projections = Seq.tabulate(windowExpr.maxNumOverlapping + 1) { i => - val division = Divide(windowExpr.timeColumn, Literal(windowExpr.windowDuration)) - val windowStart = addSlideDurationAndOffset(Multiply(Floor(division), - Literal(windowExpr.windowDuration)), windowExpr, i - 1) - val windowEnd = addSlideDurationAndOffset(Multiply(Ceil(division), - Literal(windowExpr.windowDuration)), windowExpr, i - 1) - CreateStruct(windowStart :: windowEnd :: Nil) :: Nil + private def generateWindows(p: LogicalPlan): (LogicalPlan, NamedExpression) = { + val windowExpr = p.expressions.collect { case Alias(child, _) => + child.find(_.isInstanceOf[TimeWindow]) + }.flatten.head.asInstanceOf[TimeWindow] + val windowStruct = StructType(Seq( + StructField("start", TimestampType), StructField("end", TimestampType))) + val windowAttributes = windowStruct.toAttributes + val expandedWindow = Alias(CreateStruct(windowAttributes), "window")() + val projections = Seq.tabulate(windowExpr.maxNumOverlapping) { i => + val division = Divide(Cast(windowExpr.timeColumn, LongType), + Literal(windowExpr.slideDuration)) + val windowStart = addSlideDurationAndOffset(Multiply(Ceil(division), + Literal(windowExpr.slideDuration)), windowExpr, i - windowExpr.maxNumOverlapping) + val windowEnd = + Add(windowStart, Multiply(Literal(windowExpr.windowDuration), Literal(1000000))) + windowStart :: windowEnd :: windowExpr._timeColumn :: Nil } - val windowStartCol = expandedWindow.children.head - val windowEndCol = expandedWindow.children.last + val timeCol = windowExpr._timeColumn.references.toSeq + val windowStartCol = windowAttributes.head + val windowEndCol = windowAttributes.last val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowStartCol), LessThan(windowExpr.timeColumn, windowEndCol)) - (Filter(filterExpr, - Expand(projections, expandedWindow :: Nil, p.children.head)), expandedWindow) + (Project(windowAttributes ++ Seq(expandedWindow) ++ timeCol, + Filter(filterExpr, + Expand(projections, windowAttributes ++ timeCol, p.children.head))), + expandedWindow) } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p: LogicalPlan if p.expressions.exists(_.isInstanceOf[TimeWindow]) && - p.children.length == 1 => - val (windowed, columnRef) = generateWindows(p) - val rewritten = p transformExpressions { - case windowExpr: TimeWindow => - windowExpr.validate() match { - case Some(e) => throw new AnalysisException(e) - case _ => // valid expression - } - columnRef - } - rewritten.withNewChildren(windowed :: Nil) + def apply(plan: LogicalPlan): LogicalPlan = { + val transformed = plan transform { + case p: LogicalPlan if p.expressions.exists(_.find(_.isInstanceOf[TimeWindow]).isDefined) && + p.children.length == 1 => + val (windowed, columnRef) = generateWindows(p) + val rewritten = p transformExpressions { + case Alias(windowExpr: TimeWindow, _) => + windowExpr.validate() match { + case Some(e) => throw new AnalysisException(e) + case _ => // valid expression + } + columnRef + } + rewritten.withNewChildren(windowed :: Nil) + } + transformed } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 2cce4830aecb5..b061dc5a9bd01 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -4,7 +4,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( - timeColumn: Expression, + _timeColumn: Expression, private val _windowDuration: String, private val _slideDuration: String, private val _startTime: String) extends UnaryExpression @@ -12,33 +12,43 @@ case class TimeWindow( with Unevaluable with NonSQLExpression { - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + lazy val timeColumn = Cast(_timeColumn, TimestampType) + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) override def child: Expression = timeColumn override def dataType: DataType = StructType(Seq( StructField("start", TimestampType), StructField("end", TimestampType))) - private def getIntervalInMillis(interval: CalendarInterval): Long = { - (interval.months * 4 * CalendarInterval.MICROS_PER_WEEK + interval.microseconds) / 1000 + private def getIntervalInSeconds(interval: String): Long = { + val intervalString = if (interval.startsWith("interval")) { + interval + } else { + "interval " + interval + } + val cal = CalendarInterval.fromString(intervalString) + (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 } - val windowDuration = getIntervalInMillis(CalendarInterval.fromString(_windowDuration)) - val slideDuration = getIntervalInMillis(CalendarInterval.fromString(_slideDuration)) - val startTime = getIntervalInMillis(CalendarInterval.fromString(_startTime)) + lazy val windowDuration = getIntervalInSeconds(_windowDuration) + lazy val slideDuration = getIntervalInSeconds(_slideDuration) + lazy val startTime = getIntervalInSeconds(_startTime) def validate(): Option[String] = { if (windowDuration <= 0) { - return Some(s"The window duration ($windowDuration) must be greater than 0.") + return Some(s"The window duration (${_windowDuration}) must be greater than 0.") } if (slideDuration <= 0) { - return Some(s"The slide duration ($slideDuration) must be greater than 0.") + return Some(s"The slide duration (${_slideDuration}) must be greater than 0.") + } + if (startTime < 0) { + return Some(s"The start time (${_startTime}) must be greater than or equal to 0.") } if (slideDuration > windowDuration) { - return Some(s"The slide duration ($slideDuration) must be less than or equal to the " + - s"windowDuration ($windowDuration).") + return Some(s"The slide duration (${_slideDuration}) must be less than or equal to the " + + s"windowDuration (${_windowDuration}).") } if (startTime >= slideDuration) { - return Some(s"The start time ($startTime) must be less than the " + - s"slideDuration ($slideDuration).") + return Some(s"The start time (${_startTime}) must be less than the " + + s"slideDuration (${_slideDuration}).") } None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 022249e6d4c2d..2c751c73fce3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1162,11 +1162,11 @@ object functions { } def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { - window(timeColumn, windowDuration, slideDuration, "0s") + window(timeColumn, windowDuration, slideDuration, "0 second") } def window(timeColumn: Column, windowDuration: String): Column = { - window(timeColumn, windowDuration, windowDuration, "0s") + window(timeColumn, windowDuration, windowDuration, "0 second") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 746e25a0c3ec5..562d3ae6abd7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -29,6 +29,14 @@ import org.apache.spark.sql.types._ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ + test("time windowing") { + val df = Seq((1459103954L, 1), (1459103996L, 2)).toDF("time", "b") + // df.select(struct("time", "b")).explain(true) + val windowed = df.select(window($"time", "10 second", "3 second", "2 second"), $"time") + windowed.select($"window.start", $"window.end", from_unixtime($"time")).collect().foreach(println) + } + + /* test("array with column name") { val df = Seq((0, 1)).toDF("a", "b") val row = df.select(array("a", "b")).first() @@ -394,4 +402,5 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(true), Row(true)) ) } + */ } From a41ee4c38f88990278413d7c0e61be517f7cd5cd Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 27 Mar 2016 14:10:47 -0700 Subject: [PATCH 04/21] minor clean up --- .../sql/catalyst/analysis/Analyzer.scala | 52 ++++++++++--------- .../sql/catalyst/expressions/TimeWindow.scala | 20 +++++-- .../spark/sql/DataFrameFunctionsSuite.scala | 8 +-- 3 files changed, 49 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8a4d24b1c8802..a4ad4e566b008 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1602,46 +1602,48 @@ object TimeWindowing extends Rule[LogicalPlan] { Literal(windowExpr.startTime)), Literal(1000000)) } + /** + * Depending on the operation, the TimeWindow expression may be wrapped in an Alias (in case of + * projections) or be simply by itself (in case of groupBy), + * @param f The function that we want to apply on the TimeWindow expression + * @return The user defined function applied on the TimeWindow expression + */ + private def getWindowExpr[E](f: TimeWindow => E): PartialFunction[Expression, E] = { + case Alias(windowExpr: TimeWindow, _) => f(windowExpr) + case windowExpr: TimeWindow => f(windowExpr) + } + private def generateWindows(p: LogicalPlan): (LogicalPlan, NamedExpression) = { - val windowExpr = p.expressions.collect { case Alias(child, _) => - child.find(_.isInstanceOf[TimeWindow]) - }.flatten.head.asInstanceOf[TimeWindow] - val windowStruct = StructType(Seq( - StructField("start", TimestampType), StructField("end", TimestampType))) - val windowAttributes = windowStruct.toAttributes - val expandedWindow = Alias(CreateStruct(windowAttributes), "window")() + val windowExpr = p.expressions.collect(getWindowExpr[TimeWindow](e => e)).head val projections = Seq.tabulate(windowExpr.maxNumOverlapping) { i => - val division = Divide(Cast(windowExpr.timeColumn, LongType), + val division = Divide(Subtract(Cast(windowExpr.timeColumn, LongType), Literal(windowExpr.startTime)), Literal(windowExpr.slideDuration)) val windowStart = addSlideDurationAndOffset(Multiply(Ceil(division), Literal(windowExpr.slideDuration)), windowExpr, i - windowExpr.maxNumOverlapping) val windowEnd = Add(windowStart, Multiply(Literal(windowExpr.windowDuration), Literal(1000000))) - windowStart :: windowEnd :: windowExpr._timeColumn :: Nil + windowStart :: windowEnd :: windowExpr.originalTimeColumn :: Nil } - val timeCol = windowExpr._timeColumn.references.toSeq - val windowStartCol = windowAttributes.head - val windowEndCol = windowAttributes.last - val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowStartCol), - LessThan(windowExpr.timeColumn, windowEndCol)) - (Project(windowAttributes ++ Seq(expandedWindow) ++ timeCol, + val timeCol = windowExpr.originalTimeColumn.references.toSeq + val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowExpr.windowStartCol), + LessThan(windowExpr.timeColumn, windowExpr.windowEndCol)) + (Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol, Filter(filterExpr, - Expand(projections, windowAttributes ++ timeCol, p.children.head))), - expandedWindow) + Expand(projections, windowExpr.output ++ timeCol, p.children.head))), + windowExpr.outputColumn) } def apply(plan: LogicalPlan): LogicalPlan = { val transformed = plan transform { - case p: LogicalPlan if p.expressions.exists(_.find(_.isInstanceOf[TimeWindow]).isDefined) && + case p: LogicalPlan if p.expressions.collect(getWindowExpr[TimeWindow](e => e)).nonEmpty && p.children.length == 1 => val (windowed, columnRef) = generateWindows(p) - val rewritten = p transformExpressions { - case Alias(windowExpr: TimeWindow, _) => - windowExpr.validate() match { - case Some(e) => throw new AnalysisException(e) - case _ => // valid expression - } - columnRef + val rewritten = p transformExpressions getWindowExpr { windowExpr => + windowExpr.validate() match { + case Some(e) => throw new AnalysisException(e) + case _ => // valid expression + } + columnRef } rewritten.withNewChildren(windowed :: Nil) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index b061dc5a9bd01..8fda82f15de48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -4,7 +4,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( - _timeColumn: Expression, + originalTimeColumn: Expression, private val _windowDuration: String, private val _slideDuration: String, private val _startTime: String) extends UnaryExpression @@ -12,12 +12,26 @@ case class TimeWindow( with Unevaluable with NonSQLExpression { - lazy val timeColumn = Cast(_timeColumn, TimestampType) + lazy val timeColumn = Cast(originalTimeColumn, TimestampType) override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) override def child: Expression = timeColumn - override def dataType: DataType = StructType(Seq( + override def dataType: DataType = outputType + + private def outputType: StructType = StructType(Seq( StructField("start", TimestampType), StructField("end", TimestampType))) + lazy val output: Seq[Attribute] = outputType.toAttributes + def outputColumn: NamedExpression = Alias(CreateStruct(output), "window")() + def windowStartCol: Attribute = output.head + def windowEndCol: Attribute = output.last + /** + * Parses the interval string for a valid time duration. CalendarInterval expects interval + * strings to start with the string `interval`. For usability, we prepend `interval` to the string + * if the user ommitted it. + * @param interval The interval string + * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, + * therefore we use seconds here as well. + */ private def getIntervalInSeconds(interval: String): Long = { val intervalString = if (interval.startsWith("interval")) { interval diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 562d3ae6abd7d..283bc0e1840ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -30,10 +30,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("time windowing") { - val df = Seq((1459103954L, 1), (1459103996L, 2)).toDF("time", "b") + val df = Seq((1459103954L, 1), (1459103996L, 2), (1459103967L, 2)).toDF("time", "b") // df.select(struct("time", "b")).explain(true) - val windowed = df.select(window($"time", "10 second", "3 second", "2 second"), $"time") - windowed.select($"window.start", $"window.end", from_unixtime($"time")).collect().foreach(println) + df.select(window($"time", "1 minute", "15 seconds", "5 seconds"), $"time").collect().foreach(println) + println("\n\n") + df.groupBy(window($"time", "1 minute", "15 seconds", "5 seconds")).agg(count("*")).collect().foreach(println) + // windowed.select($"window") } /* From db5046545a8761f289d516518c3e1ff91e551602 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 27 Mar 2016 23:25:38 -0700 Subject: [PATCH 05/21] tests --- .../sql/catalyst/analysis/Analyzer.scala | 66 +++++--- .../sql/catalyst/expressions/TimeWindow.scala | 27 +++- .../expressions/TimeWindowSuite.scala | 75 +++++++++ .../org/apache/spark/sql/functions.scala | 144 ++++++++++++++++-- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- 5 files changed, 275 insertions(+), 39 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a4ad4e566b008..26cb9f4f99558 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1594,14 +1594,6 @@ object ResolveUpCast extends Rule[LogicalPlan] { */ object TimeWindowing extends Rule[LogicalPlan] { - private def addSlideDurationAndOffset( - expr: Expression, - windowExpr: TimeWindow, - window: Int): Expression = { - Multiply(Add(Add(expr, Multiply(Literal(window), Literal(windowExpr.slideDuration))), - Literal(windowExpr.startTime)), Literal(1000000)) - } - /** * Depending on the operation, the TimeWindow expression may be wrapped in an Alias (in case of * projections) or be simply by itself (in case of groupBy), @@ -1613,37 +1605,75 @@ object TimeWindowing extends Rule[LogicalPlan] { case windowExpr: TimeWindow => f(windowExpr) } - private def generateWindows(p: LogicalPlan): (LogicalPlan, NamedExpression) = { + /** + * Generates the logical plan for generating window ranges on a timestamp column. Without + * knowing what the timestamp value is, it's non-trivial to figure out deterministically how many + * window ranges a timestamp will map to given all possible combinations of a window duration, + * slide duration and start time (offset). Therefore, we express and over-estimate the number of + * windows there may be, and filter the valid windows. We use last Project operator to group + * the window columns into a struct so they can be accessed as `window.start` and `window.end`. + * + * The windows are calculated as below: + * maxNumOverlapping <- ceil(windowDuration / slideDuration) + * for (i <- 0 until maxNumOverlapping) + * windowId <- ceil((timestamp - startTime) / slideDuration) + * windowStart <- windowId * slideDuration + (i - maxNumOverlapping) * slideDuration + startTime + * windowEnd <- windowStart + windowDuration + * return windowStart, windowEnd + * + * This behaves as follows for the given parameters for the time: 12:05. The valid windows are + * marked with a +, and invalid ones are marked with a x. The invalid ones are filtered using the + * Filter operator. + * window: 12m, slide: 5m, start: 0m :: window: 12m, slide: 5m, start: 2m + * 11:55 - 12:07 + 11:52 - 12:04 x + * 12:00 - 12:12 + 11:57 - 12:09 + + * 12:05 - 12:17 + 12:02 - 12:14 + + * + * @param p The logical plan + * @return the logical plan that will generate the time windows using the Expand operator, with + * the Filter operator for correctness and Project for usability. + */ + private def generateWindows(p: LogicalPlan): LogicalPlan = { val windowExpr = p.expressions.collect(getWindowExpr[TimeWindow](e => e)).head val projections = Seq.tabulate(windowExpr.maxNumOverlapping) { i => - val division = Divide(Subtract(Cast(windowExpr.timeColumn, LongType), Literal(windowExpr.startTime)), - Literal(windowExpr.slideDuration)) - val windowStart = addSlideDurationAndOffset(Multiply(Ceil(division), - Literal(windowExpr.slideDuration)), windowExpr, i - windowExpr.maxNumOverlapping) + // windowId <- ceil((timestamp - startTime) / slideDuration) + val division = Ceil(Divide(Subtract(Cast(windowExpr.timeColumn, LongType), + Literal(windowExpr.startTime)), Literal(windowExpr.slideDuration))) + // start <- (windowId + i - maxNumOverlapping) * slideDuration + startTime + // the 1000000 is necessary for properly casting a LongType to a TimestampType + val windowStart = + Multiply( + Add( + Multiply( + Literal(windowExpr.slideDuration), + Add(division, Literal(i - windowExpr.maxNumOverlapping))), + Literal(windowExpr.startTime)), + Literal(1000000)) + // windowEnd <- windowStart + windowDuration val windowEnd = Add(windowStart, Multiply(Literal(windowExpr.windowDuration), Literal(1000000))) windowStart :: windowEnd :: windowExpr.originalTimeColumn :: Nil } val timeCol = windowExpr.originalTimeColumn.references.toSeq + // val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowExpr.windowStartCol), LessThan(windowExpr.timeColumn, windowExpr.windowEndCol)) - (Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol, + Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol, Filter(filterExpr, - Expand(projections, windowExpr.output ++ timeCol, p.children.head))), - windowExpr.outputColumn) + Expand(projections, windowExpr.output ++ timeCol, p.children.head))) } def apply(plan: LogicalPlan): LogicalPlan = { val transformed = plan transform { case p: LogicalPlan if p.expressions.collect(getWindowExpr[TimeWindow](e => e)).nonEmpty && p.children.length == 1 => - val (windowed, columnRef) = generateWindows(p) + val windowed = generateWindows(p) val rewritten = p transformExpressions getWindowExpr { windowExpr => windowExpr.validate() match { case Some(e) => throw new AnalysisException(e) case _ => // valid expression } - columnRef + windowExpr.outputColumn } rewritten.withNewChildren(windowed :: Nil) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 8fda82f15de48..cde78f4edb5bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -1,5 +1,8 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.commons.lang.StringUtils + +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -12,8 +15,9 @@ case class TimeWindow( with Unevaluable with NonSQLExpression { - lazy val timeColumn = Cast(originalTimeColumn, TimestampType) override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) + // the time column in Timestamp format + lazy val timeColumn = Cast(originalTimeColumn, TimestampType) override def child: Expression = timeColumn override def dataType: DataType = outputType @@ -33,19 +37,34 @@ case class TimeWindow( * therefore we use seconds here as well. */ private def getIntervalInSeconds(interval: String): Long = { + if (StringUtils.isBlank(interval)) { + throw new AnalysisException( + "The window duration, slide duration and start time cannot be null.") + } val intervalString = if (interval.startsWith("interval")) { interval } else { "interval " + interval } val cal = CalendarInterval.fromString(intervalString) + if (cal == null) { + throw new AnalysisException( + s"The provided interval ($interval) did not correspond to a valid interval string.") + } (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 } - lazy val windowDuration = getIntervalInSeconds(_windowDuration) - lazy val slideDuration = getIntervalInSeconds(_slideDuration) - lazy val startTime = getIntervalInSeconds(_startTime) + // The window duration in seconds + lazy val windowDuration: Long = getIntervalInSeconds(_windowDuration) + // The slide duration in seconds + lazy val slideDuration: Long = getIntervalInSeconds(_slideDuration) + // The start time offset in seconds + lazy val startTime: Long = getIntervalInSeconds(_startTime) + /** + * Validate the inputs for the window duration, slide duration, and start time. + * @return Some string with a useful error message for the invalid input. + */ def validate(): Option[String] = { if (windowDuration <= 0) { return Some(s"The window duration (${_windowDuration}) must be greater than 0.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala new file mode 100644 index 0000000000000..bb58a60bd896a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -0,0 +1,75 @@ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException + +class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("time window is unevaluable") { + intercept[UnsupportedOperationException] { + evaluate(TimeWindow(Literal(10L), "1 second", "1 second", "0 second")) + } + } + + test("validation checks") { + assert(TimeWindow(Literal(10L), "1 second", "1 second", "0 second").validate().isEmpty, + "Is a valid expression. Should not have returned a validation error.") + assert(TimeWindow(Literal(10L), "1 second", "2 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is greater than window.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 minute").validate().isDefined, + "Should have thrown validation error, because start time is greater than slide duration.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 second").validate().isDefined, + "Should have thrown validation error, because start time is equal to slide duration.") + assert(TimeWindow(Literal(10L), "-1 second", "1 second", "0 second").validate().isDefined, + "Should have thrown validation error, because window duration is negative.") + assert(TimeWindow(Literal(10L), "0 second", "0 second", "0 second").validate().isDefined, + "Should have thrown validation error, because window duration is zero.") + assert(TimeWindow(Literal(10L), "1 second", "-1 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is negative.") + assert(TimeWindow(Literal(10L), "1 second", "0 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is zero.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "-2 second").validate().isDefined, + "Should have thrown validation error, because start time is negative.") + } + + test("invalid intervals throw exception") { + val validDuration = "10 second" + val validTime = "5 second" + for (invalid <- Seq(null, " ", "\n", "\t", "2 apples")) { + intercept[AnalysisException] { + TimeWindow(Literal(10L), invalid, validDuration, validTime).windowDuration + } + intercept[AnalysisException] { + TimeWindow(Literal(10L), validDuration, invalid, validTime).slideDuration + } + intercept[AnalysisException] { + TimeWindow(Literal(10L), validDuration, validDuration, invalid).startTime + } + } + } + + test("interval strings work with and without 'interval' prefix and returns seconds") { + val validDuration = "10 second" + for ((text, seconds) <- Seq(("1 second", 1), ("1 minute", 60), ("2 hours", 7200))) { + assert(TimeWindow(Literal(10L), text, validDuration, "0 seconds").windowDuration === seconds) + assert(TimeWindow(Literal(10L), "interval " + text, validDuration, "0 seconds").windowDuration + === seconds) + } + } + + test("maxNumOverlapping takes ceiling of window duration over slide duration") { + assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) + assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) + assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) + assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) + assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) + } + + test("output column name is window and is a struct") { + val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") + assert(expr.outputColumn.name === "window") + assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) + assert(expr.windowStartCol.name === "start") + assert(expr.windowEndCol.name === "end") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 2c751c73fce3b..122493c54fda3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1153,22 +1153,6 @@ object functions { CaseWhen(Seq((condition.expr, lit(value).expr))) } - def window( - timeColumn: Column, - windowDuration: String, - slideDuration: String, - startTime: String): Column = withExpr { - TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) - } - - def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { - window(timeColumn, windowDuration, slideDuration, "0 second") - } - - def window(timeColumn: Column, windowDuration: String): Column = { - window(timeColumn, windowDuration, windowDuration, "0 second") - } - /** * Computes bitwise NOT. * @@ -2567,6 +2551,134 @@ object functions { ToUTCTimestamp(ts.expr, Literal(tz)) } + /** + * Bucketize rows into one or more time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). The following example takes the average stock price + * for a one minute window every 10 seconds starting 5 seconds after the hour: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute", "10 seconds", "5 seconds"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:05-09:01:05 + * 09:00:15-09:01:15 + * 09:00:25-09:01:25 ... + * }}} + * + * For a continuous query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time can be as TimestampType or LongType, however when using LongType, + * the time must be given in seconds. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for + * valid duration identifiers. + * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. + * A new window will be generated every `slideDuration`. Must be less than + * or equal to the `windowDuration`. Check + * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration + * identifiers. + * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start + * window intervals. For example, in order to have hourly tumbling windows that + * start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide + * `startTime` as `15 minutes`. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window( + timeColumn: Column, + windowDuration: String, + slideDuration: String, + startTime: String): Column = withExpr { + TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) + } + + /** + * Bucketize rows into one or more time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). The windows start beginning at 1970-01-01 00:00:00 UTC. + * The following example takes the average stock price for a one minute window every 10 seconds: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute", "10 seconds"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:00-09:01:00 + * 09:00:10-09:01:10 + * 09:00:20-09:01:20 ... + * }}} + * + * For a continuous query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time can be as TimestampType or LongType, however when using LongType, + * the time must be given in seconds. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for + * valid duration identifiers. + * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. + * A new window will be generated every `slideDuration`. Must be less than + * or equal to the `windowDuration`. Check + * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { + window(timeColumn, windowDuration, slideDuration, "0 second") + } + + /** + * Generates tumbling time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). The windows start beginning at 1970-01-01 00:00:00 UTC. + * The following example takes the average stock price for a one minute tumbling window: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:00-09:01:00 + * 09:01:00-09:02:00 + * 09:02:00-09:03:00 ... + * }}} + * + * For a continuous query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time can be as TimestampType or LongType, however when using LongType, + * the time must be given in seconds. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for + * valid duration identifiers. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window(timeColumn: Column, windowDuration: String): Column = { + window(timeColumn, windowDuration, windowDuration, "0 second") + } + ////////////////////////////////////////////////////////////////////////////////////////////// // Collection functions ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 283bc0e1840ba..25e883797d3b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -30,7 +30,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("time windowing") { - val df = Seq((1459103954L, 1), (1459103996L, 2), (1459103967L, 2)).toDF("time", "b") + val df = Seq((1459103996L, 1), (1459103996L, 2), (1459103967L, 2)).toDF("time", "b") // df.select(struct("time", "b")).explain(true) df.select(window($"time", "1 minute", "15 seconds", "5 seconds"), $"time").collect().foreach(println) println("\n\n") From 9e7febb7dcaf9c0cdd312fe6978f9137f1dbaa64 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 11:52:19 -0700 Subject: [PATCH 06/21] finished --- .../sql/catalyst/analysis/Analyzer.scala | 33 ++++++--- .../spark/sql/DataFrameFunctionsSuite.scala | 72 ++++++++++++++++--- 2 files changed, 85 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 26cb9f4f99558..0451cd718abca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -77,7 +77,8 @@ class Analyzer( Batch("Substitution", fixedPoint, CTESubstitution, WindowsSubstitution, - EliminateUnions), + EliminateUnions, + TimeWindowing), Batch("Resolution", fixedPoint, ResolveRelations :: ResolveReferences :: @@ -93,7 +94,6 @@ class Analyzer( ResolveWindowOrder :: ResolveWindowFrame :: ResolveNaturalAndUsingJoin :: - TimeWindowing :: ExtractWindowExpressions :: GlobalAggregates :: ResolveAggregateFunctions :: @@ -1605,6 +1605,16 @@ object TimeWindowing extends Rule[LogicalPlan] { case windowExpr: TimeWindow => f(windowExpr) } + /** Pass on other columns that are required either in a projection or grouping. */ + private def getMissingAttributes(plan: LogicalPlan): Seq[NamedExpression] = { + val expressions = plan match { + case p: Project => p.projectList + case a: Aggregate => a.groupingExpressions + case _ => Seq.empty + } + expressions.flatMap(_.references) + } + /** * Generates the logical plan for generating window ranges on a timestamp column. Without * knowing what the timestamp value is, it's non-trivial to figure out deterministically how many @@ -1635,7 +1645,11 @@ object TimeWindowing extends Rule[LogicalPlan] { */ private def generateWindows(p: LogicalPlan): LogicalPlan = { val windowExpr = p.expressions.collect(getWindowExpr[TimeWindow](e => e)).head - val projections = Seq.tabulate(windowExpr.maxNumOverlapping) { i => + // get all expressions we need to pass on for projections + val otherExpressions = + getMissingAttributes(p).filterNot(_.children.exists(_.isInstanceOf[TimeWindow])) + + val projections = Seq.tabulate(windowExpr.maxNumOverlapping + 1) { i => // windowId <- ceil((timestamp - startTime) / slideDuration) val division = Ceil(Divide(Subtract(Cast(windowExpr.timeColumn, LongType), Literal(windowExpr.startTime)), Literal(windowExpr.slideDuration))) @@ -1645,22 +1659,23 @@ object TimeWindowing extends Rule[LogicalPlan] { Multiply( Add( Multiply( - Literal(windowExpr.slideDuration), - Add(division, Literal(i - windowExpr.maxNumOverlapping))), + Add(division, Literal(i - windowExpr.maxNumOverlapping)), + Literal(windowExpr.slideDuration)), Literal(windowExpr.startTime)), Literal(1000000)) // windowEnd <- windowStart + windowDuration val windowEnd = Add(windowStart, Multiply(Literal(windowExpr.windowDuration), Literal(1000000))) - windowStart :: windowEnd :: windowExpr.originalTimeColumn :: Nil + windowStart :: windowEnd :: windowExpr.originalTimeColumn :: Nil ++ otherExpressions } val timeCol = windowExpr.originalTimeColumn.references.toSeq - // + // timestamp >= window.start && timestamp < window.end val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowExpr.windowStartCol), LessThan(windowExpr.timeColumn, windowExpr.windowEndCol)) - Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol, + Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol ++ otherExpressions, Filter(filterExpr, - Expand(projections, windowExpr.output ++ timeCol, p.children.head))) + Expand(projections, windowExpr.output ++ timeCol ++ otherExpressions.map(_.toAttribute), + p.children.head))) } def apply(plan: LogicalPlan): LogicalPlan = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 25e883797d3b8..d9994b1d09478 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -29,16 +29,6 @@ import org.apache.spark.sql.types._ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ - test("time windowing") { - val df = Seq((1459103996L, 1), (1459103996L, 2), (1459103967L, 2)).toDF("time", "b") - // df.select(struct("time", "b")).explain(true) - df.select(window($"time", "1 minute", "15 seconds", "5 seconds"), $"time").collect().foreach(println) - println("\n\n") - df.groupBy(window($"time", "1 minute", "15 seconds", "5 seconds")).agg(count("*")).collect().foreach(println) - // windowed.select($"window") - } - - /* test("array with column name") { val df = Seq((0, 1)).toDF("a", "b") val row = df.select(array("a", "b")).first() @@ -404,5 +394,65 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(true), Row(true)) ) } - */ + + test("time windowing") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), + (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( + df.groupBy(window($"time", "10 seconds")).agg(count("*").as("counts")) + .orderBy($"window.start".asc).select("counts"), + Seq(Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds")) + .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + Seq(Row(2), Row(1)) + ) + checkAnswer( + df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") + .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + Seq(Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) + .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(1), Row(1), Row(1), Row(2), Row(1), Row(1), Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(4), Row(1), Row(2)) + ) + } + + test("esoteric time windowing use cases") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC + val df = Seq((1459103974L, 1, Seq("a", "b")), + (1459103996L, 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + checkAnswer( + df.select(window($"time", "10 seconds"), $"value", explode($"ids")) + .orderBy($"window.start".asc).select("value"), + // first window exploded to two rows for "a", and "b", second window exploded to 3 rows + Seq(Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + val df2 = Seq(("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), + (null, 3), (null, 4)).toDF("time", "value") + val df3 = Seq(("2016-03-27 09:00:02", 3), ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") + checkAnswer( + df2.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(1), Row(2)) // null columns are dropped + ) + checkAnswer( + df2.select(window($"time", "10 seconds"), $"value").join( + df3.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") + .agg((sum("value") + sum("othervalue")).as("total")) + .orderBy($"window.start".asc).select("total"), + Seq(Row(4), Row(8)) + ) + } } From 535d592b4e7a4ac9bb58b08a9a7ad95ae8232199 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 12:08:42 -0700 Subject: [PATCH 07/21] one more test --- .../org/apache/spark/sql/DataFrameFunctionsSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index d9994b1d09478..fdf0b805134ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -454,5 +454,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { .orderBy($"window.start".asc).select("total"), Seq(Row(4), Row(8)) ) + // should result in negative timestamps in the window + val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") + checkAnswer( + df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(1), Row(2)) + ) } } From 5787e354bbb049d66d3c04b1cfb206beb376cba4 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 12:09:31 -0700 Subject: [PATCH 08/21] added apache headers --- .../sql/catalyst/expressions/TimeWindow.scala | 17 +++++++++++++++++ .../catalyst/expressions/TimeWindowSuite.scala | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index cde78f4edb5bd..453172c1b4bae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -1,3 +1,20 @@ +/* + * 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.catalyst.expressions import org.apache.commons.lang.StringUtils diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index bb58a60bd896a..43afae384be0f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.catalyst.expressions import org.apache.spark.SparkFunSuite From ed9ec34b1d1f19c234c1bea5fa2819de298c050a Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 12:17:14 -0700 Subject: [PATCH 09/21] fix indentation --- .../sql/catalyst/analysis/Analyzer.scala | 27 +++++++++---------- .../sql/catalyst/expressions/TimeWindow.scala | 2 +- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0451cd718abca..fe92be2e9cb6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1492,7 +1492,7 @@ object EliminateUnions extends Rule[LogicalPlan] { * Window(window expressions). */ object CleanupAliases extends Rule[LogicalPlan] { - private[catalyst] def trimAliases(e: Expression): Expression = { + private def trimAliases(e: Expression): Expression = { var stop = false e.transformDown { // CreateStruct is a special case, we need to retain its top level Aliases as they decide the @@ -1678,21 +1678,18 @@ object TimeWindowing extends Rule[LogicalPlan] { p.children.head))) } - def apply(plan: LogicalPlan): LogicalPlan = { - val transformed = plan transform { - case p: LogicalPlan if p.expressions.collect(getWindowExpr[TimeWindow](e => e)).nonEmpty && - p.children.length == 1 => - val windowed = generateWindows(p) - val rewritten = p transformExpressions getWindowExpr { windowExpr => - windowExpr.validate() match { - case Some(e) => throw new AnalysisException(e) - case _ => // valid expression - } - windowExpr.outputColumn + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p: LogicalPlan if p.expressions.collect(getWindowExpr[TimeWindow](e => e)).nonEmpty && + p.children.length == 1 => + val windowed = generateWindows(p) + val rewritten = p transformExpressions getWindowExpr { windowExpr => + windowExpr.validate() match { + case Some(e) => throw new AnalysisException(e) + case _ => // valid expression } - rewritten.withNewChildren(windowed :: Nil) - } - transformed + windowExpr.outputColumn + } + rewritten.withNewChildren(windowed :: Nil) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 453172c1b4bae..40490215a9a09 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -25,7 +25,7 @@ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( originalTimeColumn: Expression, - private val _windowDuration: String, + private val _windowDuration: String, private val _slideDuration: String, private val _startTime: String) extends UnaryExpression with ExpectsInputTypes From 9301b1c218e6b49cf65cab1e3144acc8d53e3bb0 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 12:20:10 -0700 Subject: [PATCH 10/21] try this --- .../sql/catalyst/expressions/TimeWindow.scala | 156 +++++++++--------- 1 file changed, 78 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 40490215a9a09..340404dc0268e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -24,88 +24,88 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( - originalTimeColumn: Expression, - private val _windowDuration: String, - private val _slideDuration: String, - private val _startTime: String) extends UnaryExpression - with ExpectsInputTypes - with Unevaluable - with NonSQLExpression { + originalTimeColumn: Expression, + private val _windowDuration: String, + private val _slideDuration: String, + private val _startTime: String) extends UnaryExpression + with ExpectsInputTypes + with Unevaluable + with NonSQLExpression { - override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) - // the time column in Timestamp format - lazy val timeColumn = Cast(originalTimeColumn, TimestampType) - override def child: Expression = timeColumn - override def dataType: DataType = outputType + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) + // the time column in Timestamp format + lazy val timeColumn = Cast(originalTimeColumn, TimestampType) + override def child: Expression = timeColumn + override def dataType: DataType = outputType - private def outputType: StructType = StructType(Seq( - StructField("start", TimestampType), StructField("end", TimestampType))) - lazy val output: Seq[Attribute] = outputType.toAttributes - def outputColumn: NamedExpression = Alias(CreateStruct(output), "window")() - def windowStartCol: Attribute = output.head - def windowEndCol: Attribute = output.last + private def outputType: StructType = StructType(Seq( + StructField("start", TimestampType), StructField("end", TimestampType))) + lazy val output: Seq[Attribute] = outputType.toAttributes + def outputColumn: NamedExpression = Alias(CreateStruct(output), "window")() + def windowStartCol: Attribute = output.head + def windowEndCol: Attribute = output.last - /** - * Parses the interval string for a valid time duration. CalendarInterval expects interval - * strings to start with the string `interval`. For usability, we prepend `interval` to the string - * if the user ommitted it. - * @param interval The interval string - * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, - * therefore we use seconds here as well. - */ - private def getIntervalInSeconds(interval: String): Long = { - if (StringUtils.isBlank(interval)) { - throw new AnalysisException( - "The window duration, slide duration and start time cannot be null.") - } - val intervalString = if (interval.startsWith("interval")) { - interval - } else { - "interval " + interval - } - val cal = CalendarInterval.fromString(intervalString) - if (cal == null) { - throw new AnalysisException( - s"The provided interval ($interval) did not correspond to a valid interval string.") - } - (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 - } + /** + * Parses the interval string for a valid time duration. CalendarInterval expects interval + * strings to start with the string `interval`. For usability, we prepend `interval` to the string + * if the user ommitted it. + * @param interval The interval string + * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, + * therefore we use seconds here as well. + */ + private def getIntervalInSeconds(interval: String): Long = { + if (StringUtils.isBlank(interval)) { + throw new AnalysisException( + "The window duration, slide duration and start time cannot be null.") + } + val intervalString = if (interval.startsWith("interval")) { + interval + } else { + "interval " + interval + } + val cal = CalendarInterval.fromString(intervalString) + if (cal == null) { + throw new AnalysisException( + s"The provided interval ($interval) did not correspond to a valid interval string.") + } + (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 + } - // The window duration in seconds - lazy val windowDuration: Long = getIntervalInSeconds(_windowDuration) - // The slide duration in seconds - lazy val slideDuration: Long = getIntervalInSeconds(_slideDuration) - // The start time offset in seconds - lazy val startTime: Long = getIntervalInSeconds(_startTime) + // The window duration in seconds + lazy val windowDuration: Long = getIntervalInSeconds(_windowDuration) + // The slide duration in seconds + lazy val slideDuration: Long = getIntervalInSeconds(_slideDuration) + // The start time offset in seconds + lazy val startTime: Long = getIntervalInSeconds(_startTime) - /** - * Validate the inputs for the window duration, slide duration, and start time. - * @return Some string with a useful error message for the invalid input. - */ - def validate(): Option[String] = { - if (windowDuration <= 0) { - return Some(s"The window duration (${_windowDuration}) must be greater than 0.") - } - if (slideDuration <= 0) { - return Some(s"The slide duration (${_slideDuration}) must be greater than 0.") - } - if (startTime < 0) { - return Some(s"The start time (${_startTime}) must be greater than or equal to 0.") - } - if (slideDuration > windowDuration) { - return Some(s"The slide duration (${_slideDuration}) must be less than or equal to the " + - s"windowDuration (${_windowDuration}).") - } - if (startTime >= slideDuration) { - return Some(s"The start time (${_startTime}) must be less than the " + - s"slideDuration (${_slideDuration}).") - } - None - } + /** + * Validate the inputs for the window duration, slide duration, and start time. + * @return Some string with a useful error message for the invalid input. + */ + def validate(): Option[String] = { + if (windowDuration <= 0) { + return Some(s"The window duration (${_windowDuration}) must be greater than 0.") + } + if (slideDuration <= 0) { + return Some(s"The slide duration (${_slideDuration}) must be greater than 0.") + } + if (startTime < 0) { + return Some(s"The start time (${_startTime}) must be greater than or equal to 0.") + } + if (slideDuration > windowDuration) { + return Some(s"The slide duration (${_slideDuration}) must be less than or equal to the " + + s"windowDuration (${_windowDuration}).") + } + if (startTime >= slideDuration) { + return Some(s"The start time (${_startTime}) must be less than the " + + s"slideDuration (${_slideDuration}).") + } + None + } - /** - * Returns the maximum possible number of overlapping windows we will have with the given - * window and slide durations. - */ - def maxNumOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt + /** + * Returns the maximum possible number of overlapping windows we will have with the given + * window and slide durations. + */ + def maxNumOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt } From ad989026b727b0b77798ca6012c39dcaf502bfd5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 12:21:31 -0700 Subject: [PATCH 11/21] fix test file as well --- .../expressions/TimeWindowSuite.scala | 124 +++++++++--------- 1 file changed, 62 insertions(+), 62 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index 43afae384be0f..2ca66c76e0c71 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -22,71 +22,71 @@ import org.apache.spark.sql.AnalysisException class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { - test("time window is unevaluable") { - intercept[UnsupportedOperationException] { - evaluate(TimeWindow(Literal(10L), "1 second", "1 second", "0 second")) - } - } + test("time window is unevaluable") { + intercept[UnsupportedOperationException] { + evaluate(TimeWindow(Literal(10L), "1 second", "1 second", "0 second")) + } + } - test("validation checks") { - assert(TimeWindow(Literal(10L), "1 second", "1 second", "0 second").validate().isEmpty, - "Is a valid expression. Should not have returned a validation error.") - assert(TimeWindow(Literal(10L), "1 second", "2 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is greater than window.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 minute").validate().isDefined, - "Should have thrown validation error, because start time is greater than slide duration.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 second").validate().isDefined, - "Should have thrown validation error, because start time is equal to slide duration.") - assert(TimeWindow(Literal(10L), "-1 second", "1 second", "0 second").validate().isDefined, - "Should have thrown validation error, because window duration is negative.") - assert(TimeWindow(Literal(10L), "0 second", "0 second", "0 second").validate().isDefined, - "Should have thrown validation error, because window duration is zero.") - assert(TimeWindow(Literal(10L), "1 second", "-1 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is negative.") - assert(TimeWindow(Literal(10L), "1 second", "0 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is zero.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "-2 second").validate().isDefined, - "Should have thrown validation error, because start time is negative.") - } + test("validation checks") { + assert(TimeWindow(Literal(10L), "1 second", "1 second", "0 second").validate().isEmpty, + "Is a valid expression. Should not have returned a validation error.") + assert(TimeWindow(Literal(10L), "1 second", "2 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is greater than window.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 minute").validate().isDefined, + "Should have thrown validation error, because start time is greater than slide duration.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 second").validate().isDefined, + "Should have thrown validation error, because start time is equal to slide duration.") + assert(TimeWindow(Literal(10L), "-1 second", "1 second", "0 second").validate().isDefined, + "Should have thrown validation error, because window duration is negative.") + assert(TimeWindow(Literal(10L), "0 second", "0 second", "0 second").validate().isDefined, + "Should have thrown validation error, because window duration is zero.") + assert(TimeWindow(Literal(10L), "1 second", "-1 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is negative.") + assert(TimeWindow(Literal(10L), "1 second", "0 second", "0 second").validate().isDefined, + "Should have thrown validation error, because slide duration is zero.") + assert(TimeWindow(Literal(10L), "1 second", "1 second", "-2 second").validate().isDefined, + "Should have thrown validation error, because start time is negative.") + } - test("invalid intervals throw exception") { - val validDuration = "10 second" - val validTime = "5 second" - for (invalid <- Seq(null, " ", "\n", "\t", "2 apples")) { - intercept[AnalysisException] { - TimeWindow(Literal(10L), invalid, validDuration, validTime).windowDuration - } - intercept[AnalysisException] { - TimeWindow(Literal(10L), validDuration, invalid, validTime).slideDuration - } - intercept[AnalysisException] { - TimeWindow(Literal(10L), validDuration, validDuration, invalid).startTime - } - } - } + test("invalid intervals throw exception") { + val validDuration = "10 second" + val validTime = "5 second" + for (invalid <- Seq(null, " ", "\n", "\t", "2 apples")) { + intercept[AnalysisException] { + TimeWindow(Literal(10L), invalid, validDuration, validTime).windowDuration + } + intercept[AnalysisException] { + TimeWindow(Literal(10L), validDuration, invalid, validTime).slideDuration + } + intercept[AnalysisException] { + TimeWindow(Literal(10L), validDuration, validDuration, invalid).startTime + } + } + } - test("interval strings work with and without 'interval' prefix and returns seconds") { - val validDuration = "10 second" - for ((text, seconds) <- Seq(("1 second", 1), ("1 minute", 60), ("2 hours", 7200))) { - assert(TimeWindow(Literal(10L), text, validDuration, "0 seconds").windowDuration === seconds) - assert(TimeWindow(Literal(10L), "interval " + text, validDuration, "0 seconds").windowDuration - === seconds) - } - } + test("interval strings work with and without 'interval' prefix and returns seconds") { + val validDuration = "10 second" + for ((text, seconds) <- Seq(("1 second", 1), ("1 minute", 60), ("2 hours", 7200))) { + assert(TimeWindow(Literal(10L), text, validDuration, "0 seconds").windowDuration === seconds) + assert(TimeWindow(Literal(10L), "interval " + text, validDuration, "0 seconds").windowDuration + === seconds) + } + } - test("maxNumOverlapping takes ceiling of window duration over slide duration") { - assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) - assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) - assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) - assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) - assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) - } + test("maxNumOverlapping takes ceiling of window duration over slide duration") { + assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) + assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) + assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) + assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) + assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) + } - test("output column name is window and is a struct") { - val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") - assert(expr.outputColumn.name === "window") - assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) - assert(expr.windowStartCol.name === "start") - assert(expr.windowEndCol.name === "end") - } + test("output column name is window and is a struct") { + val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") + assert(expr.outputColumn.name === "window") + assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) + assert(expr.windowStartCol.name === "start") + assert(expr.windowEndCol.name === "end") + } } From 6a784b732d873aaa7d8b0edff346b34b63107d0f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 14:58:56 -0700 Subject: [PATCH 12/21] address most comments --- .../sql/catalyst/analysis/Analyzer.scala | 37 +++--- .../sql/catalyst/expressions/TimeWindow.scala | 97 +++++++++------- .../analysis/AnalysisErrorSuite.scala | 56 +++++++++ .../expressions/TimeWindowSuite.scala | 44 +++---- .../spark/sql/DataFrameFunctionsSuite.scala | 68 ----------- .../sql/DataFrameTimeWindowingSuite.scala | 109 ++++++++++++++++++ 6 files changed, 253 insertions(+), 158 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fe92be2e9cb6e..f779afa4ddbfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -23,6 +23,7 @@ import scala.annotation.tailrec import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -1601,7 +1602,12 @@ object TimeWindowing extends Rule[LogicalPlan] { * @return The user defined function applied on the TimeWindow expression */ private def getWindowExpr[E](f: TimeWindow => E): PartialFunction[Expression, E] = { - case Alias(windowExpr: TimeWindow, _) => f(windowExpr) + case Alias(windowExpr: TimeWindow, name) => + if (name == windowExpr.toString.toLowerCase) { + f(windowExpr) + } else { + f(windowExpr.withOutputColumnName(name)) + } case windowExpr: TimeWindow => f(windowExpr) } @@ -1644,34 +1650,25 @@ object TimeWindowing extends Rule[LogicalPlan] { * the Filter operator for correctness and Project for usability. */ private def generateWindows(p: LogicalPlan): LogicalPlan = { + import org.apache.spark.sql.catalyst.dsl.expressions._ val windowExpr = p.expressions.collect(getWindowExpr[TimeWindow](e => e)).head // get all expressions we need to pass on for projections val otherExpressions = getMissingAttributes(p).filterNot(_.children.exists(_.isInstanceOf[TimeWindow])) val projections = Seq.tabulate(windowExpr.maxNumOverlapping + 1) { i => - // windowId <- ceil((timestamp - startTime) / slideDuration) - val division = Ceil(Divide(Subtract(Cast(windowExpr.timeColumn, LongType), - Literal(windowExpr.startTime)), Literal(windowExpr.slideDuration))) - // start <- (windowId + i - maxNumOverlapping) * slideDuration + startTime + val windowId = Ceil((Cast(windowExpr.timeColumn, LongType) - windowExpr.startTime) / + windowExpr.slideDuration) + val windowStart = (windowId + i - windowExpr.maxNumOverlapping) * + windowExpr.slideDuration + windowExpr.startTime + val windowEnd = windowStart + windowExpr.windowDuration // the 1000000 is necessary for properly casting a LongType to a TimestampType - val windowStart = - Multiply( - Add( - Multiply( - Add(division, Literal(i - windowExpr.maxNumOverlapping)), - Literal(windowExpr.slideDuration)), - Literal(windowExpr.startTime)), - Literal(1000000)) - // windowEnd <- windowStart + windowDuration - val windowEnd = - Add(windowStart, Multiply(Literal(windowExpr.windowDuration), Literal(1000000))) - windowStart :: windowEnd :: windowExpr.originalTimeColumn :: Nil ++ otherExpressions + windowStart * 1000000 :: windowEnd * 1000000 :: windowExpr.originalTimeColumn :: Nil ++ + otherExpressions } val timeCol = windowExpr.originalTimeColumn.references.toSeq - // timestamp >= window.start && timestamp < window.end - val filterExpr = And(GreaterThanOrEqual(windowExpr.timeColumn, windowExpr.windowStartCol), - LessThan(windowExpr.timeColumn, windowExpr.windowEndCol)) + val filterExpr = windowExpr.timeColumn >= windowExpr.windowStartCol && + windowExpr.timeColumn < windowExpr.windowEndCol Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol ++ otherExpressions, Filter(filterExpr, Expand(projections, windowExpr.output ++ timeCol ++ otherExpressions.map(_.toAttribute), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 340404dc0268e..14103ffeacb94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -20,14 +20,16 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.commons.lang.StringUtils import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( originalTimeColumn: Expression, - private val _windowDuration: String, - private val _slideDuration: String, - private val _startTime: String) extends UnaryExpression + windowDuration: Long, + slideDuration: Long, + startTime: Long, + private var outputColumnName: String = "window") extends UnaryExpression with ExpectsInputTypes with Unevaluable with NonSQLExpression { @@ -41,64 +43,36 @@ case class TimeWindow( private def outputType: StructType = StructType(Seq( StructField("start", TimestampType), StructField("end", TimestampType))) lazy val output: Seq[Attribute] = outputType.toAttributes - def outputColumn: NamedExpression = Alias(CreateStruct(output), "window")() + def outputColumn: NamedExpression = Alias(CreateStruct(output), outputColumnName)() def windowStartCol: Attribute = output.head def windowEndCol: Attribute = output.last - /** - * Parses the interval string for a valid time duration. CalendarInterval expects interval - * strings to start with the string `interval`. For usability, we prepend `interval` to the string - * if the user ommitted it. - * @param interval The interval string - * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, - * therefore we use seconds here as well. - */ - private def getIntervalInSeconds(interval: String): Long = { - if (StringUtils.isBlank(interval)) { - throw new AnalysisException( - "The window duration, slide duration and start time cannot be null.") - } - val intervalString = if (interval.startsWith("interval")) { - interval - } else { - "interval " + interval - } - val cal = CalendarInterval.fromString(intervalString) - if (cal == null) { - throw new AnalysisException( - s"The provided interval ($interval) did not correspond to a valid interval string.") - } - (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 + def withOutputColumnName(newName: String): this.type = { + outputColumnName = newName + this } - // The window duration in seconds - lazy val windowDuration: Long = getIntervalInSeconds(_windowDuration) - // The slide duration in seconds - lazy val slideDuration: Long = getIntervalInSeconds(_slideDuration) - // The start time offset in seconds - lazy val startTime: Long = getIntervalInSeconds(_startTime) - /** * Validate the inputs for the window duration, slide duration, and start time. * @return Some string with a useful error message for the invalid input. */ def validate(): Option[String] = { if (windowDuration <= 0) { - return Some(s"The window duration (${_windowDuration}) must be greater than 0.") + return Some(s"The window duration ($windowDuration) must be greater than 0.") } if (slideDuration <= 0) { - return Some(s"The slide duration (${_slideDuration}) must be greater than 0.") + return Some(s"The slide duration ($slideDuration) must be greater than 0.") } if (startTime < 0) { - return Some(s"The start time (${_startTime}) must be greater than or equal to 0.") + return Some(s"The start time ($startTime) must be greater than or equal to 0.") } if (slideDuration > windowDuration) { - return Some(s"The slide duration (${_slideDuration}) must be less than or equal to the " + - s"windowDuration (${_windowDuration}).") + return Some(s"The slide duration ($slideDuration) must be less than or equal to the " + + s"windowDuration ($windowDuration).") } if (startTime >= slideDuration) { - return Some(s"The start time (${_startTime}) must be less than the " + - s"slideDuration (${_slideDuration}).") + return Some(s"The start time ($startTime) must be less than the " + + s"slideDuration ($slideDuration).") } None } @@ -109,3 +83,42 @@ case class TimeWindow( */ def maxNumOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt } + +object TimeWindow { + /** + * Parses the interval string for a valid time duration. CalendarInterval expects interval + * strings to start with the string `interval`. For usability, we prepend `interval` to the string + * if the user ommitted it. + * @param interval The interval string + * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, + * therefore we use seconds here as well. + */ + private def getIntervalInSeconds(interval: String): Long = { + if (StringUtils.isBlank(interval)) { + throw new IllegalArgumentException( + "The window duration, slide duration and start time cannot be null or blank.") + } + val intervalString = if (interval.startsWith("interval")) { + interval + } else { + "interval " + interval + } + val cal = CalendarInterval.fromString(intervalString) + if (cal == null) { + throw new IllegalArgumentException( + s"The provided interval ($interval) did not correspond to a valid interval string.") + } + (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 + } + + def apply( + timeColumn: Expression, + windowDuration: String, + slideDuration: String, + startTime: String): TimeWindow = { + TimeWindow(timeColumn, + getIntervalInSeconds(windowDuration), + getIntervalInSeconds(slideDuration), + getIntervalInSeconds(startTime)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index a90dfc50392d5..628b2e4846212 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -272,6 +272,62 @@ class AnalysisErrorSuite extends AnalysisTest { testRelation2.where('bad_column > 1).groupBy('a)(UnresolvedAlias(max('b))), "cannot resolve '`bad_column`'" :: Nil) + errorTest( + "slide duration greater than window in time window", + testRelation2.select( + TimeWindow(Literal(1L), "1 second", "2 second", "0 second").as("window")), + s"The slide duration (2) must be less than or equal to the windowDuration (1)." :: Nil + ) + + errorTest( + "start time greater than slide duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "1 second", "1 second", "1 minute").as("window")), + "The start time (60) must be less than the slideDuration (1)." :: Nil + ) + + errorTest( + "start time equal to slide duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "1 second", "1 second", "1 second").as("window")), + "The start time (1) must be less than the slideDuration (1)." :: Nil + ) + + errorTest( + "negative window duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "-1 second", "1 second", "0 second").as("window")), + "The window duration (-1) must be greater than 0." :: Nil + ) + + errorTest( + "zero window duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "0 second", "1 second", "0 second").as("window")), + "The window duration (0) must be greater than 0." :: Nil + ) + + errorTest( + "negative slide duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "1 second", "-1 second", "0 second").as("window")), + "The slide duration (-1) must be greater than 0." :: Nil + ) + + errorTest( + "zero slide duration in time window", + testRelation.select( + TimeWindow(Literal(1L), "1 second", "0 second", "0 second").as("window")), + "The slide duration (0) must be greater than 0." :: Nil + ) + + errorTest( + "negative start time in time window", + testRelation.select( + TimeWindow(Literal(1L), "1 second", "1 second", "-5 second").as("window")), + "The start time (-5) must be greater than or equal to 0." :: Nil + ) + test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) // Since we manually construct the logical plan at here and Sum only accept diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index 2ca66c76e0c71..b8069a45a69e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -28,41 +28,29 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("validation checks") { - assert(TimeWindow(Literal(10L), "1 second", "1 second", "0 second").validate().isEmpty, - "Is a valid expression. Should not have returned a validation error.") - assert(TimeWindow(Literal(10L), "1 second", "2 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is greater than window.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 minute").validate().isDefined, - "Should have thrown validation error, because start time is greater than slide duration.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "1 second").validate().isDefined, - "Should have thrown validation error, because start time is equal to slide duration.") - assert(TimeWindow(Literal(10L), "-1 second", "1 second", "0 second").validate().isDefined, - "Should have thrown validation error, because window duration is negative.") - assert(TimeWindow(Literal(10L), "0 second", "0 second", "0 second").validate().isDefined, - "Should have thrown validation error, because window duration is zero.") - assert(TimeWindow(Literal(10L), "1 second", "-1 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is negative.") - assert(TimeWindow(Literal(10L), "1 second", "0 second", "0 second").validate().isDefined, - "Should have thrown validation error, because slide duration is zero.") - assert(TimeWindow(Literal(10L), "1 second", "1 second", "-2 second").validate().isDefined, - "Should have thrown validation error, because start time is negative.") - } - test("invalid intervals throw exception") { val validDuration = "10 second" val validTime = "5 second" - for (invalid <- Seq(null, " ", "\n", "\t", "2 apples")) { - intercept[AnalysisException] { - TimeWindow(Literal(10L), invalid, validDuration, validTime).windowDuration + def checkErrorMessage(msg: String, value: String): Unit = { + val e1 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), value, validDuration, validTime).windowDuration } - intercept[AnalysisException] { - TimeWindow(Literal(10L), validDuration, invalid, validTime).slideDuration + val e2 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), validDuration, value, validTime).slideDuration } - intercept[AnalysisException] { - TimeWindow(Literal(10L), validDuration, validDuration, invalid).startTime + val e3 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), validDuration, validDuration, value).startTime } + Seq(e1, e2, e3).foreach { e => + e.getMessage.contains(msg) + } + } + for (blank <- Seq(null, " ", "\n", "\t")) { + checkErrorMessage( + "The window duration, slide duration and start time cannot be null or blank.", blank) } + checkErrorMessage( + "did not correspond to a valid interval string.", "2 apples") } test("interval strings work with and without 'interval' prefix and returns seconds") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index fdf0b805134ea..746e25a0c3ec5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -394,72 +394,4 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(true), Row(true)) ) } - - test("time windowing") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC - val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), - (1459103967L, 4, "b")).toDF("time", "value", "id") - checkAnswer( - df.groupBy(window($"time", "10 seconds")).agg(count("*").as("counts")) - .orderBy($"window.start".asc).select("counts"), - Seq(Row(1), Row(1), Row(1)) - ) - checkAnswer( - df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds")) - .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), - Seq(Row(2), Row(1)) - ) - checkAnswer( - df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") - .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), - Seq(Row(1), Row(1), Row(1)) - ) - checkAnswer( - df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) - .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), - // 2016-03-27 19:39:27 UTC -> 4 bins - // 2016-03-27 19:39:34 UTC -> 3 bins - // 2016-03-27 19:39:56 UTC -> 3 bins - Seq(Row(1), Row(1), Row(1), Row(2), Row(1), Row(1), Row(1), Row(1), Row(1)) - ) - checkAnswer( - df.select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), - Seq(Row(4), Row(1), Row(2)) - ) - } - - test("esoteric time windowing use cases") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC - val df = Seq((1459103974L, 1, Seq("a", "b")), - (1459103996L, 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") - checkAnswer( - df.select(window($"time", "10 seconds"), $"value", explode($"ids")) - .orderBy($"window.start".asc).select("value"), - // first window exploded to two rows for "a", and "b", second window exploded to 3 rows - Seq(Row(1), Row(1), Row(2), Row(2), Row(2)) - ) - val df2 = Seq(("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), - (null, 3), (null, 4)).toDF("time", "value") - val df3 = Seq(("2016-03-27 09:00:02", 3), ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") - checkAnswer( - df2.select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), - Seq(Row(1), Row(2)) // null columns are dropped - ) - checkAnswer( - df2.select(window($"time", "10 seconds"), $"value").join( - df3.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") - .agg((sum("value") + sum("othervalue")).as("total")) - .orderBy($"window.start".asc).select("total"), - Seq(Row(4), Row(8)) - ) - // should result in negative timestamps in the window - val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") - checkAnswer( - df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), - Seq(Row(1), Row(2)) - ) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala new file mode 100644 index 0000000000000..efc266eaef0f3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -0,0 +1,109 @@ +/* + * 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 org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext + +class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("time windowing - tumbling windows") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), + (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( + df.groupBy(window($"time", "10 seconds")).agg(count("*").as("counts")) + .orderBy($"window.start".asc).select("counts"), + Seq(Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") + .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + Seq(Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(4), Row(1), Row(2)) + ) + } + + test("time windowing - sliding windows") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), + (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( + df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) + .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(1), Row(1), Row(1), Row(2), Row(1), Row(1), Row(1), Row(1), Row(1)) + ) + checkAnswer( + df.select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") + .orderBy($"window.start".asc, $"value".desc).select("value"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + } + + test("esoteric time windowing use cases") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC + val df = Seq((1459103974L, 1, Seq("a", "b")), + (1459103996L, 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + checkAnswer( + df.select(window($"time", "10 seconds"), $"value", explode($"ids")) + .orderBy($"window.start".asc).select("value"), + // first window exploded to two rows for "a", and "b", second window exploded to 3 rows + Seq(Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + // string timestamps + val df2 = Seq(("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), + (null, 3), (null, 4)).toDF("time", "value") + val df3 = Seq(("2016-03-27 09:00:02", 3), ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") + checkAnswer( + df2.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(1), Row(2)) // null columns are dropped + ) + checkAnswer( + df2.select(window($"time", "10 seconds"), $"value").join( + df3.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") + .agg((sum("value") + sum("othervalue")).as("total")) + .orderBy($"window.start".asc).select("total"), + Seq(Row(4), Row(8)) + ) + // should result in negative timestamps in the window + val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") + checkAnswer( + df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") + .orderBy($"window.start".asc).select("value"), + Seq(Row(1), Row(2)) + ) + // we don't lose aliasing + checkAnswer( + df.select(window($"time", "10 seconds").as("time_window"), $"value") + .orderBy($"time_window.start".asc).select("value"), + Seq(Row(1), Row(2)) + ) + } +} From b7154b2e0c0035e74ce5f25044a576c03c24acc8 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 15:13:53 -0700 Subject: [PATCH 13/21] added experimental tags --- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 122493c54fda3..67ded4a850a2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2593,6 +2593,7 @@ object functions { * @group datetime_funcs * @since 2.0.0 */ + @Experimental def window( timeColumn: Column, windowDuration: String, @@ -2638,6 +2639,7 @@ object functions { * @group datetime_funcs * @since 2.0.0 */ + @Experimental def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { window(timeColumn, windowDuration, slideDuration, "0 second") } @@ -2675,6 +2677,7 @@ object functions { * @group datetime_funcs * @since 2.0.0 */ + @Experimental def window(timeColumn: Column, windowDuration: String): Column = { window(timeColumn, windowDuration, windowDuration, "0 second") } From 7eea4486c41e80d5fac1aa0bcc487f27631b3f29 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 28 Mar 2016 15:20:17 -0700 Subject: [PATCH 14/21] fix ss --- .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f779afa4ddbfb..d035da974734c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -23,7 +23,6 @@ import scala.annotation.tailrec import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.{CatalystConf, ScalaReflection, SimpleCatalystConf} import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.OuterScopes From 200e4808c2adb4f2168dcbbe63de01cc27856291 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Mar 2016 18:11:17 -0700 Subject: [PATCH 15/21] some cleanup --- .../sql/catalyst/analysis/Analyzer.scala | 110 +++++++--------- .../sql/catalyst/expressions/TimeWindow.scala | 31 ++--- .../expressions/TimeWindowSuite.scala | 30 ++--- .../org/apache/spark/sql/functions.scala | 7 +- .../sql/DataFrameTimeWindowingSuite.scala | 123 +++++++++++++----- 5 files changed, 169 insertions(+), 132 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d035da974734c..cf653cf31713c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -77,8 +77,7 @@ class Analyzer( Batch("Substitution", fixedPoint, CTESubstitution, WindowsSubstitution, - EliminateUnions, - TimeWindowing), + EliminateUnions), Batch("Resolution", fixedPoint, ResolveRelations :: ResolveReferences :: @@ -97,6 +96,7 @@ class Analyzer( ExtractWindowExpressions :: GlobalAggregates :: ResolveAggregateFunctions :: + TimeWindowing :: HiveTypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, @@ -1593,32 +1593,7 @@ object ResolveUpCast extends Rule[LogicalPlan] { * filter out the rows where the time column is not inside the time window. */ object TimeWindowing extends Rule[LogicalPlan] { - - /** - * Depending on the operation, the TimeWindow expression may be wrapped in an Alias (in case of - * projections) or be simply by itself (in case of groupBy), - * @param f The function that we want to apply on the TimeWindow expression - * @return The user defined function applied on the TimeWindow expression - */ - private def getWindowExpr[E](f: TimeWindow => E): PartialFunction[Expression, E] = { - case Alias(windowExpr: TimeWindow, name) => - if (name == windowExpr.toString.toLowerCase) { - f(windowExpr) - } else { - f(windowExpr.withOutputColumnName(name)) - } - case windowExpr: TimeWindow => f(windowExpr) - } - - /** Pass on other columns that are required either in a projection or grouping. */ - private def getMissingAttributes(plan: LogicalPlan): Seq[NamedExpression] = { - val expressions = plan match { - case p: Project => p.projectList - case a: Aggregate => a.groupingExpressions - case _ => Seq.empty - } - expressions.flatMap(_.references) - } + import org.apache.spark.sql.catalyst.dsl.expressions._ /** * Generates the logical plan for generating window ranges on a timestamp column. Without @@ -1648,44 +1623,53 @@ object TimeWindowing extends Rule[LogicalPlan] { * @return the logical plan that will generate the time windows using the Expand operator, with * the Filter operator for correctness and Project for usability. */ - private def generateWindows(p: LogicalPlan): LogicalPlan = { - import org.apache.spark.sql.catalyst.dsl.expressions._ - val windowExpr = p.expressions.collect(getWindowExpr[TimeWindow](e => e)).head - // get all expressions we need to pass on for projections - val otherExpressions = - getMissingAttributes(p).filterNot(_.children.exists(_.isInstanceOf[TimeWindow])) - - val projections = Seq.tabulate(windowExpr.maxNumOverlapping + 1) { i => - val windowId = Ceil((Cast(windowExpr.timeColumn, LongType) - windowExpr.startTime) / - windowExpr.slideDuration) - val windowStart = (windowId + i - windowExpr.maxNumOverlapping) * - windowExpr.slideDuration + windowExpr.startTime - val windowEnd = windowStart + windowExpr.windowDuration - // the 1000000 is necessary for properly casting a LongType to a TimestampType - windowStart * 1000000 :: windowEnd * 1000000 :: windowExpr.originalTimeColumn :: Nil ++ - otherExpressions - } - val timeCol = windowExpr.originalTimeColumn.references.toSeq - val filterExpr = windowExpr.timeColumn >= windowExpr.windowStartCol && - windowExpr.timeColumn < windowExpr.windowEndCol - Project(windowExpr.output ++ Seq(windowExpr.outputColumn) ++ timeCol ++ otherExpressions, - Filter(filterExpr, - Expand(projections, windowExpr.output ++ timeCol ++ otherExpressions.map(_.toAttribute), - p.children.head))) - } + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case p: LogicalPlan if p.children.size == 1 => + val child = p.children.head + val windowExpressions = + p.expressions.flatMap(_.collect { case t: TimeWindow => t }).distinct.toList // Not correct. + + println(s"found: $windowExpressions") + + // Only support a single window expression for now? + if (windowExpressions.size == 1 && + windowExpressions.head.timeColumn.resolved && + windowExpressions.head.timeColumn.dataType == TimestampType) { + val window = windowExpressions.head + val windowAttr = AttributeReference("window", window.dataType)() + + val maxNumOverlapping = math.ceil(window.windowDuration * 1.0 / window.slideDuration).toInt + val windows = Seq.tabulate(maxNumOverlapping + 1) { i => + val windowId = Ceil((Cast(window.timeColumn, LongType) - window.startTime) / + window.slideDuration) + val windowStart = (windowId + i - maxNumOverlapping) * + window.slideDuration + window.startTime + val windowEnd = windowStart + window.windowDuration + + // the 1000000 is necessary for properly casting a LongType to a TimestampType + CreateNamedStruct( + Literal("start") :: windowStart * 1000000 :: + Literal("end") :: windowEnd * 1000000 :: Nil) + } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p: LogicalPlan if p.expressions.collect(getWindowExpr[TimeWindow](e => e)).nonEmpty && - p.children.length == 1 => - val windowed = generateWindows(p) - val rewritten = p transformExpressions getWindowExpr { windowExpr => - windowExpr.validate() match { - case Some(e) => throw new AnalysisException(e) - case _ => // valid expression + val projections = windows.map(_ +: p.children.head.output) + + val filterExpr = + window.timeColumn >= windowAttr.getField("start") + window.timeColumn < windowAttr.getField("end") + + val expandedPlan = + Filter(filterExpr, + Expand(projections, windowAttr +: child.output, child)) + + val substituedPlan = p transformExpressions { + case t: TimeWindow => windowAttr } - windowExpr.outputColumn + + substituedPlan.withNewChildren(expandedPlan :: Nil) + } else { + p // Return unchanged } - rewritten.withNewChildren(windowed :: Nil) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 14103ffeacb94..b24eccd09d995 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -25,32 +25,23 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( - originalTimeColumn: Expression, + timeColumn: Expression, windowDuration: Long, slideDuration: Long, startTime: Long, private var outputColumnName: String = "window") extends UnaryExpression - with ExpectsInputTypes + with ImplicitCastInputTypes with Unevaluable with NonSQLExpression { - override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(TimestampType, LongType)) - // the time column in Timestamp format - lazy val timeColumn = Cast(originalTimeColumn, TimestampType) override def child: Expression = timeColumn - override def dataType: DataType = outputType + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def dataType: DataType = new StructType() + .add(StructField("start", TimestampType)) + .add(StructField("end", TimestampType)) - private def outputType: StructType = StructType(Seq( - StructField("start", TimestampType), StructField("end", TimestampType))) - lazy val output: Seq[Attribute] = outputType.toAttributes - def outputColumn: NamedExpression = Alias(CreateStruct(output), outputColumnName)() - def windowStartCol: Attribute = output.head - def windowEndCol: Attribute = output.last - - def withOutputColumnName(newName: String): this.type = { - outputColumnName = newName - this - } + // This expression is replaced in the analyzer. + override lazy val resolved = false /** * Validate the inputs for the window duration, slide duration, and start time. @@ -76,12 +67,6 @@ case class TimeWindow( } None } - - /** - * Returns the maximum possible number of overlapping windows we will have with the given - * window and slide durations. - */ - def maxNumOverlapping: Int = math.ceil(windowDuration * 1.0 / slideDuration).toInt } object TimeWindow { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index b8069a45a69e8..9f47338f28423 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -62,19 +62,19 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("maxNumOverlapping takes ceiling of window duration over slide duration") { - assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) - assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) - assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) - assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) - assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) - } - - test("output column name is window and is a struct") { - val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") - assert(expr.outputColumn.name === "window") - assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) - assert(expr.windowStartCol.name === "start") - assert(expr.windowEndCol.name === "end") - } +// test("maxNumOverlapping takes ceiling of window duration over slide duration") { +// assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) +// assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) +// assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) +// assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) +// assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) +// } +// +// test("output column name is window and is a struct") { +// val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") +// assert(expr.outputColumn.name === "window") +// assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) +// assert(expr.windowStartCol.name === "start") +// assert(expr.windowEndCol.name === "end") +// } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 67ded4a850a2a..2199ecf9781af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2598,10 +2598,13 @@ object functions { timeColumn: Column, windowDuration: String, slideDuration: String, - startTime: String): Column = withExpr { - TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) + startTime: String): Column = { + withExpr { + TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) + }.as("window") } + /** * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index efc266eaef0f3..ae95987fea906 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -21,36 +21,65 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { + import testImplicits._ - test("time windowing - tumbling windows") { + test("tumbling windows") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC - val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), + val df = Seq( + (1459103974L, 1, "a"), + (1459103996L, 2, "a"), (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( - df.groupBy(window($"time", "10 seconds")).agg(count("*").as("counts")) - .orderBy($"window.start".asc).select("counts"), + df.groupBy(window($"time", "10 seconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select("counts"), Seq(Row(1), Row(1), Row(1)) ) + } + + test("explicit tumbling window") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + checkAnswer( df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") - .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), - Seq(Row(1), Row(1), Row(1)) - ) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select("counts"), + Seq(Row(1), Row(1), Row(1))) + } + + test("?") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + val df = Seq( + (1459103974L, 1, "a"), + (1459103996L, 2, "a"), + (1459103967L, 4, "b")).toDF("time", "value", "id") checkAnswer( df.select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), + .orderBy($"window.start".asc) + .select("value"), Seq(Row(4), Row(1), Row(2)) ) } test("time windowing - sliding windows") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC - val df = Seq((1459103974L, 1, "a"), (1459103996L, 2, "a"), + val df = Seq( + (1459103974L, 1, "a"), + (1459103996L, 2, "a"), (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) - .agg(count("*").as("counts")).orderBy($"window.start".asc).select("counts"), + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select("counts"), // 2016-03-27 19:39:27 UTC -> 4 bins // 2016-03-27 19:39:34 UTC -> 3 bins // 2016-03-27 19:39:56 UTC -> 3 bins @@ -68,41 +97,77 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { test("esoteric time windowing use cases") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC - val df = Seq((1459103974L, 1, Seq("a", "b")), + val df = Seq( + (1459103974L, 1, Seq("a", "b")), (1459103996L, 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + checkAnswer( df.select(window($"time", "10 seconds"), $"value", explode($"ids")) .orderBy($"window.start".asc).select("value"), // first window exploded to two rows for "a", and "b", second window exploded to 3 rows Seq(Row(1), Row(1), Row(2), Row(2), Row(2)) ) - // string timestamps - val df2 = Seq(("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), - (null, 3), (null, 4)).toDF("time", "value") - val df3 = Seq(("2016-03-27 09:00:02", 3), ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") - checkAnswer( - df2.select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), - Seq(Row(1), Row(2)) // null columns are dropped - ) + } + + test("string timestamps") { + val df2 = Seq( + ("2016-03-27 09:00:05", 1), + ("2016-03-27 09:00:32", 2), + (null, 3), + (null, 4)).toDF("time", "value") + + checkDataset( + df2 + .select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc) + .select("value") + .as[Int], + 1, 2) // null columns are dropped + } + + test("another test") { + val df2 = Seq( + ("2016-03-27 09:00:05", 1), + ("2016-03-27 09:00:32", 2), + (null, 3), + (null, 4)).toDF("time", "value") + + val df3 = Seq( + ("2016-03-27 09:00:02", 3), + ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") + + checkAnswer( df2.select(window($"time", "10 seconds"), $"value").join( - df3.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") - .agg((sum("value") + sum("othervalue")).as("total")) - .orderBy($"window.start".asc).select("total"), - Seq(Row(4), Row(8)) - ) - // should result in negative timestamps in the window + df3 + .select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") + .agg((sum("value") + sum("othervalue")).as("total")) + .orderBy($"window.start".asc).select("total"), + Seq(Row(4), Row(8))) + } + + test("negative timestamps") { val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") checkAnswer( df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), + .orderBy($"window.start".asc).select("value"), Seq(Row(1), Row(2)) ) - // we don't lose aliasing + } + + ignore("multiple time windows in a single operator") { + + } + + test("aliased windows") { + val df = Seq( + ("2016-03-27 19:39:34", 1, Seq("a", "b")), + ("2016-03-27 19:39:56", 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + checkAnswer( df.select(window($"time", "10 seconds").as("time_window"), $"value") - .orderBy($"time_window.start".asc).select("value"), + .orderBy($"time_window.start".asc) + .select("value"), Seq(Row(1), Row(2)) ) } From 83dd46d71b6a421bf9c847ca26c74e58b39891be Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Mar 2016 18:29:37 -0700 Subject: [PATCH 16/21] more cleanup --- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../sql/catalyst/expressions/TimeWindow.scala | 28 +++++++++++++++++++ .../analysis/AnalysisErrorSuite.scala | 2 +- 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cf653cf31713c..2621b4ca09ff6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1634,7 +1634,8 @@ object TimeWindowing extends Rule[LogicalPlan] { // Only support a single window expression for now? if (windowExpressions.size == 1 && windowExpressions.head.timeColumn.resolved && - windowExpressions.head.timeColumn.dataType == TimestampType) { + windowExpressions.head.timeColumn.dataType == TimestampType && + windowExpressions.head.checkInputDataTypes().isSuccess) { val window = windowExpressions.head val windowAttr = AttributeReference("window", window.dataType)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index b24eccd09d995..1379e3c97119e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -21,6 +21,7 @@ import org.apache.commons.lang.StringUtils import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -43,8 +44,34 @@ case class TimeWindow( // This expression is replaced in the analyzer. override lazy val resolved = false + override def checkInputDataTypes(): TypeCheckResult = { + val dataTypeCheck = super.checkInputDataTypes() + if (dataTypeCheck.isSuccess) { + if (windowDuration <= 0) { + return TypeCheckFailure(s"The window duration ($windowDuration) must be greater than 0.") + } + if (slideDuration <= 0) { + return TypeCheckFailure(s"The slide duration ($slideDuration) must be greater than 0.") + } + if (startTime < 0) { + return TypeCheckFailure(s"The start time ($startTime) must be greater than or equal to 0.") + } + if (slideDuration > windowDuration) { + return TypeCheckFailure(s"The slide duration ($slideDuration) must be less than or equal to the " + + s"windowDuration ($windowDuration).") + } + if (startTime >= slideDuration) { + return TypeCheckFailure(s"The start time ($startTime) must be less than the " + + s"slideDuration ($slideDuration).") + } + return dataTypeCheck + } else { + return dataTypeCheck + } + } /** * Validate the inputs for the window duration, slide duration, and start time. + * * @return Some string with a useful error message for the invalid input. */ def validate(): Option[String] = { @@ -74,6 +101,7 @@ object TimeWindow { * Parses the interval string for a valid time duration. CalendarInterval expects interval * strings to start with the string `interval`. For usability, we prepend `interval` to the string * if the user ommitted it. + * * @param interval The interval string * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, * therefore we use seconds here as well. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 628b2e4846212..013362cea137d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -296,7 +296,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "negative window duration in time window", testRelation.select( - TimeWindow(Literal(1L), "-1 second", "1 second", "0 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "-1 second", "1 second", "0 second").as("window")), "The window duration (-1) must be greater than 0." :: Nil ) From f1aa35706b6e256836a4d9378470202f4e5e4846 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 29 Mar 2016 22:44:16 -0700 Subject: [PATCH 17/21] make as much progress as possible --- .../sql/catalyst/analysis/Analyzer.scala | 26 ++++---- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../sql/catalyst/expressions/TimeWindow.scala | 44 +++---------- .../analysis/AnalysisErrorSuite.scala | 14 ++--- .../expressions/TimeWindowSuite.scala | 16 ----- .../org/apache/spark/sql/functions.scala | 11 ++-- .../sql/DataFrameTimeWindowingSuite.scala | 63 +++++++++++-------- 7 files changed, 76 insertions(+), 99 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2621b4ca09ff6..a9fb3a2612472 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1595,6 +1595,9 @@ object ResolveUpCast extends Rule[LogicalPlan] { object TimeWindowing extends Rule[LogicalPlan] { import org.apache.spark.sql.catalyst.dsl.expressions._ + private final val WINDOW_START = "start" + private final val WINDOW_END = "end" + /** * Generates the logical plan for generating window ranges on a timestamp column. Without * knowing what the timestamp value is, it's non-trivial to figure out deterministically how many @@ -1619,7 +1622,7 @@ object TimeWindowing extends Rule[LogicalPlan] { * 12:00 - 12:12 + 11:57 - 12:09 + * 12:05 - 12:17 + 12:02 - 12:14 + * - * @param p The logical plan + * @param plan The logical plan * @return the logical plan that will generate the time windows using the Expand operator, with * the Filter operator for correctness and Project for usability. */ @@ -1629,9 +1632,7 @@ object TimeWindowing extends Rule[LogicalPlan] { val windowExpressions = p.expressions.flatMap(_.collect { case t: TimeWindow => t }).distinct.toList // Not correct. - println(s"found: $windowExpressions") - - // Only support a single window expression for now? + // Only support a single window expression for now if (windowExpressions.size == 1 && windowExpressions.head.timeColumn.resolved && windowExpressions.head.timeColumn.dataType == TimestampType && @@ -1649,27 +1650,30 @@ object TimeWindowing extends Rule[LogicalPlan] { // the 1000000 is necessary for properly casting a LongType to a TimestampType CreateNamedStruct( - Literal("start") :: windowStart * 1000000 :: - Literal("end") :: windowEnd * 1000000 :: Nil) + Literal(WINDOW_START) :: windowStart * 1000000 :: + Literal(WINDOW_END) :: windowEnd * 1000000 :: Nil) } val projections = windows.map(_ +: p.children.head.output) val filterExpr = - window.timeColumn >= windowAttr.getField("start") - window.timeColumn < windowAttr.getField("end") + window.timeColumn >= windowAttr.getField(WINDOW_START) + window.timeColumn < windowAttr.getField(WINDOW_END) val expandedPlan = Filter(filterExpr, Expand(projections, windowAttr +: child.output, child)) - val substituedPlan = p transformExpressions { + val substitutedPlan = p transformExpressions { case t: TimeWindow => windowAttr } - substituedPlan.withNewChildren(expandedPlan :: Nil) + substitutedPlan.withNewChildren(expandedPlan :: Nil) + } else if (windowExpressions.size > 1) { + p.failAnalysis("Multiple time window expressions would result in a cartesian product " + + "of rows, therefore they are not currently not supported.") } else { - p // Return unchanged + p // Return unchanged. Analyzer will throw exception later } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index f584a4b73a007..2da4eed481137 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -273,6 +273,7 @@ object FunctionRegistry { expression[UnixTimestamp]("unix_timestamp"), expression[WeekOfYear]("weekofyear"), expression[Year]("year"), + expression[TimeWindow]("window"), // collection functions expression[ArrayContains]("array_contains"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 1379e3c97119e..58f3da7bbbf4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -29,8 +29,7 @@ case class TimeWindow( timeColumn: Expression, windowDuration: Long, slideDuration: Long, - startTime: Long, - private var outputColumnName: String = "window") extends UnaryExpression + startTime: Long) extends UnaryExpression with ImplicitCastInputTypes with Unevaluable with NonSQLExpression { @@ -44,6 +43,10 @@ case class TimeWindow( // This expression is replaced in the analyzer. override lazy val resolved = false + /** + * Validate the inputs for the window duration, slide duration, and start time in addition to + * the input data type. + */ override def checkInputDataTypes(): TypeCheckResult = { val dataTypeCheck = super.checkInputDataTypes() if (dataTypeCheck.isSuccess) { @@ -57,42 +60,15 @@ case class TimeWindow( return TypeCheckFailure(s"The start time ($startTime) must be greater than or equal to 0.") } if (slideDuration > windowDuration) { - return TypeCheckFailure(s"The slide duration ($slideDuration) must be less than or equal to the " + - s"windowDuration ($windowDuration).") + return TypeCheckFailure(s"The slide duration ($slideDuration) must be less than or equal" + + s" to the windowDuration ($windowDuration).") } if (startTime >= slideDuration) { return TypeCheckFailure(s"The start time ($startTime) must be less than the " + - s"slideDuration ($slideDuration).") + s"slideDuration ($slideDuration).") } - return dataTypeCheck - } else { - return dataTypeCheck - } - } - /** - * Validate the inputs for the window duration, slide duration, and start time. - * - * @return Some string with a useful error message for the invalid input. - */ - def validate(): Option[String] = { - if (windowDuration <= 0) { - return Some(s"The window duration ($windowDuration) must be greater than 0.") - } - if (slideDuration <= 0) { - return Some(s"The slide duration ($slideDuration) must be greater than 0.") - } - if (startTime < 0) { - return Some(s"The start time ($startTime) must be greater than or equal to 0.") - } - if (slideDuration > windowDuration) { - return Some(s"The slide duration ($slideDuration) must be less than or equal to the " + - s"windowDuration ($windowDuration).") - } - if (startTime >= slideDuration) { - return Some(s"The start time ($startTime) must be less than the " + - s"slideDuration ($slideDuration).") } - None + dataTypeCheck } } @@ -100,7 +76,7 @@ object TimeWindow { /** * Parses the interval string for a valid time duration. CalendarInterval expects interval * strings to start with the string `interval`. For usability, we prepend `interval` to the string - * if the user ommitted it. + * if the user omitted it. * * @param interval The interval string * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 013362cea137d..1102d2bad3401 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -275,21 +275,21 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "slide duration greater than window in time window", testRelation2.select( - TimeWindow(Literal(1L), "1 second", "2 second", "0 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "2 second", "0 second").as("window")), s"The slide duration (2) must be less than or equal to the windowDuration (1)." :: Nil ) errorTest( "start time greater than slide duration in time window", testRelation.select( - TimeWindow(Literal(1L), "1 second", "1 second", "1 minute").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 minute").as("window")), "The start time (60) must be less than the slideDuration (1)." :: Nil ) errorTest( "start time equal to slide duration in time window", testRelation.select( - TimeWindow(Literal(1L), "1 second", "1 second", "1 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 second").as("window")), "The start time (1) must be less than the slideDuration (1)." :: Nil ) @@ -303,28 +303,28 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "zero window duration in time window", testRelation.select( - TimeWindow(Literal(1L), "0 second", "1 second", "0 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "0 second", "1 second", "0 second").as("window")), "The window duration (0) must be greater than 0." :: Nil ) errorTest( "negative slide duration in time window", testRelation.select( - TimeWindow(Literal(1L), "1 second", "-1 second", "0 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "-1 second", "0 second").as("window")), "The slide duration (-1) must be greater than 0." :: Nil ) errorTest( "zero slide duration in time window", testRelation.select( - TimeWindow(Literal(1L), "1 second", "0 second", "0 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "0 second", "0 second").as("window")), "The slide duration (0) must be greater than 0." :: Nil ) errorTest( "negative start time in time window", testRelation.select( - TimeWindow(Literal(1L), "1 second", "1 second", "-5 second").as("window")), + TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "-5 second").as("window")), "The start time (-5) must be greater than or equal to 0." :: Nil ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index 9f47338f28423..fee15a0286d66 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -61,20 +61,4 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { === seconds) } } - -// test("maxNumOverlapping takes ceiling of window duration over slide duration") { -// assert(TimeWindow(Literal(10L), "5 second", "5 second", "0 second").maxNumOverlapping === 1) -// assert(TimeWindow(Literal(10L), "5 second", "4 second", "0 second").maxNumOverlapping === 2) -// assert(TimeWindow(Literal(10L), "5 second", "3 second", "0 second").maxNumOverlapping === 2) -// assert(TimeWindow(Literal(10L), "5 second", "2 second", "0 second").maxNumOverlapping === 3) -// assert(TimeWindow(Literal(10L), "5 second", "1 second", "0 second").maxNumOverlapping === 5) -// } -// -// test("output column name is window and is a struct") { -// val expr = TimeWindow(Literal(10L), "5 second", "5 second", "0 second") -// assert(expr.outputColumn.name === "window") -// assert(expr.outputColumn.children.head.isInstanceOf[CreateStruct]) -// assert(expr.windowStartCol.name === "start") -// assert(expr.windowEndCol.name === "end") -// } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 2199ecf9781af..726235bc14f51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2554,8 +2554,9 @@ object functions { /** * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). The following example takes the average stock price - * for a one minute window every 10 seconds starting 5 seconds after the hour: + * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. The following + * example takes the average stock price for a one minute window every 10 seconds starting 5 + * seconds after the hour: * * {{{ * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType @@ -2608,7 +2609,8 @@ object functions { /** * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). The windows start beginning at 1970-01-01 00:00:00 UTC. + * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. + * The windows start beginning at 1970-01-01 00:00:00 UTC. * The following example takes the average stock price for a one minute window every 10 seconds: * * {{{ @@ -2650,7 +2652,8 @@ object functions { /** * Generates tumbling time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). The windows start beginning at 1970-01-01 00:00:00 UTC. + * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. + * The windows start beginning at 1970-01-01 00:00:00 UTC. * The following example takes the average stock price for a one minute tumbling window: * * {{{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index ae95987fea906..dd589bf2d069b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -24,7 +24,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { import testImplicits._ - test("tumbling windows") { + test("tumbling window groupBy statement") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC val df = Seq( (1459103974L, 1, "a"), @@ -34,13 +34,12 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkAnswer( df.groupBy(window($"time", "10 seconds")) .agg(count("*").as("counts")) - .orderBy($"window.start".asc) .select("counts"), Seq(Row(1), Row(1), Row(1)) ) } - test("explicit tumbling window") { + test("tumbling window groupBy statement with startTime") { val df = Seq( ("2016-03-27 19:39:34", 1, "a"), ("2016-03-27 19:39:56", 2, "a"), @@ -49,12 +48,11 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkAnswer( df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") .agg(count("*").as("counts")) - .orderBy($"window.start".asc) .select("counts"), Seq(Row(1), Row(1), Row(1))) } - test("?") { + test("tumbling window with multi-column projection") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC val df = Seq( (1459103974L, 1, "a"), @@ -62,13 +60,12 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { (1459103967L, 4, "b")).toDF("time", "value", "id") checkAnswer( df.select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc) .select("value"), Seq(Row(4), Row(1), Row(2)) ) } - test("time windowing - sliding windows") { + test("time windowing - sliding window grouping") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC val df = Seq( (1459103974L, 1, "a"), @@ -78,13 +75,21 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkAnswer( df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) .agg(count("*").as("counts")) - .orderBy($"window.start".asc) .select("counts"), // 2016-03-27 19:39:27 UTC -> 4 bins // 2016-03-27 19:39:34 UTC -> 3 bins // 2016-03-27 19:39:56 UTC -> 3 bins Seq(Row(1), Row(1), Row(1), Row(2), Row(1), Row(1), Row(1), Row(1), Row(1)) ) + } + + test("time windowing - sliding window projection") { + // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + val df = Seq( + (1459103974L, 1, "a"), + (1459103996L, 2, "a"), + (1459103967L, 4, "b")).toDF("time", "value", "id") + checkAnswer( df.select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") .orderBy($"window.start".asc, $"value".desc).select("value"), @@ -95,7 +100,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { ) } - test("esoteric time windowing use cases") { + test("windowing combined with explode expression") { // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC val df = Seq( (1459103974L, 1, Seq("a", "b")), @@ -110,39 +115,36 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { } test("string timestamps") { - val df2 = Seq( + val df = Seq( ("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), (null, 3), (null, 4)).toDF("time", "value") checkDataset( - df2 - .select(window($"time", "10 seconds"), $"value") - .orderBy($"window.start".asc) + df.select(window($"time", "10 seconds"), $"value") .select("value") .as[Int], 1, 2) // null columns are dropped } - test("another test") { - val df2 = Seq( + test("time window joins") { + val df = Seq( ("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), (null, 3), (null, 4)).toDF("time", "value") - val df3 = Seq( + val df2 = Seq( ("2016-03-27 09:00:02", 3), ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") - checkAnswer( - df2.select(window($"time", "10 seconds"), $"value").join( - df3 - .select(window($"time", "10 seconds"), $"othervalue"), Seq("window")).groupBy("window") - .agg((sum("value") + sum("othervalue")).as("total")) - .orderBy($"window.start".asc).select("total"), + df.select(window($"time", "10 seconds"), $"value").join( + df2.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")) + .groupBy("window") + .agg((sum("value") + sum("othervalue")).as("total")) + .orderBy($"window.start".asc).select("total"), Seq(Row(4), Row(8))) } @@ -150,13 +152,20 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") checkAnswer( df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), + .orderBy($"window.start".asc).select("value"), Seq(Row(1), Row(2)) ) } - ignore("multiple time windows in a single operator") { - + test("multiple time windows in a single operator throws nice exception") { + val df = Seq( + ("2016-03-27 09:00:02", 3), + ("2016-03-27 09:00:35", 6)).toDF("time", "value") + val e = intercept[AnalysisException] { + df.select(window($"time", "10 second"), window($"time", "15 second")).collect() + } + assert(e.getMessage.contains( + "Multiple time window expressions would result in a cartesian product")) } test("aliased windows") { @@ -166,8 +175,8 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkAnswer( df.select(window($"time", "10 seconds").as("time_window"), $"value") - .orderBy($"time_window.start".asc) - .select("value"), + .orderBy($"time_window.start".asc) + .select("value"), Seq(Row(1), Row(2)) ) } From f756556a40e9696331a242f93408030f064ef852 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 30 Mar 2016 18:20:57 -0700 Subject: [PATCH 18/21] finished --- .../sql/catalyst/analysis/Analyzer.scala | 13 +- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/TimeWindow.scala | 35 ++++- .../analysis/AnalysisErrorSuite.scala | 7 + .../expressions/TimeWindowSuite.scala | 44 ++++--- .../org/apache/spark/sql/functions.scala | 14 +- .../sql/DataFrameTimeWindowingSuite.scala | 121 +++++++++++++----- 7 files changed, 166 insertions(+), 70 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a9fb3a2612472..b18c0f1778087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1635,29 +1635,27 @@ object TimeWindowing extends Rule[LogicalPlan] { // Only support a single window expression for now if (windowExpressions.size == 1 && windowExpressions.head.timeColumn.resolved && - windowExpressions.head.timeColumn.dataType == TimestampType && windowExpressions.head.checkInputDataTypes().isSuccess) { val window = windowExpressions.head val windowAttr = AttributeReference("window", window.dataType)() val maxNumOverlapping = math.ceil(window.windowDuration * 1.0 / window.slideDuration).toInt val windows = Seq.tabulate(maxNumOverlapping + 1) { i => - val windowId = Ceil((Cast(window.timeColumn, LongType) - window.startTime) / - window.slideDuration) + val windowId = Ceil((PreciseTimestamp(window.timeColumn) - window.startTime) / + window.slideDuration) val windowStart = (windowId + i - maxNumOverlapping) * window.slideDuration + window.startTime val windowEnd = windowStart + window.windowDuration - // the 1000000 is necessary for properly casting a LongType to a TimestampType CreateNamedStruct( - Literal(WINDOW_START) :: windowStart * 1000000 :: - Literal(WINDOW_END) :: windowEnd * 1000000 :: Nil) + Literal(WINDOW_START) :: windowStart :: + Literal(WINDOW_END) :: windowEnd :: Nil) } val projections = windows.map(_ +: p.children.head.output) val filterExpr = - window.timeColumn >= windowAttr.getField(WINDOW_START) + window.timeColumn >= windowAttr.getField(WINDOW_START) && window.timeColumn < windowAttr.getField(WINDOW_END) val expandedPlan = @@ -1676,5 +1674,4 @@ object TimeWindowing extends Rule[LogicalPlan] { p // Return unchanged. Analyzer will throw exception later } } - } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index a965cc8d5332b..7f7ec6bce72f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -759,7 +759,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { (c, evPrim, evNull) => s"$evPrim = (int) $c;" } - private[this] def castToLongCode(from: DataType): CastFunction = from match { + protected def castToLongCode(from: DataType): CastFunction = from match { case StringType => (c, evPrim, evNull) => s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 58f3da7bbbf4b..5e6e58cc48dab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -22,6 +22,7 @@ import org.apache.commons.lang.StringUtils import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure +import org.apache.spark.sql.catalyst.expressions.codegen.{ExprCode, CodegenContext} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -79,10 +80,10 @@ object TimeWindow { * if the user omitted it. * * @param interval The interval string - * @return The interval duration in seconds. SparkSQL casts TimestampType to Long in seconds, - * therefore we use seconds here as well. + * @return The interval duration in microseconds. SparkSQL casts TimestampType has microsecond + * precision. */ - private def getIntervalInSeconds(interval: String): Long = { + private def getIntervalInMicroSeconds(interval: String): Long = { if (StringUtils.isBlank(interval)) { throw new IllegalArgumentException( "The window duration, slide duration and start time cannot be null or blank.") @@ -97,7 +98,11 @@ object TimeWindow { throw new IllegalArgumentException( s"The provided interval ($interval) did not correspond to a valid interval string.") } - (cal.months * 4 * CalendarInterval.MICROS_PER_WEEK + cal.microseconds) / 1000000 + if (cal.months > 0) { + throw new IllegalArgumentException( + s"Intervals greater than a month is not supported ($interval).") + } + cal.microseconds } def apply( @@ -106,8 +111,24 @@ object TimeWindow { slideDuration: String, startTime: String): TimeWindow = { TimeWindow(timeColumn, - getIntervalInSeconds(windowDuration), - getIntervalInSeconds(slideDuration), - getIntervalInSeconds(startTime)) + getIntervalInMicroSeconds(windowDuration), + getIntervalInMicroSeconds(slideDuration), + getIntervalInMicroSeconds(startTime)) + } +} + +/** + * Expression used internally to convert the TimestampType to Long without losing + * precision, i.e. in microseconds. Used in time windowing. + */ +case class PreciseTimestamp(child: Expression) extends UnaryExpression with ExpectsInputTypes { + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def dataType: DataType = LongType + override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.gen(ctx) + eval.code + + s"""boolean ${ev.isNull} = ${eval.isNull}; + |${ctx.javaType(dataType)} ${ev.value} = ${eval.value}; + """.stripMargin } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 1102d2bad3401..4e231e7ad8d9a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -328,6 +328,13 @@ class AnalysisErrorSuite extends AnalysisTest { "The start time (-5) must be greater than or equal to 0." :: Nil ) + errorTest( + "time window longer than a month", + testRelation.select( + TimeWindow(Literal("2016-01-01 01:01:01"), "1 month", "1 second", "-5 second").as("window")), + "The start time (-5) must be greater than or equal to 0." :: Nil + ) + test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) // Since we manually construct the logical plan at here and Sum only accept diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index fee15a0286d66..71f969aee2ee4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -28,34 +28,46 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("invalid intervals throw exception") { + private def checkErrorMessage(msg: String, value: String): Unit = { val validDuration = "10 second" val validTime = "5 second" - def checkErrorMessage(msg: String, value: String): Unit = { - val e1 = intercept[IllegalArgumentException] { - TimeWindow(Literal(10L), value, validDuration, validTime).windowDuration - } - val e2 = intercept[IllegalArgumentException] { - TimeWindow(Literal(10L), validDuration, value, validTime).slideDuration - } - val e3 = intercept[IllegalArgumentException] { - TimeWindow(Literal(10L), validDuration, validDuration, value).startTime - } - Seq(e1, e2, e3).foreach { e => - e.getMessage.contains(msg) - } + val e1 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), value, validDuration, validTime).windowDuration + } + val e2 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), validDuration, value, validTime).slideDuration + } + val e3 = intercept[IllegalArgumentException] { + TimeWindow(Literal(10L), validDuration, validDuration, value).startTime } + Seq(e1, e2, e3).foreach { e => + e.getMessage.contains(msg) + } + } + + test("blank intervals throw exception") { for (blank <- Seq(null, " ", "\n", "\t")) { checkErrorMessage( "The window duration, slide duration and start time cannot be null or blank.", blank) } + } + + test("invalid intervals throw exception") { checkErrorMessage( "did not correspond to a valid interval string.", "2 apples") } - test("interval strings work with and without 'interval' prefix and returns seconds") { + test("intervals greater than a month throws exception") { + checkErrorMessage( + "Intervals greater than or equal to a month is not supported (1 month).", "1 month") + } + + test("interval strings work with and without 'interval' prefix and return microseconds") { val validDuration = "10 second" - for ((text, seconds) <- Seq(("1 second", 1), ("1 minute", 60), ("2 hours", 7200))) { + for ((text, seconds) <- Seq( + ("1 second", 1000000), // 1e6 + ("1 minute", 60000000), // 6e7 + ("2 hours", 7200000000L))) { // 72e9 assert(TimeWindow(Literal(10L), text, validDuration, "0 seconds").windowDuration === seconds) assert(TimeWindow(Literal(10L), "interval " + text, validDuration, "0 seconds").windowDuration === seconds) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 726235bc14f51..1c35c1164a228 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2554,9 +2554,9 @@ object functions { /** * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. The following - * example takes the average stock price for a one minute window every 10 seconds starting 5 - * seconds after the hour: + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The following example takes the average stock price for + * a one minute window every 10 seconds starting 5 seconds after the hour: * * {{{ * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType @@ -2609,8 +2609,8 @@ object functions { /** * Bucketize rows into one or more time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. - * The windows start beginning at 1970-01-01 00:00:00 UTC. + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. * The following example takes the average stock price for a one minute window every 10 seconds: * * {{{ @@ -2652,8 +2652,8 @@ object functions { /** * Generates tumbling time windows given a timestamp specifying column. Window * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window - * [12:05,12:10) but not in [12:00,12:05). Windows must be at least 1 second long. - * The windows start beginning at 1970-01-01 00:00:00 UTC. + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. * The following example takes the average stock price for a one minute tumbling window: * * {{{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index dd589bf2d069b..1a44bb2cd20a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -17,23 +17,37 @@ package org.apache.spark.sql +import java.util.TimeZone + +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{TimestampType, LongType, StringType} -class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { +class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { import testImplicits._ + override def beforeEach(): Unit = { + super.beforeEach() + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + } + + override def afterEach(): Unit = { + super.beforeEach() + TimeZone.setDefault(null) + } + test("tumbling window groupBy statement") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC val df = Seq( - (1459103974L, 1, "a"), - (1459103996L, 2, "a"), - (1459103967L, 4, "b")).toDF("time", "value", "id") - + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") checkAnswer( df.groupBy(window($"time", "10 seconds")) .agg(count("*").as("counts")) + .orderBy($"window.start".asc) .select("counts"), Seq(Row(1), Row(1), Row(1)) ) @@ -48,47 +62,61 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkAnswer( df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") .agg(count("*").as("counts")) + .orderBy($"window.start".asc) .select("counts"), Seq(Row(1), Row(1), Row(1))) } test("tumbling window with multi-column projection") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC val df = Seq( - (1459103974L, 1, "a"), - (1459103996L, 2, "a"), - (1459103967L, 4, "b")).toDF("time", "value", "id") + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + checkAnswer( df.select(window($"time", "10 seconds"), $"value") - .select("value"), - Seq(Row(4), Row(1), Row(2)) + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"value"), + Seq( + Row("2016-03-27 19:39:20", "2016-03-27 19:39:30", 4), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:50", "2016-03-27 19:40:00", 2) + ) ) } - test("time windowing - sliding window grouping") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + test("sliding window grouping") { val df = Seq( - (1459103974L, 1, "a"), - (1459103996L, 2, "a"), - (1459103967L, 4, "b")).toDF("time", "value", "id") + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") checkAnswer( df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) .agg(count("*").as("counts")) - .select("counts"), + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), // 2016-03-27 19:39:27 UTC -> 4 bins // 2016-03-27 19:39:34 UTC -> 3 bins // 2016-03-27 19:39:56 UTC -> 3 bins - Seq(Row(1), Row(1), Row(1), Row(2), Row(1), Row(1), Row(1), Row(1), Row(1)) + Seq( + Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), + Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), + Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), + Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), + Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), + Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), + Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) ) } - test("time windowing - sliding window projection") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC, 2016-03-27 19:39:27 UTC + test("sliding window projection") { val df = Seq( - (1459103974L, 1, "a"), - (1459103996L, 2, "a"), - (1459103967L, 4, "b")).toDF("time", "value", "id") + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") checkAnswer( df.select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") @@ -101,10 +129,9 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { } test("windowing combined with explode expression") { - // 2016-03-27 19:39:34 UTC, 2016-03-27 19:39:56 UTC val df = Seq( - (1459103974L, 1, Seq("a", "b")), - (1459103996L, 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + ("2016-03-27 19:39:34", 1, Seq("a", "b")), + ("2016-03-27 19:39:56", 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") checkAnswer( df.select(window($"time", "10 seconds"), $"value", explode($"ids")) @@ -114,7 +141,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { ) } - test("string timestamps") { + test("null timestamps") { val df = Seq( ("2016-03-27 09:00:05", 1), ("2016-03-27 09:00:32", 2), @@ -123,6 +150,7 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { checkDataset( df.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc) .select("value") .as[Int], 1, 2) // null columns are dropped @@ -149,11 +177,16 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { } test("negative timestamps") { - val df4 = Seq((2L, 1), (12L, 2)).toDF("time", "value") + val df4 = Seq( + ("1970-01-01 00:00:02", 1), + ("1970-01-01 00:00:12", 2)).toDF("time", "value") checkAnswer( df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") - .orderBy($"window.start".asc).select("value"), - Seq(Row(1), Row(2)) + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"value"), + Seq( + Row("1969-12-31 23:59:55", "1970-01-01 00:00:05", 1), + Row("1970-01-01 00:00:05", "1970-01-01 00:00:15", 2)) ) } @@ -180,4 +213,30 @@ class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext { Seq(Row(1), Row(2)) ) } + + test("millisecond precision sliding windows") { + val df = Seq( + ("2016-03-27 09:00:00.41", 3), + ("2016-03-27 09:00:00.62", 6), + ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") + checkAnswer( + df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), + Seq( + Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), + Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), + Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), + Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), + Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), + Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), + Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), + Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), + Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), + Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), + Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), + Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) + ) + } } From 4eb57c5e80a8802d4a61f70f493643139b04dfeb Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 30 Mar 2016 18:23:23 -0700 Subject: [PATCH 19/21] minor clean up --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../spark/sql/catalyst/analysis/AnalysisErrorSuite.scala | 7 ------- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 7f7ec6bce72f8..a965cc8d5332b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -759,7 +759,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { (c, evPrim, evNull) => s"$evPrim = (int) $c;" } - protected def castToLongCode(from: DataType): CastFunction = from match { + private[this] def castToLongCode(from: DataType): CastFunction = from match { case StringType => (c, evPrim, evNull) => s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 4e231e7ad8d9a..1102d2bad3401 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -328,13 +328,6 @@ class AnalysisErrorSuite extends AnalysisTest { "The start time (-5) must be greater than or equal to 0." :: Nil ) - errorTest( - "time window longer than a month", - testRelation.select( - TimeWindow(Literal("2016-01-01 01:01:01"), "1 month", "1 second", "-5 second").as("window")), - "The start time (-5) must be greater than or equal to 0." :: Nil - ) - test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) // Since we manually construct the logical plan at here and Sum only accept From 8bc9799db4e252f334e8239765732ded6dbe32c4 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 30 Mar 2016 18:26:29 -0700 Subject: [PATCH 20/21] fix ss --- .../org/apache/spark/sql/catalyst/expressions/TimeWindow.scala | 3 +-- .../org/apache/spark/sql/DataFrameTimeWindowingSuite.scala | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 5e6e58cc48dab..8e13833486931 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.commons.lang.StringUtils -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure -import org.apache.spark.sql.catalyst.expressions.codegen.{ExprCode, CodegenContext} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala index 1a44bb2cd20a9..e8103a31d5833 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{TimestampType, LongType, StringType} +import org.apache.spark.sql.types.StringType class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { From ff27b7aef7478ef17c1cede901688deb78d26a4c Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 31 Mar 2016 09:00:40 -0700 Subject: [PATCH 21/21] fix test --- .../catalyst/analysis/AnalysisErrorSuite.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 1102d2bad3401..ad101d1c406b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -276,56 +276,56 @@ class AnalysisErrorSuite extends AnalysisTest { "slide duration greater than window in time window", testRelation2.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "2 second", "0 second").as("window")), - s"The slide duration (2) must be less than or equal to the windowDuration (1)." :: Nil + s"The slide duration " :: " must be less than or equal to the windowDuration " :: Nil ) errorTest( "start time greater than slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 minute").as("window")), - "The start time (60) must be less than the slideDuration (1)." :: Nil + "The start time " :: " must be less than the slideDuration " :: Nil ) errorTest( "start time equal to slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "1 second").as("window")), - "The start time (1) must be less than the slideDuration (1)." :: Nil + "The start time " :: " must be less than the slideDuration " :: Nil ) errorTest( "negative window duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "-1 second", "1 second", "0 second").as("window")), - "The window duration (-1) must be greater than 0." :: Nil + "The window duration " :: " must be greater than 0." :: Nil ) errorTest( "zero window duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "0 second", "1 second", "0 second").as("window")), - "The window duration (0) must be greater than 0." :: Nil + "The window duration " :: " must be greater than 0." :: Nil ) errorTest( "negative slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "-1 second", "0 second").as("window")), - "The slide duration (-1) must be greater than 0." :: Nil + "The slide duration " :: " must be greater than 0." :: Nil ) errorTest( "zero slide duration in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "0 second", "0 second").as("window")), - "The slide duration (0) must be greater than 0." :: Nil + "The slide duration" :: " must be greater than 0." :: Nil ) errorTest( "negative start time in time window", testRelation.select( TimeWindow(Literal("2016-01-01 01:01:01"), "1 second", "1 second", "-5 second").as("window")), - "The start time (-5) must be greater than or equal to 0." :: Nil + "The start time" :: "must be greater than or equal to 0." :: Nil ) test("SPARK-6452 regression test") {