-
Notifications
You must be signed in to change notification settings - Fork 155
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(DATA-2956_influx_lag_reporter) (#308)
* feat(DATA-2956_influx_lag_reporter) * from map to string type * influx_lag_reporter-fix * new case class for StreamingWriting * suuport lag units * using TimeUnit * unitTest + getMaxDfTime function * removed outside from Metric class * move MetricReporting to new file * logging
- Loading branch information
Showing
7 changed files
with
201 additions
and
20 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
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
46 changes: 46 additions & 0 deletions
46
src/main/scala/com/yotpo/metorikku/metric/MetricReporting.scala
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,46 @@ | ||
package com.yotpo.metorikku.metric | ||
|
||
import java.util.concurrent.TimeUnit | ||
|
||
import com.yotpo.metorikku.exceptions.MetorikkuWriteFailedException | ||
import com.yotpo.metorikku.instrumentation.InstrumentationProvider | ||
import org.apache.log4j.LogManager | ||
import org.apache.spark.sql.DataFrame | ||
|
||
class MetricReporting { | ||
val log = LogManager.getLogger(this.getClass) | ||
|
||
def getMaxDataframeTime(dataFrame: DataFrame, reportLagTimeColumn: Option[String], | ||
reportLagTimeColumnUnits:Option[String]): Long ={ | ||
reportLagTimeColumn match { | ||
case Some(timeColumn) => { | ||
dataFrame.cache() | ||
try { | ||
reportLagTimeColumnUnits match { | ||
case Some(units) => TimeUnit.valueOf(units) match { | ||
case TimeUnit.SECONDS => TimeUnit.SECONDS.toMillis(dataFrame.agg({timeColumn.toString -> "max"}).collect()(0).getLong(0)) | ||
case TimeUnit.MILLISECONDS => TimeUnit.MILLISECONDS.toMillis(dataFrame.agg({timeColumn.toString -> "max"}).collect()(0).getLong(0)) | ||
} | ||
case _=> dataFrame.agg({timeColumn.toString -> "max"}).collect()(0).getTimestamp(0).getTime() | ||
} | ||
} catch { | ||
case e: ClassCastException => throw new ClassCastException(s"Lag instrumentation column -${timeColumn} " + | ||
s"cannot be cast to spark.sql.Timestamp or spark.sql.Long") | ||
case e: IllegalArgumentException => throw new MetorikkuWriteFailedException( | ||
s"${reportLagTimeColumnUnits} is not a legal argument for units, use one of the following: [SECONDS,MILLISECONDS]") | ||
} | ||
} | ||
case _=> throw MetorikkuWriteFailedException("Failed to report lag time, reportLagTimeColumn is not defined") | ||
} | ||
} | ||
|
||
def reportLagTime(dataFrame: DataFrame, reportLagTimeColumn: Option[String], | ||
reportLagTimeColumnUnits:Option[String], | ||
instrumentationProvider: InstrumentationProvider) : Unit ={ | ||
val maxDataframeTime = getMaxDataframeTime(dataFrame, reportLagTimeColumn, reportLagTimeColumnUnits) | ||
log.info(s"Max column ${reportLagTimeColumn} value is ${maxDataframeTime} for ${dataFrame}") | ||
val lag = System.currentTimeMillis - maxDataframeTime | ||
log.info(s"Reporting lag value: ${lag} for ${dataFrame}") | ||
instrumentationProvider.gauge(name = "lag", lag) | ||
} | ||
} |
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,21 @@ | ||
"userId","movieId","rating","created_at","updated_at" | ||
1,31,2.5,1260759144,"2016-05-05T22:37:36.000" | ||
1,1029,3,1260759179,"2016-05-05T22:43:36.000" | ||
1,1061,3,1260759182,"2016-05-05T22:38:36.000" | ||
1,1129,2,1260759185,"2016-05-05T22:39:36.000" | ||
1,1172,4,1260759205,"2016-05-05T22:30:36.000" | ||
1,1263,2,1260759151,"2016-05-05T22:31:36.000" | ||
1,1287,2,1260759187,"2016-05-05T22:32:36.000" | ||
1,1293,2,1260759148,"2016-05-05T22:33:36.000" | ||
1,1339,3.5,1260759125,"2016-05-05T22:34:36.000" | ||
1,1343,2,1260759131,"2016-05-05T22:35:36.000" | ||
1,1371,2.5,1260759135,"2016-05-05T22:36:36.000" | ||
1,1405,1,1260759203,"2016-05-05T22:37:36.000" | ||
1,1953,4,1260759191,"2016-05-05T20:37:36.000" | ||
1,2105,4,1260759139,"2016-05-05T20:37:33.000" | ||
1,2150,3,1260759194,"2016-05-05T22:37:36.000" | ||
1,2193,2,1260759198,"2016-05-05T21:30:36.000" | ||
1,2294,2,1260759108,"2016-05-05T20:37:36.000" | ||
1,2455,2.5,1260759113,"2016-05-05T20:27:36.000" | ||
1,2968,1,1260759200,"2016-05-05T20:25:36.000" | ||
1,3671,3,1260759117,"2016-05-05T20:15:36.000" |
87 changes: 87 additions & 0 deletions
87
src/test/scala/com/yotpo/metorikku/metric/test/MetricReporterTester.scala
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,87 @@ | ||
package com.yotpo.metorikku.metric.test | ||
import com.yotpo.metorikku.metric.{Metric, MetricReporting} | ||
import org.apache.log4j.{Level, LogManager, Logger} | ||
import org.apache.spark.sql.{DataFrame, SQLContext, SparkSession} | ||
import org.scalatest.{FunSuite, _} | ||
import org.apache.spark.sql.types._ | ||
|
||
//noinspection ScalaStyle | ||
class MetricReporterTester extends FunSuite with BeforeAndAfterEach { | ||
private val log: Logger = LogManager.getLogger(this.getClass) | ||
private var sparkSession : SparkSession = _ | ||
Logger.getLogger("org").setLevel(Level.WARN) | ||
|
||
override def beforeEach() { | ||
sparkSession = SparkSession.builder().appName("udf tests") | ||
.master("local") | ||
.config("", "") | ||
.getOrCreate() | ||
} | ||
|
||
test("Test getMaxDataframeTime") { | ||
|
||
val schema = StructType(Array( | ||
StructField("userId", IntegerType, true), | ||
StructField("movieId", IntegerType, true), | ||
StructField("rating", DoubleType, true), | ||
StructField("created_at", LongType, true), | ||
StructField("updated_at", TimestampType, true))) | ||
|
||
val sparkSession = SparkSession.builder.appName("test").getOrCreate() | ||
val sqlContext= new SQLContext(sparkSession.sparkContext) | ||
val df = sparkSession.read.format("csv").option("header", "true").schema(schema).load("src/test/configurations/mocks/ratings_time.csv") | ||
val metricReport = new MetricReporting() | ||
val maxUpdatedAt = metricReport.getMaxDataframeTime(df, Option("updated_at"), None) | ||
val maxCreatedAt = metricReport.getMaxDataframeTime(df, Option("created_at"), Option("SECONDS")) | ||
|
||
df.cache | ||
assert(maxUpdatedAt == 1462488216000L) | ||
assert(maxCreatedAt == 1260759205000L) | ||
} | ||
|
||
test("Test getMaxDataframeTime FAILs with invalid reportLagTimeColumn Units specified") { | ||
|
||
val schema = StructType(Array( | ||
StructField("userId", IntegerType, true), | ||
StructField("movieId", IntegerType, true), | ||
StructField("rating", DoubleType, true), | ||
StructField("created_at", LongType, true), | ||
StructField("updated_at", TimestampType, true))) | ||
|
||
val sparkSession = SparkSession.builder.appName("test").getOrCreate() | ||
val sqlContext= new SQLContext(sparkSession.sparkContext) | ||
|
||
val df = sparkSession.read.format("csv").option("header", "true").schema(schema).load("src/test/configurations/mocks/ratings_time.csv") | ||
val metricReport = new MetricReporting() | ||
|
||
val thrown = intercept[Exception] { | ||
metricReport.getMaxDataframeTime(df, Option("created_at"), Option("HOUR")) | ||
} | ||
assert(thrown.getMessage.startsWith("Some(HOUR) is not a legal argument for units, use one of the following: [SECONDS,MILLISECONDS]")) | ||
} | ||
|
||
test("Test getMaxDataframeTime FAILs reportLagTimeColumn is not defined") { | ||
|
||
val schema = StructType(Array( | ||
StructField("userId", IntegerType, true), | ||
StructField("movieId", IntegerType, true), | ||
StructField("rating", DoubleType, true), | ||
StructField("created_at", LongType, true), | ||
StructField("updated_at", TimestampType, true))) | ||
|
||
val sparkSession = SparkSession.builder.appName("test").getOrCreate() | ||
val sqlContext= new SQLContext(sparkSession.sparkContext) | ||
|
||
val df = sparkSession.read.format("csv").option("header", "true").schema(schema).load("src/test/configurations/mocks/ratings_time.csv") | ||
val metricReport = new MetricReporting() | ||
|
||
val thrown = intercept[Exception] { | ||
metricReport.getMaxDataframeTime(df, None, None) | ||
} | ||
assert(thrown.getMessage.startsWith("Failed to report lag time, reportLagTimeColumn is not defined")) | ||
} | ||
|
||
override def afterEach() { | ||
sparkSession.stop() | ||
} | ||
} |