Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.streaming.binning

import scala.reflect.ClassTag

import org.apache.spark.streaming.dstream.DStream

class BinStream[T: ClassTag](
@transient ds: DStream[T], sizeInNumBatches: Int, delayInNumBatches: Int) {
def getDStream = ds
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* 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.streaming.binning

import org.apache.spark.streaming.dstream.{DStream, ProratedEventDStream, BinAlignedWindowDStream, PulsatingWindowDStream}
import org.apache.spark.streaming.Time
import scala.reflect.ClassTag


class BinStreamer[T: ClassTag](
@transient ds: DStream[T], getStartTime: (T) => Time, getEndTime: (T) => Time
) extends Serializable {

def prorate(binStart: Time, binEnd: Time)(x: T) = {

val sx = getStartTime(x)
val ex = getEndTime(x)

if (ex == sx) {
(x, 1.0)
}
else {

// Even though binStart is not inclusive,
// setting s = binStart implies limit s as x approaches binStart+
val s = if (sx > binStart) sx else binStart

val e = if (ex < binEnd) ex else binEnd

(x, (e - s) / (ex - sx))
}
}

def filter(binStart: Time, binEnd: Time)(x: T) = {

// The flow is starting in the subsequent bin
if (getStartTime(x) > binEnd) false

// The flow ended in the prior bin
else if (getEndTime(x) <= binStart) false

// s approaches from binEnd+
else if (getStartTime(x) == binEnd && getEndTime(x) > binEnd) false

// defensive check
else if (getStartTime(x) > getEndTime(x)) false

else true

}

def numStreams(sz: Int, delay: Int) = (sz + delay - 1)/sz + 1

def incrementalStreams(sizeInNumBatches: Int, delayInNumBatches: Int) = {

val num = numStreams(sizeInNumBatches, delayInNumBatches)

Array.tabulate(num)(
delayNumBins =>
new BinStream(
new ProratedEventDStream[T](ds, filter, prorate, sizeInNumBatches, delayNumBins),
sizeInNumBatches, delayNumBins)
)
}

def finalStream(sizeInNumBatches: Int, delayInNumBatches: Int) = {

val num = numStreams(sizeInNumBatches, delayInNumBatches)

new BinStream(
new ProratedEventDStream[T](
new BinAlignedWindowDStream(ds, sizeInNumBatches, num - 1),
filter, prorate, sizeInNumBatches, num - 1),
sizeInNumBatches, num - 1
)
}

def updatedStreams(sizeInNumBatches: Int, delayInNumBatches: Int) = {

val num = numStreams(sizeInNumBatches, delayInNumBatches)

Array.tabulate(num)(
delayNumBins => new BinStream(
new ProratedEventDStream(
new PulsatingWindowDStream(ds, sizeInNumBatches, delayNumBins),
filter, prorate, sizeInNumBatches, delayNumBins),
sizeInNumBatches, delayNumBins)
)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.streaming.dstream

import org.apache.spark.rdd.{UnionRDD, RDD}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Time, Duration, Interval}
import scala.reflect.ClassTag


private[streaming]
class BinAlignedWindowDStream[T: ClassTag](
parent: DStream[T],
sizeNumBatches: Int,
delayNumBins: Int)
extends DStream[T](parent.ssc) {

parent.persist(StorageLevel.MEMORY_ONLY_SER)

override def dependencies = List(parent)

override def slideDuration: Duration = parent.slideDuration

override def parentRememberDuration: Duration =
rememberDuration + parent.slideDuration * sizeNumBatches * (delayNumBins + 1)

override def compute(validTime: Time): Option[RDD[T]] = {

val binStart =
(validTime - Duration(1)).floor(slideDuration * sizeNumBatches) -
slideDuration * sizeNumBatches * delayNumBins

if ((validTime - binStart).isMultipleOf(slideDuration * sizeNumBatches)) {
val currentWindow = new Interval(binStart + slideDuration, validTime)
Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
}
else {
None
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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.streaming.dstream

import org.apache.spark.streaming.{Time, Duration}
import org.apache.spark.rdd.RDD
import scala.reflect.ClassTag



private[streaming]
class ProratedEventDStream[T: ClassTag](parent: DStream[T],
filterFunc: (Time,Time) => T => Boolean,
prorateFunc: (Time,Time) => T => (T,Double),
sizeNumBatches: Int,

delayNumBins: Int

)
extends DStream[(T, Double)](parent.ssc) {

override def dependencies = List(parent)

override def slideDuration: Duration = parent.slideDuration

override def compute(validTime: Time) = {

//
// Assumption: start(x) <= end(x) <= boundaryEnd
//

def binStart = (validTime - Duration(1)).floor(slideDuration * sizeNumBatches) -
slideDuration * sizeNumBatches * delayNumBins
def binEnd = binStart + slideDuration * sizeNumBatches

parent.getOrCompute(validTime).map(

_.filter(
filterFunc(binStart, binEnd)
).map(
prorateFunc(binStart, binEnd)
)
)
}
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.streaming.dstream

import org.apache.spark.storage.StorageLevel
import org.apache.spark.rdd.{UnionRDD, RDD}
import org.apache.spark.streaming.{Time, Duration, Interval}
import scala.reflect.ClassTag


private[streaming]
class PulsatingWindowDStream[T: ClassTag](parent: DStream[T],
sizeNumBatches: Int,
delayNumBins: Int)
extends DStream[T](parent.ssc) {

parent.persist(StorageLevel.MEMORY_ONLY_SER)

override def dependencies = List(parent)

override def slideDuration: Duration = parent.slideDuration

override def parentRememberDuration: Duration =
rememberDuration + parent.slideDuration * sizeNumBatches * (delayNumBins + 1)

override def compute(validTime: Time): Option[RDD[T]] = {

val binStart = (validTime - Duration(1)).floor(slideDuration * sizeNumBatches) -
slideDuration * sizeNumBatches * delayNumBins

val currentWindow = new Interval(binStart + slideDuration, validTime)

Some(new UnionRDD(parent.ssc.sc, parent.slice(currentWindow)))
}
}



Loading