Permalink
Show file tree
Hide file tree
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Showing
5 changed files
with
143 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,25 @@ | ||
package sample | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.streaming.fluentd.FluentdUtils | ||
import org.apache.spark.streaming.{Milliseconds, StreamingContext} | ||
|
||
object EventCounter { | ||
def main(args: Array[String]): Unit = { | ||
val host = args(0) | ||
val port = args(1).toInt | ||
|
||
val sparkConf = new SparkConf().setAppName("fluentd-event-counter") | ||
val ssc = new StreamingContext(sparkConf, Milliseconds(5000)) | ||
|
||
val stream = FluentdUtils.createStream(ssc, host, port, StorageLevel.MEMORY_ONLY) | ||
|
||
stream.map { eventStream => | ||
eventStream.getTag.getName -> eventStream.getEntries.size().toLong | ||
}.reduceByKey(_ + _).print() | ||
|
||
ssc.start() | ||
ssc.awaitTerminationOrTimeout(300 * 1000) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,16 @@ | ||
package org.apache.spark.streaming.fluentd | ||
|
||
import influent.EventStream | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.streaming.StreamingContext | ||
import org.apache.spark.streaming.dstream.ReceiverInputDStream | ||
import org.apache.spark.streaming.receiver.Receiver | ||
|
||
private[fluentd] class FluentdInputDStream(ssc: StreamingContext, | ||
host: String, | ||
port: Int, | ||
storageLevel: StorageLevel) | ||
extends ReceiverInputDStream[EventStream](ssc) { | ||
|
||
override def getReceiver(): Receiver[EventStream] = new FluentdReceiver(host, port, storageLevel) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,67 @@ | ||
package org.apache.spark.streaming.fluentd | ||
|
||
import java.net.InetSocketAddress | ||
import java.util.concurrent.CompletableFuture | ||
|
||
import influent.EventStream | ||
import influent.forward.{ForwardCallback, ForwardServer} | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.streaming.fluentd.FluentdReceiver.SparkStreamingCallback | ||
import org.apache.spark.streaming.receiver.Receiver | ||
|
||
import scala.util.control.NonFatal | ||
|
||
private[fluentd] class FluentdReceiver(host: String, | ||
port: Int, | ||
storageLevel: StorageLevel) | ||
extends Receiver[EventStream](storageLevel) with Logging { | ||
private[this] var server: Option[ForwardServer] = None | ||
|
||
override def onStart(): Unit = { | ||
synchronized { | ||
server match { | ||
case None => | ||
val forwardServer = new ForwardServer.Builder(new SparkStreamingCallback(this)) | ||
.workerPoolSize(1) | ||
.localAddress(new InetSocketAddress(host, port)) | ||
.build() | ||
forwardServer.start() | ||
server = Some(forwardServer) | ||
logInfo(s"Fluentd server started on $host:$port.") | ||
case Some(_) => | ||
logWarning("Fluentd receiver being asked to start more then once with out close.") | ||
} | ||
} | ||
logInfo("Fluentd receiver started.") | ||
} | ||
|
||
override def onStop(): Unit = { | ||
synchronized { | ||
server.foreach { x => | ||
x.shutdown() | ||
server = None | ||
} | ||
logInfo("Fluentd receiver stopped.") | ||
} | ||
} | ||
|
||
override def preferredLocation: Option[String] = Some(host) | ||
} | ||
|
||
private[fluentd] object FluentdReceiver { | ||
|
||
private class SparkStreamingCallback(receiver: FluentdReceiver) extends ForwardCallback { | ||
override def consume(stream: EventStream): CompletableFuture[Void] = { | ||
val future = new CompletableFuture[Void]() | ||
try { | ||
receiver.store(stream) | ||
future.complete(null) | ||
} catch { | ||
case NonFatal(e) => future.completeExceptionally(e) | ||
} | ||
future | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,13 @@ | ||
package org.apache.spark.streaming.fluentd | ||
|
||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.streaming.StreamingContext | ||
|
||
object FluentdUtils { | ||
def createStream(ssc: StreamingContext, | ||
host: String, | ||
port: Int, | ||
storageLevel: StorageLevel): FluentdInputDStream = { | ||
new FluentdInputDStream(ssc, host, port, storageLevel) | ||
} | ||
} |