Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add other streaming trigger than only ProcessingTime. #7

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
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
Expand Up @@ -43,6 +43,6 @@ object ConfigNode {

val name = Try(config.getString("Name")).getOrElse(s"$typeName-${randomUUID().toString}")

ConfigNode(name, typeName, config.withoutPath("Type").withoutPath("Name"))
ConfigNode(name, typeName, config.withoutPath("Type"))
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
package com.amadeus.dataio.config.fields

import com.typesafe.config.Config
import org.apache.spark.sql.streaming.Trigger

import scala.concurrent.duration.Duration
import scala.util.Try

/**
* Retrieve the trigger to be used from the configuration.
*
* See documentation: [[https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#continuous-processing]]
*
* Trigger = "" AvailableNow or Continuous. Optional in case of ProcessingTime trigger.
* Duration = "1 minute". Optional in case of AvailableNow trigger.
*/
trait StreamingTriggerConfigurator {

/**
* @param config The typesafe Config object holding the configuration.
* @return the trigger of None is not defined.
* @throws IllegalArgumentException in case the combination of Trigger and Duration is not supported.
*/
def getStreamingTrigger(implicit config: Config): Option[Trigger] = {
guleclerc marked this conversation as resolved.
Show resolved Hide resolved
val streamingTrigger = Try(config.getString("Trigger")).toOption
val duration = Try(config.getString("Duration")).toOption

(streamingTrigger, duration) match {
case (Some("AvailableNow"), _) => Some(Trigger.AvailableNow())
case (Some("Continuous"), Some(duration)) => Some(Trigger.Continuous(Duration(duration)))
case (None, Some(duration)) => Some(Trigger.ProcessingTime(Duration(duration)))
case (None, None) => None
case _ => throw new IllegalArgumentException(s"The couple ($streamingTrigger, $duration) is not part of the allowed values")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,3 +14,4 @@ package object fields
with PartitionByConfigurator
with TimeoutConfigurator
with SchemaConfigurator
with StreamingTriggerConfigurator
Original file line number Diff line number Diff line change
Expand Up @@ -2,19 +2,17 @@ package com.amadeus.dataio.pipes.elk.streaming

import com.amadeus.dataio.core.{Logging, Output}
import com.amadeus.dataio.pipes.elk.ElkOutputCommons
import com.amadeus.dataio.pipes.elk.ElkOutputCommons.{DefaultSuffixDatePattern, checkNodesIsDefined, checkPortIsDefined}
import com.typesafe.config.{Config, ConfigFactory}
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.sql.{Dataset, SparkSession}

import scala.concurrent.duration.Duration
import scala.util.Try

/**
* Allows to write stream data to Elasticsearch with automatic date sub-indexing.
*
* @param index the Index to write to.
* @param processingTimeTrigger processingTimeTrigger.
* @param trigger the trigger to be used for the streaming query.
* @param timeout timeout in milliseconds.
* @param mode mode.
* @param options options.
Expand All @@ -25,7 +23,7 @@ import scala.util.Try
*/
case class ElkOutput(
index: String,
processingTimeTrigger: Trigger,
trigger: Option[Trigger],
timeout: Long,
mode: String,
options: Map[String, String] = Map.empty,
Expand All @@ -45,19 +43,22 @@ case class ElkOutput(
*/
def write[T](data: Dataset[T])(implicit spark: SparkSession): Unit = {
val fullIndexName = computeFullIndexName()
logger.info(s"Write dataframe to Elasticsearch index [$fullIndexName]")
logger.info(s"Write dataframe to Elasticsearch index [$fullIndexName] using trigger [$trigger]")

val queryName = createQueryName()

val streamWriter = data.writeStream
var streamWriter = data.writeStream
.queryName(queryName)
.outputMode(mode)
.format(Format)
.options(options)

val streamingQuery = streamWriter
.trigger(processingTimeTrigger)
.start(fullIndexName)
streamWriter = trigger match {
case Some(trigger) => streamWriter.trigger(trigger)
case _ => streamWriter
}

val streamingQuery = streamWriter.start(fullIndexName)

streamingQuery.awaitTermination(timeout)
streamingQuery.stop()
Expand All @@ -81,6 +82,7 @@ case class ElkOutput(
object ElkOutput {
import com.amadeus.dataio.config.fields._
import com.amadeus.dataio.pipes.elk.ElkConfigurator._
import com.amadeus.dataio.pipes.elk.ElkOutputCommons.{DefaultSuffixDatePattern, checkNodesIsDefined, checkPortIsDefined}

/**
* Creates an ElkOutput based on a given configuration.
Expand All @@ -94,8 +96,7 @@ object ElkOutput {

val mode = config.getString("Mode")

val duration = Duration(config.getString("Duration"))
val processingTimeTrigger = Trigger.ProcessingTime(duration)
val trigger = getStreamingTrigger

val timeout = getTimeout

Expand All @@ -112,7 +113,7 @@ object ElkOutput {

ElkOutput(
index = index,
processingTimeTrigger = processingTimeTrigger,
trigger = trigger,
timeout = timeout,
mode = mode,
options = options,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,15 +5,14 @@ import com.typesafe.config.{Config, ConfigFactory}
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.sql.{Dataset, SparkSession}

import scala.concurrent.duration.Duration
import scala.util.Try

/**
* Class for reading kafka dataframe
*
* @param brokers brokers
* @param topic topic
* @param processingTimeTrigger processingTimeTrigger.
* @param trigger the trigger to be used for the streaming query.
* @param timeout timeout in milliseconds.
* @param mode mode.
* @param options options
Expand All @@ -23,7 +22,7 @@ import scala.util.Try
case class KafkaOutput(
guleclerc marked this conversation as resolved.
Show resolved Hide resolved
brokers: String,
topic: Option[String],
processingTimeTrigger: Trigger,
trigger: Option[Trigger],
timeout: Long,
mode: String,
options: Map[String, String] = Map(),
Expand All @@ -39,23 +38,30 @@ case class KafkaOutput(
* @param spark The SparkSession which will be used to write the data.
*/
def write[T](data: Dataset[T])(implicit spark: SparkSession): Unit = {
logger.info(s"Write dataframe to kafka [$topic]")
logger.info(s"Write dataframe to kafka [$topic] using trigger [$trigger]")

val queryName = createQueryName()

val streamWriter = data.writeStream
var fullOptions = options + ("kafka.bootstrap.servers" -> brokers)

fullOptions = topic match {
case Some(kafkaTopic) => options + ("topic" -> kafkaTopic)
case _ => options
}

var streamWriter = data.writeStream
.queryName(queryName)
.format("kafka")
.options(options)
.option("kafka.bootstrap.servers", brokers)
.options(fullOptions)
.outputMode(mode)
.trigger(processingTimeTrigger)

val streamingQuery = topic match {
case Some(t) => streamWriter.option("topic", t).start()
case _ => streamWriter.start()
streamWriter = trigger match {
case Some(trigger) => streamWriter.trigger(trigger)
case _ => streamWriter
}

val streamingQuery = streamWriter.start()

streamingQuery.awaitTermination(timeout)
streamingQuery.stop()
}
Expand All @@ -68,10 +74,10 @@ case class KafkaOutput(
private[streaming] def createQueryName(): String = {

(outputName, topic) match {
case (Some(name), Some(t)) => s"QN_${name}_${t}_${java.util.UUID.randomUUID}"
case (Some(name), None) => s"QN_${name}_${java.util.UUID.randomUUID}"
case (None, Some(t)) => s"QN_KafkaOutput_${t}_${java.util.UUID.randomUUID}"
case _ => s"QN_KafkaOutput_${java.util.UUID.randomUUID}"
case (Some(name), Some(kafkaTopic)) => s"QN_${name}_${kafkaTopic}_${java.util.UUID.randomUUID}"
case (Some(name), None) => s"QN_${name}_${java.util.UUID.randomUUID}"
case (None, Some(kafkaTopic)) => s"QN_${kafkaTopic}_${java.util.UUID.randomUUID}"
case _ => s"QN_KafkaOutput_${java.util.UUID.randomUUID}"
}

}
Expand All @@ -92,8 +98,7 @@ object KafkaOutput {
val brokers = getBroker
val topic = getTopic

val duration = Duration(config.getString("Duration"))
val processingTimeTrigger = Trigger.ProcessingTime(duration)
val trigger = getStreamingTrigger

val timeout = getTimeout
val mode = config.getString("Mode")
Expand All @@ -104,7 +109,7 @@ object KafkaOutput {
KafkaOutput(
brokers,
topic,
processingTimeTrigger,
trigger,
timeout,
mode,
options,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@ import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.sql.{Dataset, SparkSession}

import java.io.File
import scala.concurrent.duration.Duration
import scala.util.Try

/**
Expand All @@ -15,7 +14,7 @@ import scala.util.Try
* @param format the output format.
* @param path the path.
* @param partitioningColumns the columns to partition by.
* @param processingTimeTrigger processingTimeTrigger.
* @param trigger the trigger to be used for the streaming query.
* @param timeout timeout in milliseconds.
* @param mode mode.
* @param options options.
Expand All @@ -26,7 +25,7 @@ case class StorageOutput(
format: String,
path: String,
partitioningColumns: Seq[String],
processingTimeTrigger: Trigger,
trigger: Option[Trigger],
timeout: Long,
mode: String,
options: Map[String, String] = Map(),
Expand All @@ -42,7 +41,7 @@ case class StorageOutput(
* @param spark The SparkSession which will be used to write the data.
*/
def write[T](data: Dataset[T])(implicit spark: SparkSession): Unit = {
logger.info(s"Write dataframe to storage [$path]")
logger.info(s"Write dataframe to storage [$path] using trigger [$trigger]")

val queryName = createQueryName()

Expand All @@ -57,9 +56,12 @@ case class StorageOutput(
streamWriter = streamWriter.partitionBy(partitioningColumns: _*)
}

val streamingQuery = streamWriter
.trigger(processingTimeTrigger)
.start(path)
streamWriter = trigger match {
case Some(trigger) => streamWriter.trigger(trigger)
case _ => streamWriter
}

val streamingQuery = streamWriter.start(path)

streamingQuery.awaitTermination(timeout)
streamingQuery.stop()
Expand All @@ -73,6 +75,8 @@ case class StorageOutput(
private[streaming] def createQueryName(): String = {
val directory = Try { path.split(File.separatorChar).reverse.head }.toOption

logger.info(s"CreateQueryName based on $directory and $outputName.")

val queryName: String = (directory, outputName) match {
case (Some(directoryName), Some(name)) => s"QN_${name}_${directoryName}_${java.util.UUID.randomUUID}"
case (Some(directoryName), _) => s"QN_${directoryName}_${java.util.UUID.randomUUID}"
Expand All @@ -93,6 +97,7 @@ object StorageOutput {
* @return a new instance of StorageOutput.
*/
def apply(implicit config: Config): StorageOutput = {

val format = config.getString("Format")
val path = getPath

Expand All @@ -102,8 +107,7 @@ object StorageOutput {

val mode = config.getString("Mode")

val duration = Duration(config.getString("Duration"))
val processingTimeTrigger = Trigger.ProcessingTime(duration)
val trigger = getStreamingTrigger

val timeout = getTimeout

Expand All @@ -113,7 +117,7 @@ object StorageOutput {
format,
path,
partitioningColumns,
processingTimeTrigger,
trigger,
timeout,
mode,
options,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ class ConfigNodeTest extends AnyWordSpec with Matchers {
result.name shouldBe "my-entity"
result.typeName shouldBe "com.Entity"
result.config.getInt("Field1") shouldBe 5
result.config.withoutPath("Field1").isEmpty shouldBe true
result.config.withoutPath("Field1").withoutPath("Name").isEmpty shouldBe true
}
}

Expand Down
Loading
Loading