From 50ea2225df2c514b71259e1bd362c832e4d4778f Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Wed, 16 Sep 2020 12:09:37 +0800 Subject: [PATCH 01/15] test scala spark Signed-off-by: Oleksii Moskalenko --- ingestion-spark/pom.xml | 196 ++++++++++++++++++ .../feast/ingestion/StreamIngestionApp.scala | 80 +++++++ pom.xml | 1 + 3 files changed, 277 insertions(+) create mode 100644 ingestion-spark/pom.xml create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala diff --git a/ingestion-spark/pom.xml b/ingestion-spark/pom.xml new file mode 100644 index 0000000000..8941cc0b17 --- /dev/null +++ b/ingestion-spark/pom.xml @@ -0,0 +1,196 @@ + + + + 4.0.0 + + + dev.feast + feast-parent + ${revision} + + + Feast Spark Ingestion + feast-ingestion-spark + 0.7-SNAPSHOT + + + + 2.12 + ${scala.version}.12 + 3.0.1 + 4.4.0 + 3.3.0 + 0.7-SNAPSHOT + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + org.scala-lang + scala-library + ${scala.fullVersion} + + + + org.apache.spark + spark-core_${scala.version} + ${spark.version} + + + + org.apache.spark + spark-streaming_${scala.version} + ${spark.version} + + + + org.apache.spark + spark-sql_${scala.version} + ${spark.version} + + + + org.apache.spark + spark-sql-kafka-0-10_${scala.version} + ${spark.version} + + + + + com.github.scopt + scopt_${scala.version} + 3.7.1 + + + + + + + + src/main/scala + src/test/scala + + + net.alchim31.maven + scala-maven-plugin + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.fullVersion} + + + + org.apache.maven.plugins + maven-assembly-plugin + + + jar-with-dependencies + + + + + make-assembly + package + + single + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + compile + + compile + + + + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + org.apache.maven.plugins + maven-assembly-plugin + ${maven-assembly-plugin.version} + + + + + diff --git a/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala b/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala new file mode 100644 index 0000000000..640e7b20a8 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala @@ -0,0 +1,80 @@ +package feast.ingestion + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + + +class StreamIngestionApp { + def createSparkSession[T](contextName: String, + sqlJoinPartitions: Int = 10 + ): SparkSession = { + val conf = new SparkConf() + conf + .setAppName(contextName) + .set("spark.sql.shuffle.partitions", sqlJoinPartitions.toString) + + val sparkSession = SparkSession + .builder() + .config("spark.sql.warehouse.dir", "file:///tmp/spark-warehouse") + .config(conf) + .getOrCreate() + + sparkSession + } + + case class Config( + sourceBootstrapServers: String = "", + sourceTopic: String = "", + specsBootstrapServers: String = "", + specsTopic: String = "" + ) + + val parser = new scopt.OptionParser[Config]("scopt") { + head("scopt", "3.x") + + opt[String]("source-broker") + .action((x, c) => c.copy(sourceBootstrapServers = x)) + + opt[String]("source-topic") + .action((x, c) => c.copy(sourceTopic = x)) + + opt[String]("specs-broker") + .action((x, c) => c.copy(specsBootstrapServers = x)) + + opt[String]("specs-broker") + .action((x, c) => c.copy(specsTopic = x)) + } + + def start(args: Array[String]) { + val config = parser.parse(args, Config()).get + val spark = createSparkSession("TestApp") + + import spark.implicits._ + + val rows = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", config.sourceBootstrapServers) + .option("subscribe", config.sourceTopic) + .load() + .selectExpr("CAST(key as STRING)", "value") + .as[(String, Array[Byte])] + + val specs = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", config.specsBootstrapServers) + .option("subscribe", config.specsTopic) + .load() + .selectExpr("CASE(key as STRING)", "value") + .as[(String, Array[Byte])] + + rows.printSchema() + specs.printSchema() + } +} + +object StreamIngestionApp extends App { + val app = new StreamIngestionApp + app.start(args) +} diff --git a/pom.xml b/pom.xml index fd3faf5622..61dff77a37 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,7 @@ common job-controller common-test + ingestion-spark From b1b80874b519b2d7f9c4d2b347132bb4a7b5690f Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 28 Sep 2020 16:16:09 +0800 Subject: [PATCH 02/15] offline batch ingestion in spark Signed-off-by: Oleksii Moskalenko --- ingestion-spark/pom.xml | 126 +++++++++++++---- .../scala/feast/ingestion/BasePipeline.scala | 36 +++++ .../scala/feast/ingestion/FeatureTable.scala | 23 ++++ .../scala/feast/ingestion/IngestionJob.scala | 54 ++++++++ .../feast/ingestion/IngestionJobConfig.scala | 26 ++++ .../feast/ingestion/OfflinePipeline.scala | 47 +++++++ .../feast/ingestion/StreamIngestionApp.scala | 80 ----------- .../ingestion/sources/bq/BigQueryReader.scala | 19 +++ .../ingestion/sources/file/FileReader.scala | 18 +++ .../stores/redis/DefaultSource.scala | 21 +++ .../stores/redis/HashTypePersistence.scala | 55 ++++++++ .../stores/redis/RedisSinkRelation.scala | 86 ++++++++++++ .../stores/redis/SparkRedisConfig.scala | 27 ++++ .../ingestion/utils/TypeConversion.scala | 32 +++++ .../source/RedisSinkMetricSource.scala | 11 ++ .../feast/ingestion/OfflinePipelineSpec.scala | 128 ++++++++++++++++++ .../test/scala/feast/ingestion/UnitSpec.scala | 8 ++ 17 files changed, 691 insertions(+), 106 deletions(-) create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala delete mode 100644 ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala create mode 100644 ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala create mode 100644 ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala create mode 100644 ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala diff --git a/ingestion-spark/pom.xml b/ingestion-spark/pom.xml index 8941cc0b17..b786660417 100644 --- a/ingestion-spark/pom.xml +++ b/ingestion-spark/pom.xml @@ -29,40 +29,23 @@ Feast Spark Ingestion feast-ingestion-spark 0.7-SNAPSHOT - 2.12 ${scala.version}.12 - 3.0.1 + 2.4.7 4.4.0 3.3.0 0.7-SNAPSHOT - - - - - - - - - - - - - - - - - - - - - + + dev.feast + datatypes-java + ${project.version} + @@ -103,12 +86,24 @@ - org.apache.spark - spark-sql-kafka-0-10_${scala.version} - ${spark.version} + org.codehaus.janino + janino + 3.0.16 + + + + + + + + + + + + com.github.scopt scopt_${scala.version} @@ -116,6 +111,67 @@ + + com.thesamet.scalapb + sparksql-scalapb_${scala.version} + 0.10.4 + + + + com.google.cloud.spark + spark-bigquery_${scala.version} + 0.17.2 + + + + com.google.cloud + google-cloud-bigquery + + + + joda-time + joda-time + 2.10.6 + + + + com.redislabs + spark-redis_${scala.version} + 2.5.0 + + + + org.apache.arrow + arrow-vector + 0.16.0 + + + + io.netty + netty-all + 4.1.52.Final + + + + org.scalatest + scalatest_${scala.version} + 3.2.2 + test + + + + org.scalacheck + scalacheck_${scala.version} + 1.14.3 + test + + + + com.dimafeng + testcontainers-scala-scalatest_${scala.version} + 0.38.3 + + @@ -147,6 +203,24 @@ ${scala.fullVersion} + + org.scalatest + scalatest-maven-plugin + 2.0.0 + + ${project.build.directory}/surefire-reports + . + TestSuiteReport.txt + + + + integration-test + + integration-test + + + + org.apache.maven.plugins maven-assembly-plugin diff --git a/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala new file mode 100644 index 0000000000..64fd556592 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala @@ -0,0 +1,36 @@ +package feast.ingestion + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +trait BasePipeline { + def createSparkSession(jobConfig: IngestionJobConfig): SparkSession = { + // workaround for issue with arrow & netty + // see https://github.com/apache/arrow/tree/master/java#java-properties + System.setProperty("io.netty.tryReflectionSetAccessible", "true") + + val conf = new SparkConf() + conf + .setAppName(s"${jobConfig.mode} IngestionJob for ${jobConfig.featureTable.name}") + .setMaster("local") + + jobConfig.metrics match { + case Some(c: StatsDConfig) => + conf + .set("spark.metrics.conf.*.source.redis.class", "org.apache.spark.metrics.source.RedisSinkMetricSource") + .set("spark.metrics.conf.*.sink.statsd.class", "org.apache.spark.metrics.sink.StatsdSink") + .set("spark.metrics.conf.*.sink.statsd.host", c.host) + .set("spark.metrics.conf.*.sink.statsd.port", c.port.toString) + .set("spark.metrics.conf.*.sink.statsd.period", "1") + .set("spark.metrics.conf.*.sink.statsd.unit", "seconds") + } + + SparkSession + .builder() + .config(conf) + .getOrCreate() + } + + + def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala new file mode 100644 index 0000000000..5a427c0c8f --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala @@ -0,0 +1,23 @@ +package feast.ingestion + +abstract class Source { + def mapping: Map[String, String] + def timestampColumn: String +} + +abstract class OfflineSource extends Source +abstract class OnlineSource extends Source + +case class GSSource(path: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource +case class BQSource(project: String, dataset: String, table: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource + + +case class KafkaSource(bootstrapServers: String, topic: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OnlineSource + +case class Field(name: String, `type`: feast.proto.types.ValueProto.ValueType.Enum) + +case class FeatureTable(name: String, + entities: Seq[Field], + features: Seq[Field], + offline_source: Option[OfflineSource] = None, + online_source: Option[OnlineSource] = None) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala new file mode 100644 index 0000000000..9bd54124d3 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala @@ -0,0 +1,54 @@ +package feast.ingestion + +import feast.proto.types.ValueProto.ValueType +import org.joda.time.DateTime + +object IngestionJob { + import Modes._ + implicit val modesRead: scopt.Read[Modes.Value] = scopt.Read.reads(Modes withName _.capitalize) + + val parser = new scopt.OptionParser[IngestionJobConfig]("IngestionJon") { + head("feast.ingestion.IngestionJob", "0.8") + + opt[Modes]("mode") + .action((x, c) => c.copy(mode = x)) + .required() + .text("Mode to operate ingestion job (offline or online)") + + opt[String](name="feature-table-spec") + .action((x, c) => c.copy(featureTable = FeatureTable( + name="jaeger-car", + entities = Seq(Field(name="customer", `type` = ValueType.Enum.STRING)), + features = Seq( + Field("car_customer_id_avg_customer_distance_cancelled", ValueType.Enum.FLOAT), + Field("car_customer_id_num_completed", ValueType.Enum.FLOAT), + Field("car_customer_id_origin_completed_1", ValueType.Enum.INT32) + ), + offline_source = Some(BQSource("gods-staging", "feast", "default_jaeger_car_customer", Map.empty, "event_timestamp")) + ))) + .required() + .text("JSON-encoded FeatureTableSpec object") + + opt[String](name="start") + .action((x, c) => c.copy(startTime = DateTime.parse(x))) + .text("Start timestamp for offline ingestion") + + opt[String](name="end") + .action((x, c) => c.copy(endTime = DateTime.parse(x))) + .text("End timestamp for offline ingestion") + } + + def main(args: Array[String]): Unit = { + parser.parse(args, IngestionJobConfig()) match { + case Some(config) => + config.mode match { + case Modes.Offline => + val sparkSession = OfflinePipeline.createSparkSession(config) + OfflinePipeline.createPipeline(sparkSession, config) + } + case None => + println("Parameters can't be parsed") + } + } + +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala new file mode 100644 index 0000000000..b395146c7a --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala @@ -0,0 +1,26 @@ +package feast.ingestion + +import feast.ingestion.Modes.Modes +import org.joda.time.DateTime + +object Modes extends Enumeration { + type Modes = Value + val Offline, Online = Value +} + +abstract class StoreConfig + +case class RedisConfig(connection: String) extends StoreConfig + +abstract class MetricConfig + +case class StatsDConfig(host: String, port: Int) extends MetricConfig + +case class IngestionJobConfig( + mode: Modes = Modes.Offline, + featureTable: FeatureTable = null, + startTime: DateTime = DateTime.now(), + endTime: DateTime = DateTime.now(), + store: StoreConfig = RedisConfig("localhost:6379"), + metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)) + ) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala new file mode 100644 index 0000000000..d46f8f4768 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala @@ -0,0 +1,47 @@ +package feast.ingestion + +import feast.ingestion.sources.bq.BigQueryReader +import feast.ingestion.sources.file.FileReader +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.functions.col + +object OfflinePipeline extends BasePipeline { + override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { + val input = config.featureTable.offline_source match { + case Some(source:BQSource) => + BigQueryReader.createBatchSource( + sparkSession.sqlContext, + source, + config.startTime, + config.endTime + ) + case Some(source:GSSource) => + FileReader.createBatchSource( + sparkSession.sqlContext, source, config.startTime, config.endTime + ) + } + + val projection = inputProjection(config.featureTable.offline_source.get, config.featureTable.features, config.featureTable.entities) + input.select(projection:_*) + .write + .format("feast.ingestion.stores.redis") + .option("entity_columns", config.featureTable.entities.map(_.name).mkString(",")) + .option("entity_names", config.featureTable.entities.map(_.name).mkString(",")) + .option("namespace", config.featureTable.name) + .option("timestamp_column", config.featureTable.offline_source.get.timestampColumn) + .save() + } + + private def inputProjection(source: OfflineSource, features: Seq[Field], entities: Seq[Field]): Array[Column] = { + val mapping = ( + if (source.mapping.nonEmpty) source.mapping + else features.map(f => (f.name, f.name)) + ) ++ Seq( + (source.timestampColumn, source.timestampColumn) + ) ++ entities.map(e => (e.name, e.name)) + + mapping.map { + case (alias, source) => col(source).alias(alias) + }.toArray + } +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala b/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala deleted file mode 100644 index 640e7b20a8..0000000000 --- a/ingestion-spark/src/main/scala/feast/ingestion/StreamIngestionApp.scala +++ /dev/null @@ -1,80 +0,0 @@ -package feast.ingestion - -import org.apache.spark.SparkConf -import org.apache.spark.sql.SparkSession - - -class StreamIngestionApp { - def createSparkSession[T](contextName: String, - sqlJoinPartitions: Int = 10 - ): SparkSession = { - val conf = new SparkConf() - conf - .setAppName(contextName) - .set("spark.sql.shuffle.partitions", sqlJoinPartitions.toString) - - val sparkSession = SparkSession - .builder() - .config("spark.sql.warehouse.dir", "file:///tmp/spark-warehouse") - .config(conf) - .getOrCreate() - - sparkSession - } - - case class Config( - sourceBootstrapServers: String = "", - sourceTopic: String = "", - specsBootstrapServers: String = "", - specsTopic: String = "" - ) - - val parser = new scopt.OptionParser[Config]("scopt") { - head("scopt", "3.x") - - opt[String]("source-broker") - .action((x, c) => c.copy(sourceBootstrapServers = x)) - - opt[String]("source-topic") - .action((x, c) => c.copy(sourceTopic = x)) - - opt[String]("specs-broker") - .action((x, c) => c.copy(specsBootstrapServers = x)) - - opt[String]("specs-broker") - .action((x, c) => c.copy(specsTopic = x)) - } - - def start(args: Array[String]) { - val config = parser.parse(args, Config()).get - val spark = createSparkSession("TestApp") - - import spark.implicits._ - - val rows = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", config.sourceBootstrapServers) - .option("subscribe", config.sourceTopic) - .load() - .selectExpr("CAST(key as STRING)", "value") - .as[(String, Array[Byte])] - - val specs = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", config.specsBootstrapServers) - .option("subscribe", config.specsTopic) - .load() - .selectExpr("CASE(key as STRING)", "value") - .as[(String, Array[Byte])] - - rows.printSchema() - specs.printSchema() - } -} - -object StreamIngestionApp extends App { - val app = new StreamIngestionApp - app.start(args) -} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala b/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala new file mode 100644 index 0000000000..f540a168cc --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala @@ -0,0 +1,19 @@ +package feast.ingestion.sources.bq + +import java.sql.Timestamp + +import feast.ingestion.BQSource +import org.joda.time.DateTime +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.functions.col + +object BigQueryReader { + def createBatchSource(sqlContext: SQLContext, source: BQSource, + start: DateTime, end: DateTime): DataFrame = { + sqlContext.read + .format("bigquery") + .load(s"${source.project}.${source.dataset}.${source.table}") + .filter(col(source.timestampColumn) >= new Timestamp(start.getMillis)) + .filter(col(source.timestampColumn) < new Timestamp(end.getMillis)) + } +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala b/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala new file mode 100644 index 0000000000..8c4500437a --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala @@ -0,0 +1,18 @@ +package feast.ingestion.sources.file + +import java.sql.Timestamp + +import feast.ingestion.GSSource +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.joda.time.DateTime + +object FileReader { + def createBatchSource(sqlContext: SQLContext, source: GSSource, + start: DateTime, end: DateTime): DataFrame = { + sqlContext.read + .parquet(source.path) + .filter(col(source.timestampColumn) >= new Timestamp(start.getMillis)) + .filter(col(source.timestampColumn) < new Timestamp(end.getMillis)) + } +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala new file mode 100644 index 0000000000..d02358c080 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala @@ -0,0 +1,21 @@ +package feast.ingestion.stores.redis + +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} + +class RedisRelationProvider extends RelationProvider with CreatableRelationProvider { + + override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = ??? + + override def createRelation(sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { + val config = SparkRedisConfig.parse(parameters) + val relation = new RedisSinkRelation(sqlContext, config) + + relation.insert(data, overwrite = false) + + relation + } +} + + +class DefaultSource extends RedisRelationProvider \ No newline at end of file diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala new file mode 100644 index 0000000000..6f07dd72d7 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala @@ -0,0 +1,55 @@ +package feast.ingestion.stores.redis + + +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ +import redis.clients.jedis.{Pipeline, Response} + +import scala.jdk.CollectionConverters._ +import com.google.protobuf.Timestamp +import feast.ingestion.utils.TypeConversion +import feast.proto.types.ValueProto + + +class HashTypePersistence(config: SparkRedisConfig) extends Serializable { + def encodeRow(keyColumns: Array[String], timestampField: String, value: Row): Map[Array[Byte], Array[Byte]] = { + val fields = value.schema.fields.map(_.name) + val types = value.schema.fields.map(f => (f.name, f.dataType)).toMap + val kvMap = value.getValuesMap[Any](fields) + + val values = kvMap + .filter { case (_, v) => + // don't store null values + v != null + } + .filter { case (k, _) => + // don't store entities & timestamp + !keyColumns.contains(k) && k != config.timestampColumn + } + .map { case (k, v) => + k.getBytes -> encodeValue(v, types(k)) + } + + val timestamp = Seq(( + timestampField.getBytes, + encodeValue(value.getAs[Timestamp](config.timestampColumn), TimestampType))) + + values ++ timestamp + } + + def encodeValue(value: Any, `type`: DataType): Array[Byte] = { + TypeConversion.sqlTypeToProtoValue(value, `type`).toByteArray + } + + def save(pipeline: Pipeline, key: String, value: Map[Array[Byte], Array[Byte]], ttl: Int): Unit = { + pipeline.hset(key.getBytes, value.asJava) + if (ttl > 0) { + pipeline.expire(key, ttl) + } + } + + def getTimestamp(pipeline: Pipeline, key: String, timestampField: String): Response[Array[Byte]] = { + pipeline.hget(key.getBytes(), timestampField.getBytes) + } + +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala new file mode 100644 index 0000000000..a8072e552f --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -0,0 +1,86 @@ +package feast.ingestion.stores.redis + +import com.google.protobuf.Timestamp +import com.redislabs.provider.redis.{ReadWriteConfig, RedisConfig, RedisEndpoint} +import com.redislabs.provider.redis.rdd.Keys.groupKeysByNode +import com.redislabs.provider.redis.util.PipelineUtils.{foreachWithPipeline, mapWithPipeline} +import org.apache.spark.SparkEnv +import org.apache.spark.metrics.source.RedisSinkMetricSource +import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + + +class RedisSinkRelation(override val sqlContext: SQLContext, + config: SparkRedisConfig) extends BaseRelation + with InsertableRelation with Serializable +{ + private implicit val redisConfig: RedisConfig = { + new RedisConfig( + new RedisEndpoint(sqlContext.sparkContext.getConf) + ) + } + + private implicit val readWriteConfig: ReadWriteConfig = { + ReadWriteConfig.fromSparkConf(sqlContext.sparkContext.getConf) + } + + override def schema: StructType = ??? + + val persistence = new HashTypePersistence(config) + + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + data.foreachPartition { partition: Iterator[Row] => + // grouped iterator to only allocate memory for a portion of rows + partition.grouped(config.iteratorGroupingSize).foreach { batch => + val rowsWithKey: Map[String, Row] = batch.map(row => dataKeyId(row) -> row).toMap + + groupKeysByNode(redisConfig.hosts, rowsWithKey.keysIterator).foreach { case (node, keys) => + val conn = node.connect() + val timestamps = mapWithPipeline(conn, keys) { (pipeline, key) => + persistence.getTimestamp(pipeline, key, timestampField) + } + + val timestampByKey = timestamps + .map(_.asInstanceOf[Array[Byte]]) + .map(Option(_).map(Timestamp.parseFrom).map(t => new java.sql.Timestamp(t.getSeconds * 1000))) + .zip(rowsWithKey.keys) + .map(_.swap) + .toMap + + foreachWithPipeline(conn, keys) { (pipeline, key) => + val row = rowsWithKey(key) + + timestampByKey(key) match { + case Some(t) if !t.before(row.getAs[java.sql.Timestamp](config.timestampColumn)) => () + case _ => + if (metricSource.nonEmpty) { + metricSource.get.METRIC_TOTAL_ROWS_INSERTED.inc() + } + + val encodedRow = persistence.encodeRow(config.entityColumns, timestampField, row) + persistence.save(pipeline, key, encodedRow, ttl=0) + } + } + conn.close() + } + } + } + } + + private def dataKeyId(row: Row): String = { + val entityKey = config.entityColumns.map(row.getAs[Any]).map(_.toString).mkString(":") + val entityPrefix = config.entityColumns.sorted.mkString("_") + s"${entityPrefix}:$entityKey" + } + + private def timestampField: String = { + s"${config.timestampPrefix}:${config.namespace}" + } + + private lazy val metricSource: Option[RedisSinkMetricSource] = + SparkEnv.get.metricsSystem.getSourcesByName("redis_sink") match { + case Seq(head) => Some(head.asInstanceOf[RedisSinkMetricSource]) + case _ => None + } +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala new file mode 100644 index 0000000000..518690485f --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala @@ -0,0 +1,27 @@ +package feast.ingestion.stores.redis + +import redis.clients.jedis.Protocol + +case class SparkRedisConfig( + val namespace: String, + val entityNames: Array[String], + val entityColumns: Array[String], + val timestampColumn: String, + val iteratorGroupingSize: Int = 1000, + val timestampPrefix: String = "_ts" + ) + +object SparkRedisConfig { + val NAMESPACE = "namespace" + val ENTITY_NAMES = "entity_names" + val ENTITY_COLUMNS = "entity_columns" + val TS_COLUMN = "timestamp_column" + + def parse(parameters: Map[String, String]): SparkRedisConfig = + SparkRedisConfig( + namespace = parameters.getOrElse(NAMESPACE, ""), + entityNames = parameters.getOrElse(ENTITY_NAMES, "").split(","), + entityColumns = parameters.getOrElse(ENTITY_COLUMNS, "").split(","), + timestampColumn = parameters.getOrElse(TS_COLUMN, "event_timestamp") + ) +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala new file mode 100644 index 0000000000..1bd81ac5c7 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala @@ -0,0 +1,32 @@ +package feast.ingestion.utils + +import com.google.protobuf.{Message, Timestamp} +import feast.proto.types.ValueProto +import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, IntegerType, LongType, StringType, TimestampType} + +object TypeConversion { + def sqlTypeToProtoValue(value: Any, `type`: DataType): Message = { + (`type` match { + case IntegerType => ValueProto.Value.newBuilder().setInt32Val(value.asInstanceOf[Int]) + case LongType => ValueProto.Value.newBuilder().setInt64Val(value.asInstanceOf[Long]) + case StringType => ValueProto.Value.newBuilder().setStringVal(value.asInstanceOf[String]) + case DoubleType => ValueProto.Value.newBuilder().setDoubleVal(value.asInstanceOf[Double]) + case FloatType => ValueProto.Value.newBuilder().setFloatVal(value.asInstanceOf[Float]) + case TimestampType => Timestamp.newBuilder() + .setSeconds(value.asInstanceOf[java.sql.Timestamp].getTime / 1000) + }).build + } + + class AsScala[A](op: => A) { + def asScala: A = op + } + + implicit def protoValueAsScala(v: ValueProto.Value): AsScala[Any] = new AsScala[Any]( + v.getValCase match { + case ValueProto.Value.ValCase.INT32_VAL => v.getInt32Val + case ValueProto.Value.ValCase.FLOAT_VAL => v.getFloatVal + case ValueProto.Value.ValCase.VAL_NOT_SET => throw new RuntimeException(s"$v not a ValueProto") + } + ) + +} diff --git a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala new file mode 100644 index 0000000000..123c79038b --- /dev/null +++ b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala @@ -0,0 +1,11 @@ +package org.apache.spark.metrics.source + +import com.codahale.metrics.MetricRegistry + +class RedisSinkMetricSource extends Source { + override val sourceName: String = "redis_sink" + + override val metricRegistry: MetricRegistry = new MetricRegistry + + val METRIC_TOTAL_ROWS_INSERTED = metricRegistry.counter(MetricRegistry.name("rowsInserted")) +} diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala new file mode 100644 index 0000000000..089d1fcf18 --- /dev/null +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala @@ -0,0 +1,128 @@ +package feast.ingestion + +import java.nio.file.Files + +import com.dimafeng.testcontainers.{ForAllTestContainer, GenericContainer} +import com.google.protobuf.Timestamp +import feast.ingestion.utils.TypeConversion.protoValueAsScala +import feast.proto.types.ValueProto +import feast.proto.types.ValueProto.ValueType +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions._ +import org.joda.time.DateTime +import org.scalacheck._ +import org.scalatest._ +import matchers.should.Matchers._ +import org.scalatest.matchers.Matcher +import redis.clients.jedis.Jedis + +import scala.jdk.CollectionConverters.mapAsScalaMapConverter +import scala.reflect.runtime.universe.TypeTag + +case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) + + +class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { + + override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) + + trait SparkContext { + val sparkConf = new SparkConf() + .setMaster("local") + .setAppName("Testing") + .set("spark.redis.host", container.host) + .set("spark.redis.port", container.mappedPort(6379).toString) + + val sparkSession = SparkSession + .builder() + .config(sparkConf) + .getOrCreate() + } + + trait DataGenerator { + self: SparkContext => + + def storeFiles[T <: Product : TypeTag](rows: Seq[T]): String = { + import self.sparkSession.implicits._ + + val tempPath = Files.createTempDirectory("test-dir").toFile.toString + "/rows" + + sparkSession.createDataset(rows) + .withColumn("date", to_date($"eventTimestamp")) + .write + .partitionBy("eventTimestamp") + .save(tempPath) + + tempPath + } + } + + trait Scope { + val rowGenerator = for { + customer <- Gen.asciiPrintableStr + feature1 <- Gen.choose(0, 100) + feature2 <- Gen.choose(0, 1) + eventTimestamp <- Gen.choose(0, 30 * 24 * 3600).map(DateTime.now().minusSeconds(_)) + } + yield Row(customer, feature1, feature2, new java.sql.Timestamp(eventTimestamp.getMillis)) + + val config = IngestionJobConfig( + featureTable = FeatureTable( + name = "test-fs", + entities = Seq(Field("customer", ValueType.Enum.STRING)), + features = Seq( + Field("feature1", ValueType.Enum.INT32), + Field("feature2", ValueType.Enum.FLOAT), + ), + //offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp")) + ), + startTime = DateTime.now().minusDays(30), + endTime = DateTime.now() + ) + + def beStoredRow(row: Row) = { + val m: Matcher[Map[String, Any]] = contain.allElementsOf(Seq( + "feature1" -> row.feature1, + "feature2" -> row.feature2, + "_ts:test-fs" -> Timestamp.newBuilder().setSeconds(row.eventTimestamp.getTime / 1000).build() + )).matcher + + m compose { + (_: Map[Array[Byte], Array[Byte]]) + .map(e => ( + new String(e._1), + if (new String(e._1).startsWith("_ts")) + Timestamp.parseFrom(e._2) else + ValueProto.Value.parseFrom(e._2).asScala + )) + } + } + } + + + "Parquet source file" should "be ingested in redis" in new Scope with SparkContext with DataGenerator { + val rows = Gen.listOfN(100, rowGenerator).sample.get + val tempPath = storeFiles(rows) + val configWithOfflineSource = config.copy(featureTable = config.featureTable.copy(offline_source = + Some(GSSource(tempPath, Map.empty, "eventTimestamp")))) + + OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + + val jedis = new Jedis("localhost", container.mappedPort(6379)) + rows.foreach( + r => { + val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap + storedValues should beStoredRow(r) + // val m = contain allElementsOf(Seq( + // "feature1" -> r.feature1, + // "feature2" -> r.feature2, + // "_ts:test-fs" -> r.eventTimestamp + // )) + // + // storedValues should m + } + ) + + } +} diff --git a/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala new file mode 100644 index 0000000000..3fa5320562 --- /dev/null +++ b/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala @@ -0,0 +1,8 @@ +package feast.ingestion + +import org.scalatest._ +import matchers._ +import org.scalatest.flatspec.AnyFlatSpec + +abstract class UnitSpec extends AnyFlatSpec with should.Matchers with + OptionValues with Inside with Inspectors From 6b48e33f0c63036d4e294fe965f0502b14ff4e6a Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 28 Sep 2020 16:35:58 +0800 Subject: [PATCH 03/15] clean up Signed-off-by: Oleksii Moskalenko --- .../feast/ingestion/OfflinePipeline.scala | 19 ++++++++++--------- .../ingestion/utils/TypeConversion.scala | 6 ++++++ .../feast/ingestion/OfflinePipelineSpec.scala | 15 ++++----------- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala index d46f8f4768..96f0cfa40a 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala @@ -8,21 +8,21 @@ import org.apache.spark.sql.functions.col object OfflinePipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { val input = config.featureTable.offline_source match { - case Some(source:BQSource) => + case Some(source: BQSource) => BigQueryReader.createBatchSource( sparkSession.sqlContext, source, config.startTime, config.endTime ) - case Some(source:GSSource) => + case Some(source: GSSource) => FileReader.createBatchSource( sparkSession.sqlContext, source, config.startTime, config.endTime ) } val projection = inputProjection(config.featureTable.offline_source.get, config.featureTable.features, config.featureTable.entities) - input.select(projection:_*) + input.select(projection: _*) .write .format("feast.ingestion.stores.redis") .option("entity_columns", config.featureTable.entities.map(_.name).mkString(",")) @@ -33,14 +33,15 @@ object OfflinePipeline extends BasePipeline { } private def inputProjection(source: OfflineSource, features: Seq[Field], entities: Seq[Field]): Array[Column] = { - val mapping = ( - if (source.mapping.nonEmpty) source.mapping + val mainColumns = + if (source.mapping.nonEmpty) + source.mapping else features.map(f => (f.name, f.name)) - ) ++ Seq( - (source.timestampColumn, source.timestampColumn) - ) ++ entities.map(e => (e.name, e.name)) - mapping.map { + val timestampColumn = Seq((source.timestampColumn, source.timestampColumn)) + val entitiesColumns = entities.map(e => (e.name, e.name)) + + (mainColumns ++ entitiesColumns ++ timestampColumn).map { case (alias, source) => col(source).alias(alias) }.toArray } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala index 1bd81ac5c7..e550496a01 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala @@ -1,5 +1,7 @@ package feast.ingestion.utils +import java.sql + import com.google.protobuf.{Message, Timestamp} import feast.proto.types.ValueProto import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, IntegerType, LongType, StringType, TimestampType} @@ -29,4 +31,8 @@ object TypeConversion { } ) + implicit def timestampAsScala(t: Timestamp): AsScala[java.sql.Timestamp] = new AsScala[java.sql.Timestamp]( + new sql.Timestamp(t.getSeconds * 1000) + ) + } diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala index 089d1fcf18..ac6dfc056c 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala @@ -4,7 +4,7 @@ import java.nio.file.Files import com.dimafeng.testcontainers.{ForAllTestContainer, GenericContainer} import com.google.protobuf.Timestamp -import feast.ingestion.utils.TypeConversion.protoValueAsScala +import feast.ingestion.utils.TypeConversion._ import feast.proto.types.ValueProto import feast.proto.types.ValueProto.ValueType import org.apache.spark.SparkConf @@ -63,7 +63,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { customer <- Gen.asciiPrintableStr feature1 <- Gen.choose(0, 100) feature2 <- Gen.choose(0, 1) - eventTimestamp <- Gen.choose(0, 30 * 24 * 3600).map(DateTime.now().minusSeconds(_)) + eventTimestamp <- Gen.choose(0, 30 * 24 * 3600).map(DateTime.now().withMillisOfSecond(0).minusSeconds(_)) } yield Row(customer, feature1, feature2, new java.sql.Timestamp(eventTimestamp.getMillis)) @@ -85,7 +85,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { val m: Matcher[Map[String, Any]] = contain.allElementsOf(Seq( "feature1" -> row.feature1, "feature2" -> row.feature2, - "_ts:test-fs" -> Timestamp.newBuilder().setSeconds(row.eventTimestamp.getTime / 1000).build() + "_ts:test-fs" -> row.eventTimestamp )).matcher m compose { @@ -93,7 +93,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { .map(e => ( new String(e._1), if (new String(e._1).startsWith("_ts")) - Timestamp.parseFrom(e._2) else + Timestamp.parseFrom(e._2).asScala else ValueProto.Value.parseFrom(e._2).asScala )) } @@ -114,13 +114,6 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { r => { val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap storedValues should beStoredRow(r) - // val m = contain allElementsOf(Seq( - // "feature1" -> r.feature1, - // "feature2" -> r.feature2, - // "_ts:test-fs" -> r.eventTimestamp - // )) - // - // storedValues should m } ) From c9028be3bb41225a03ae8177bc1eff1cfbe7163e Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 28 Sep 2020 19:25:09 +0800 Subject: [PATCH 04/15] deduplicate rows & use latest Signed-off-by: Oleksii Moskalenko --- .../stores/redis/RedisSinkRelation.scala | 23 +++- .../stores/redis/SparkRedisConfig.scala | 18 +-- .../feast/ingestion/OfflinePipelineSpec.scala | 108 ++++++++++++++---- 3 files changed, 112 insertions(+), 37 deletions(-) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index a8072e552f..07c130f538 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -8,16 +8,16 @@ import org.apache.spark.SparkEnv import org.apache.spark.metrics.source.RedisSinkMetricSource import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.functions.col import org.apache.spark.sql.{DataFrame, Row, SQLContext} class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisConfig) extends BaseRelation - with InsertableRelation with Serializable -{ + with InsertableRelation with Serializable { private implicit val redisConfig: RedisConfig = { new RedisConfig( - new RedisEndpoint(sqlContext.sparkContext.getConf) + new RedisEndpoint(sqlContext.sparkContext.getConf) ) } @@ -30,10 +30,15 @@ class RedisSinkRelation(override val sqlContext: SQLContext, val persistence = new HashTypePersistence(config) override def insert(data: DataFrame, overwrite: Boolean): Unit = { - data.foreachPartition { partition: Iterator[Row] => + val dataToStore = + if (config.repartitionByEntity) + data.repartition(config.entityColumns.map(col): _*) + else data + + dataToStore.foreachPartition { partition: Iterator[Row] => // grouped iterator to only allocate memory for a portion of rows partition.grouped(config.iteratorGroupingSize).foreach { batch => - val rowsWithKey: Map[String, Row] = batch.map(row => dataKeyId(row) -> row).toMap + val rowsWithKey: Map[String, Row] = compactRowsToLatest(batch.map(row => dataKeyId(row) -> row)).toMap groupKeysByNode(redisConfig.hosts, rowsWithKey.keysIterator).foreach { case (node, keys) => val conn = node.connect() @@ -59,7 +64,7 @@ class RedisSinkRelation(override val sqlContext: SQLContext, } val encodedRow = persistence.encodeRow(config.entityColumns, timestampField, row) - persistence.save(pipeline, key, encodedRow, ttl=0) + persistence.save(pipeline, key, encodedRow, ttl = 0) } } conn.close() @@ -68,6 +73,12 @@ class RedisSinkRelation(override val sqlContext: SQLContext, } } + private def compactRowsToLatest(rows: Seq[(String, Row)]) = rows + .groupBy(_._1) + .values + .map(_.maxBy(_._2.getAs[java.sql.Timestamp](config.timestampColumn).getTime)) + + private def dataKeyId(row: Row): String = { val entityKey = config.entityColumns.map(row.getAs[Any]).map(_.toString).mkString(":") val entityPrefix = config.entityColumns.sorted.mkString("_") diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala index 518690485f..f2691f89a0 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala @@ -1,14 +1,14 @@ package feast.ingestion.stores.redis -import redis.clients.jedis.Protocol case class SparkRedisConfig( - val namespace: String, - val entityNames: Array[String], - val entityColumns: Array[String], - val timestampColumn: String, - val iteratorGroupingSize: Int = 1000, - val timestampPrefix: String = "_ts" + namespace: String, + entityNames: Array[String], + entityColumns: Array[String], + timestampColumn: String, + iteratorGroupingSize: Int = 1000, + timestampPrefix: String = "_ts", + repartitionByEntity: Boolean = true ) object SparkRedisConfig { @@ -16,12 +16,14 @@ object SparkRedisConfig { val ENTITY_NAMES = "entity_names" val ENTITY_COLUMNS = "entity_columns" val TS_COLUMN = "timestamp_column" + val ENTITY_REPARTITION = "entity_repartition" def parse(parameters: Map[String, String]): SparkRedisConfig = SparkRedisConfig( namespace = parameters.getOrElse(NAMESPACE, ""), entityNames = parameters.getOrElse(ENTITY_NAMES, "").split(","), entityColumns = parameters.getOrElse(ENTITY_COLUMNS, "").split(","), - timestampColumn = parameters.getOrElse(TS_COLUMN, "event_timestamp") + timestampColumn = parameters.getOrElse(TS_COLUMN, "event_timestamp"), + repartitionByEntity = parameters.getOrElse(ENTITY_REPARTITION, "true") == "true" ) } diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala index ac6dfc056c..cf0897f966 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala @@ -10,28 +10,31 @@ import feast.proto.types.ValueProto.ValueType import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ -import org.joda.time.DateTime +import org.joda.time.{DateTime, Seconds} import org.scalacheck._ import org.scalatest._ import matchers.should.Matchers._ + import org.scalatest.matchers.Matcher import redis.clients.jedis.Jedis import scala.jdk.CollectionConverters.mapAsScalaMapConverter import scala.reflect.runtime.universe.TypeTag + case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) -class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { +class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer with BeforeAndAfter { override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) trait SparkContext { val sparkConf = new SparkConf() - .setMaster("local") + .setMaster("local[4]") .setAppName("Testing") .set("spark.redis.host", container.host) + .set("spark.default.parallelism", "20") .set("spark.redis.port", container.mappedPort(6379).toString) val sparkSession = SparkSession @@ -40,10 +43,10 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { .getOrCreate() } - trait DataGenerator { + trait DataHelper { self: SparkContext => - def storeFiles[T <: Product : TypeTag](rows: Seq[T]): String = { + def storeAsParquet[T <: Product : TypeTag](rows: Seq[T]): String = { import self.sparkSession.implicits._ val tempPath = Files.createTempDirectory("test-dir").toFile.toString + "/rows" @@ -51,7 +54,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { sparkSession.createDataset(rows) .withColumn("date", to_date($"eventTimestamp")) .write - .partitionBy("eventTimestamp") + .partitionBy("date") .save(tempPath) tempPath @@ -59,14 +62,21 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { } trait Scope { - val rowGenerator = for { - customer <- Gen.asciiPrintableStr + val jedis = new Jedis("localhost", container.mappedPort(6379)) + jedis.flushAll() + + def rowGenerator(start: DateTime, end: DateTime, customerGen: Option[Gen[String]] = None) = for { + customer <- customerGen.getOrElse(Gen.asciiPrintableStr) feature1 <- Gen.choose(0, 100) - feature2 <- Gen.choose(0, 1) - eventTimestamp <- Gen.choose(0, 30 * 24 * 3600).map(DateTime.now().withMillisOfSecond(0).minusSeconds(_)) + feature2 <- Gen.choose[Float](0, 1) + eventTimestamp <- Gen.choose(0, Seconds.secondsBetween(start, end).getSeconds) + .map(start.withMillisOfSecond(0).plusSeconds) } yield Row(customer, feature1, feature2, new java.sql.Timestamp(eventTimestamp.getMillis)) + def generateDistinctRows(gen: Gen[Row], N: Int) = + Gen.listOfN(N, gen).sample.get.groupBy(_.customer).map(_._2.head).toSeq + val config = IngestionJobConfig( featureTable = FeatureTable( name = "test-fs", @@ -75,10 +85,9 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { Field("feature1", ValueType.Enum.INT32), Field("feature2", ValueType.Enum.FLOAT), ), - //offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp")) ), - startTime = DateTime.now().minusDays(30), - endTime = DateTime.now() + startTime = DateTime.parse("2020-08-01"), + endTime = DateTime.parse("2020-09-01") ) def beStoredRow(row: Row) = { @@ -90,32 +99,85 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { m compose { (_: Map[Array[Byte], Array[Byte]]) - .map(e => ( - new String(e._1), - if (new String(e._1).startsWith("_ts")) - Timestamp.parseFrom(e._2).asScala else - ValueProto.Value.parseFrom(e._2).asScala - )) + .map { + case (k, v) => ( + new String(k), + if (new String(k).startsWith("_ts")) + Timestamp.parseFrom(v).asScala else + ValueProto.Value.parseFrom(v).asScala + ) + } } } } - "Parquet source file" should "be ingested in redis" in new Scope with SparkContext with DataGenerator { - val rows = Gen.listOfN(100, rowGenerator).sample.get - val tempPath = storeFiles(rows) + "Parquet source file" should "be ingested in redis" in new Scope with SparkContext with DataHelper { + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val rows = generateDistinctRows(gen, 10000) + val tempPath = storeAsParquet(rows) val configWithOfflineSource = config.copy(featureTable = config.featureTable.copy(offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp")))) OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) - val jedis = new Jedis("localhost", container.mappedPort(6379)) rows.foreach( r => { val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap storedValues should beStoredRow(r) } ) + } + + "Ingested rows" should "be compacted before storing" in new Scope with SparkContext with DataHelper { + val entities = (0 to 10000).map(_.toString) + + val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) + val latest = generateDistinctRows(genLatest, 10000) + + val genOld = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-08-14"), Some(Gen.oneOf(entities))) + val old = generateDistinctRows(genOld, 10000) + + val tempPath = storeAsParquet(latest ++ old) + val configWithOfflineSource = config.copy(featureTable = config.featureTable.copy(offline_source = + Some(GSSource(tempPath, Map.empty, "eventTimestamp")))) + + OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + + latest.foreach( + r => { + val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap + storedValues should beStoredRow(r) + } + ) + } + + "Old rows in ingestion" should "not overwrite more recent rows from storage" in new Scope with SparkContext with DataHelper { + val entities = (0 to 10000).map(_.toString) + + val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) + val latest = generateDistinctRows(genLatest, 10000) + + val tempPath1 = storeAsParquet(latest) + val config1 = config.copy(featureTable = config.featureTable.copy(offline_source = + Some(GSSource(tempPath1, Map.empty, "eventTimestamp")))) + + OfflinePipeline.createPipeline(sparkSession, config1) + val genOld = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-08-14"), Some(Gen.oneOf(entities))) + val old = generateDistinctRows(genOld, 10000) + + val tempPath2 = storeAsParquet(old) + val config2 = config.copy(featureTable = config.featureTable.copy(offline_source = + Some(GSSource(tempPath2, Map.empty, "eventTimestamp")))) + + OfflinePipeline.createPipeline(sparkSession, config2) + + latest.foreach( + r => { + val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap + storedValues should beStoredRow(r) + } + ) } } From 33e8ff32d42e64d7d2b882b482a7524781d67ce8 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 28 Sep 2020 19:33:03 +0800 Subject: [PATCH 05/15] clarify Signed-off-by: Oleksii Moskalenko --- .../feast/ingestion/stores/redis/RedisSinkRelation.scala | 4 ++-- .../src/test/scala/feast/ingestion/OfflinePipelineSpec.scala | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index 07c130f538..ae58e68b30 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -38,7 +38,7 @@ class RedisSinkRelation(override val sqlContext: SQLContext, dataToStore.foreachPartition { partition: Iterator[Row] => // grouped iterator to only allocate memory for a portion of rows partition.grouped(config.iteratorGroupingSize).foreach { batch => - val rowsWithKey: Map[String, Row] = compactRowsToLatest(batch.map(row => dataKeyId(row) -> row)).toMap + val rowsWithKey: Map[String, Row] = compactRowsToLatestTimestamp(batch.map(row => dataKeyId(row) -> row)).toMap groupKeysByNode(redisConfig.hosts, rowsWithKey.keysIterator).foreach { case (node, keys) => val conn = node.connect() @@ -73,7 +73,7 @@ class RedisSinkRelation(override val sqlContext: SQLContext, } } - private def compactRowsToLatest(rows: Seq[(String, Row)]) = rows + private def compactRowsToLatestTimestamp(rows: Seq[(String, Row)]) = rows .groupBy(_._1) .values .map(_.maxBy(_._2.getAs[java.sql.Timestamp](config.timestampColumn).getTime)) diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala index cf0897f966..022a78027f 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala @@ -111,7 +111,6 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer with BeforeA } } - "Parquet source file" should "be ingested in redis" in new Scope with SparkContext with DataHelper { val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) val rows = generateDistinctRows(gen, 10000) @@ -129,7 +128,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer with BeforeA ) } - "Ingested rows" should "be compacted before storing" in new Scope with SparkContext with DataHelper { + "Ingested rows" should "be compacted before storing by timestamp column" in new Scope with SparkContext with DataHelper { val entities = (0 to 10000).map(_.toString) val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) From c326f5a3cabf1b5242951c2addafcd35ab0d765a Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 28 Sep 2020 21:10:01 +0800 Subject: [PATCH 06/15] validation & deadletter Signed-off-by: Oleksii Moskalenko --- .../feast/ingestion/IngestionJobConfig.scala | 3 +- .../feast/ingestion/OfflinePipeline.scala | 28 ++++++++++++++++--- .../stores/redis/RedisSinkRelation.scala | 3 ++ .../ingestion/validation/RowValidator.scala | 17 +++++++++++ .../source/RedisSinkMetricSource.scala | 4 ++- .../feast/ingestion/OfflinePipelineSpec.scala | 7 ++--- 6 files changed, 52 insertions(+), 10 deletions(-) create mode 100644 ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala index b395146c7a..f9f37c96ff 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala @@ -22,5 +22,6 @@ case class IngestionJobConfig( startTime: DateTime = DateTime.now(), endTime: DateTime = DateTime.now(), store: StoreConfig = RedisConfig("localhost:6379"), - metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)) + metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)), + deadLetterPath: Option[String] = None ) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala index 96f0cfa40a..ed1bdb8891 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala @@ -2,11 +2,16 @@ package feast.ingestion import feast.ingestion.sources.bq.BigQueryReader import feast.ingestion.sources.file.FileReader +import feast.ingestion.validation.RowValidator import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.functions.col object OfflinePipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { + val projection = inputProjection( + config.featureTable.offline_source.get, config.featureTable.features, config.featureTable.entities) + val validator = new RowValidator(config.featureTable) + val input = config.featureTable.offline_source match { case Some(source: BQSource) => BigQueryReader.createBatchSource( @@ -21,8 +26,12 @@ object OfflinePipeline extends BasePipeline { ) } - val projection = inputProjection(config.featureTable.offline_source.get, config.featureTable.features, config.featureTable.entities) - input.select(projection: _*) + val projected = input.select(projection: _*).cache() + + val validRows = projected + .filter(validator.checkAll) + + validRows .write .format("feast.ingestion.stores.redis") .option("entity_columns", config.featureTable.entities.map(_.name).mkString(",")) @@ -30,10 +39,21 @@ object OfflinePipeline extends BasePipeline { .option("namespace", config.featureTable.name) .option("timestamp_column", config.featureTable.offline_source.get.timestampColumn) .save() + + config.deadLetterPath match { + case Some(path) => + projected + .filter(!validator.checkAll) + .write + .format("parquet") + .save(path) + case _ => None + } + } private def inputProjection(source: OfflineSource, features: Seq[Field], entities: Seq[Field]): Array[Column] = { - val mainColumns = + val featureColumns = if (source.mapping.nonEmpty) source.mapping else features.map(f => (f.name, f.name)) @@ -41,7 +61,7 @@ object OfflinePipeline extends BasePipeline { val timestampColumn = Seq((source.timestampColumn, source.timestampColumn)) val entitiesColumns = entities.map(e => (e.name, e.name)) - (mainColumns ++ entitiesColumns ++ timestampColumn).map { + (featureColumns ++ entitiesColumns ++ timestampColumn).map { case (alias, source) => col(source).alias(alias) }.toArray } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index ae58e68b30..71336fb0d6 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -60,7 +60,10 @@ class RedisSinkRelation(override val sqlContext: SQLContext, case Some(t) if !t.before(row.getAs[java.sql.Timestamp](config.timestampColumn)) => () case _ => if (metricSource.nonEmpty) { + val lag = System.currentTimeMillis() - row.getAs[java.sql.Timestamp](config.timestampColumn).getTime + metricSource.get.METRIC_TOTAL_ROWS_INSERTED.inc() + metricSource.get.METRIC_ROWS_LAG.update(lag) } val encodedRow = persistence.encodeRow(config.entityColumns, timestampField, row) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala b/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala new file mode 100644 index 0000000000..6205d08237 --- /dev/null +++ b/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala @@ -0,0 +1,17 @@ +package feast.ingestion.validation + +import feast.ingestion.FeatureTable +import org.apache.spark.sql.Column +import org.apache.spark.sql.functions.col + +class RowValidator(featureTable: FeatureTable) extends Serializable { + def allEntitiesPresent: Column = + featureTable.entities.map(e => col(e.name).isNotNull).reduce(_.&&(_)) + + def atLeastOneFeatureNotNull: Column = + featureTable.features.map(f => col(f.name).isNotNull).reduce(_.||(_)) + + def checkAll: Column = + allEntitiesPresent && atLeastOneFeatureNotNull +} + diff --git a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala index 123c79038b..8b2ebb5911 100644 --- a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala +++ b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala @@ -7,5 +7,7 @@ class RedisSinkMetricSource extends Source { override val metricRegistry: MetricRegistry = new MetricRegistry - val METRIC_TOTAL_ROWS_INSERTED = metricRegistry.counter(MetricRegistry.name("rowsInserted")) + val METRIC_TOTAL_ROWS_INSERTED = metricRegistry.counter(MetricRegistry.name("feast_ingestion_feature_row_ingested_count")) + + val METRIC_ROWS_LAG = metricRegistry.histogram(MetricRegistry.name("feast_ingestion_feature_row_lag_ms")) } diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala index 022a78027f..9869c2f3d6 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala @@ -1,6 +1,6 @@ package feast.ingestion -import java.nio.file.Files +import java.nio.file.{Files, Paths} import com.dimafeng.testcontainers.{ForAllTestContainer, GenericContainer} import com.google.protobuf.Timestamp @@ -14,7 +14,6 @@ import org.joda.time.{DateTime, Seconds} import org.scalacheck._ import org.scalatest._ import matchers.should.Matchers._ - import org.scalatest.matchers.Matcher import redis.clients.jedis.Jedis @@ -25,7 +24,7 @@ import scala.reflect.runtime.universe.TypeTag case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) -class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer with BeforeAndAfter { +class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) @@ -49,7 +48,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer with BeforeA def storeAsParquet[T <: Product : TypeTag](rows: Seq[T]): String = { import self.sparkSession.implicits._ - val tempPath = Files.createTempDirectory("test-dir").toFile.toString + "/rows" + val tempPath = Paths.get(Files.createTempDirectory("test-dir").toString, "rows").toString sparkSession.createDataset(rows) .withColumn("date", to_date($"eventTimestamp")) From 738afd50c3a34931fa4c421e776230e4f959a364 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Tue, 29 Sep 2020 12:37:24 +0800 Subject: [PATCH 07/15] tests on mapping & deadletter Signed-off-by: Oleksii Moskalenko --- .../scala/feast/ingestion/FeatureTable.scala | 10 +- .../scala/feast/ingestion/IngestionJob.scala | 3 +- .../feast/ingestion/OfflinePipeline.scala | 16 +-- .../stores/redis/HashTypePersistence.scala | 10 +- .../stores/redis/RedisSinkRelation.scala | 7 +- .../stores/redis/SparkRedisConfig.scala | 6 +- ...lineSpec.scala => OfflinePipelineIT.scala} | 131 +++++++++++++++--- 7 files changed, 141 insertions(+), 42 deletions(-) rename ingestion-spark/src/test/scala/feast/ingestion/{OfflinePipelineSpec.scala => OfflinePipelineIT.scala} (55%) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala index 5a427c0c8f..09a436088d 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala @@ -2,21 +2,25 @@ package feast.ingestion abstract class Source { def mapping: Map[String, String] + def timestampColumn: String } abstract class OfflineSource extends Source + abstract class OnlineSource extends Source -case class GSSource(path: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource -case class BQSource(project: String, dataset: String, table: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource +case class GSSource(path: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource + +case class BQSource(project: String, dataset: String, table: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource -case class KafkaSource(bootstrapServers: String, topic: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OnlineSource +case class KafkaSource(bootstrapServers: String, topic: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OnlineSource case class Field(name: String, `type`: feast.proto.types.ValueProto.ValueType.Enum) case class FeatureTable(name: String, + project: String, entities: Seq[Field], features: Seq[Field], offline_source: Option[OfflineSource] = None, diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala index 9bd54124d3..c4c2de7321 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala @@ -17,7 +17,8 @@ object IngestionJob { opt[String](name="feature-table-spec") .action((x, c) => c.copy(featureTable = FeatureTable( - name="jaeger-car", + name = "jaeger-car", + project = "default", entities = Seq(Field(name="customer", `type` = ValueType.Enum.STRING)), features = Seq( Field("car_customer_id_avg_customer_distance_cancelled", ValueType.Enum.FLOAT), diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala index ed1bdb8891..b9d27ce853 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala @@ -8,9 +8,9 @@ import org.apache.spark.sql.functions.col object OfflinePipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { - val projection = inputProjection( - config.featureTable.offline_source.get, config.featureTable.features, config.featureTable.entities) - val validator = new RowValidator(config.featureTable) + val featureTable = config.featureTable + val projection = inputProjection(featureTable.offline_source.get, featureTable.features, featureTable.entities) + val validator = new RowValidator(featureTable) val input = config.featureTable.offline_source match { case Some(source: BQSource) => @@ -34,10 +34,10 @@ object OfflinePipeline extends BasePipeline { validRows .write .format("feast.ingestion.stores.redis") - .option("entity_columns", config.featureTable.entities.map(_.name).mkString(",")) - .option("entity_names", config.featureTable.entities.map(_.name).mkString(",")) - .option("namespace", config.featureTable.name) - .option("timestamp_column", config.featureTable.offline_source.get.timestampColumn) + .option("entity_columns", featureTable.entities.map(_.name).mkString(",")) + .option("namespace", featureTable.name) + .option("project_name", featureTable.project) + .option("timestamp_column", featureTable.offline_source.get.timestampColumn) .save() config.deadLetterPath match { @@ -59,7 +59,7 @@ object OfflinePipeline extends BasePipeline { else features.map(f => (f.name, f.name)) val timestampColumn = Seq((source.timestampColumn, source.timestampColumn)) - val entitiesColumns = entities.map(e => (e.name, e.name)) + val entitiesColumns = entities.filter(e => !source.mapping.contains(e.name)).map(e => (e.name, e.name)) (featureColumns ++ entitiesColumns ++ timestampColumn).map { case (alias, source) => col(source).alias(alias) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala index 6f07dd72d7..b8e2b7dad3 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala @@ -8,8 +8,7 @@ import redis.clients.jedis.{Pipeline, Response} import scala.jdk.CollectionConverters._ import com.google.protobuf.Timestamp import feast.ingestion.utils.TypeConversion -import feast.proto.types.ValueProto - +import scala.util.hashing.MurmurHash3 class HashTypePersistence(config: SparkRedisConfig) extends Serializable { def encodeRow(keyColumns: Array[String], timestampField: String, value: Row): Map[Array[Byte], Array[Byte]] = { @@ -27,7 +26,7 @@ class HashTypePersistence(config: SparkRedisConfig) extends Serializable { !keyColumns.contains(k) && k != config.timestampColumn } .map { case (k, v) => - k.getBytes -> encodeValue(v, types(k)) + encodeKey(k) -> encodeValue(v, types(k)) } val timestamp = Seq(( @@ -41,6 +40,11 @@ class HashTypePersistence(config: SparkRedisConfig) extends Serializable { TypeConversion.sqlTypeToProtoValue(value, `type`).toByteArray } + def encodeKey(key: String): Array[Byte] = { + val fullFeatureReference = s"${config.namespace}:$key" + MurmurHash3.stringHash(fullFeatureReference).toHexString.getBytes + } + def save(pipeline: Pipeline, key: String, value: Map[Array[Byte], Array[Byte]], ttl: Int): Unit = { pipeline.hset(key.getBytes, value.asJava) if (ttl > 0) { diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index 71336fb0d6..369662c2cc 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -83,9 +83,10 @@ class RedisSinkRelation(override val sqlContext: SQLContext, private def dataKeyId(row: Row): String = { - val entityKey = config.entityColumns.map(row.getAs[Any]).map(_.toString).mkString(":") - val entityPrefix = config.entityColumns.sorted.mkString("_") - s"${entityPrefix}:$entityKey" + val sortedEntities = config.entityColumns.sorted + val entityKey = sortedEntities.map(row.getAs[Any]).map(_.toString).mkString(":") + val entityPrefix = sortedEntities.mkString("_") + s"${config.projectName}_${entityPrefix}:$entityKey" } private def timestampField: String = { diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala index f2691f89a0..3ace7fe0dd 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala @@ -3,7 +3,7 @@ package feast.ingestion.stores.redis case class SparkRedisConfig( namespace: String, - entityNames: Array[String], + projectName: String, entityColumns: Array[String], timestampColumn: String, iteratorGroupingSize: Int = 1000, @@ -13,15 +13,15 @@ case class SparkRedisConfig( object SparkRedisConfig { val NAMESPACE = "namespace" - val ENTITY_NAMES = "entity_names" val ENTITY_COLUMNS = "entity_columns" val TS_COLUMN = "timestamp_column" val ENTITY_REPARTITION = "entity_repartition" + val PROJECT_NAME = "project_name" def parse(parameters: Map[String, String]): SparkRedisConfig = SparkRedisConfig( namespace = parameters.getOrElse(NAMESPACE, ""), - entityNames = parameters.getOrElse(ENTITY_NAMES, "").split(","), + projectName = parameters.getOrElse(PROJECT_NAME, "default"), entityColumns = parameters.getOrElse(ENTITY_COLUMNS, "").split(","), timestampColumn = parameters.getOrElse(TS_COLUMN, "event_timestamp"), repartitionByEntity = parameters.getOrElse(ENTITY_REPARTITION, "true") == "true" diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala similarity index 55% rename from ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala rename to ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala index 9869c2f3d6..5125c2fd55 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala @@ -15,16 +15,17 @@ import org.scalacheck._ import org.scalatest._ import matchers.should.Matchers._ import org.scalatest.matchers.Matcher + import redis.clients.jedis.Jedis -import scala.jdk.CollectionConverters.mapAsScalaMapConverter +import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag - +import scala.util.hashing.MurmurHash3 case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) -class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { +class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) @@ -33,7 +34,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { .setMaster("local[4]") .setAppName("Testing") .set("spark.redis.host", container.host) - .set("spark.default.parallelism", "20") + .set("spark.default.parallelism", "8") .set("spark.redis.port", container.mappedPort(6379).toString) val sparkSession = SparkSession @@ -45,10 +46,13 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { trait DataHelper { self: SparkContext => + def generateTempPath(last: String) = + Paths.get(Files.createTempDirectory("test-dir").toString, last).toString + def storeAsParquet[T <: Product : TypeTag](rows: Seq[T]): String = { import self.sparkSession.implicits._ - val tempPath = Paths.get(Files.createTempDirectory("test-dir").toString, "rows").toString + val tempPath = generateTempPath("rows") sparkSession.createDataset(rows) .withColumn("date", to_date($"eventTimestamp")) @@ -60,7 +64,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { } } - trait Scope { + trait Scope extends SparkContext with DataHelper { val jedis = new Jedis("localhost", container.mappedPort(6379)) jedis.flushAll() @@ -79,6 +83,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { val config = IngestionJobConfig( featureTable = FeatureTable( name = "test-fs", + project = "default", entities = Seq(Field("customer", ValueType.Enum.STRING)), features = Seq( Field("feature1", ValueType.Enum.INT32), @@ -89,12 +94,18 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { endTime = DateTime.parse("2020-09-01") ) - def beStoredRow(row: Row) = { - val m: Matcher[Map[String, Any]] = contain.allElementsOf(Seq( - "feature1" -> row.feature1, - "feature2" -> row.feature2, - "_ts:test-fs" -> row.eventTimestamp - )).matcher + def encodeEntityKey(row: Row, featureTable: FeatureTable) = { + val entityPrefix = featureTable.entities.map(_.name).mkString("_") + s"${featureTable.project}_${entityPrefix}:${row.customer}".getBytes + } + + def encodeFeatureKey(featureTable: FeatureTable)(feature: String): String = { + val fullReference = s"${featureTable.name}:$feature" + MurmurHash3.stringHash(fullReference).toHexString + } + + def beStoredRow(mappedRow: Map[String, Any]) = { + val m: Matcher[Map[String, Any]] = contain.allElementsOf(mappedRow).matcher m compose { (_: Map[Array[Byte], Array[Byte]]) @@ -110,7 +121,7 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { } } - "Parquet source file" should "be ingested in redis" in new Scope with SparkContext with DataHelper { + "Parquet source file" should "be ingested in redis" in new Scope { val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) val rows = generateDistinctRows(gen, 10000) val tempPath = storeAsParquet(rows) @@ -119,15 +130,21 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) + rows.foreach( r => { - val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap - storedValues should beStoredRow(r) + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow(Map( + featureKeyEncoder("feature1") -> r.feature1, + featureKeyEncoder("feature2") -> r.feature2, + "_ts:test-fs" -> r.eventTimestamp + )) } ) } - "Ingested rows" should "be compacted before storing by timestamp column" in new Scope with SparkContext with DataHelper { + "Ingested rows" should "be compacted before storing by timestamp column" in new Scope { val entities = (0 to 10000).map(_.toString) val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) @@ -142,15 +159,21 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) + latest.foreach( r => { - val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap - storedValues should beStoredRow(r) + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow(Map( + featureKeyEncoder("feature1") -> r.feature1, + featureKeyEncoder("feature2") -> r.feature2, + "_ts:test-fs" -> r.eventTimestamp + )) } ) } - "Old rows in ingestion" should "not overwrite more recent rows from storage" in new Scope with SparkContext with DataHelper { + "Old rows in ingestion" should "not overwrite more recent rows from storage" in new Scope { val entities = (0 to 10000).map(_.toString) val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) @@ -171,10 +194,76 @@ class OfflinePipelineSpec extends UnitSpec with ForAllTestContainer { OfflinePipeline.createPipeline(sparkSession, config2) + val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) + latest.foreach( r => { - val storedValues = jedis.hgetAll(s"customer:${r.customer}".getBytes).asScala.toMap - storedValues should beStoredRow(r) + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow(Map( + featureKeyEncoder("feature1") -> r.feature1, + featureKeyEncoder("feature2") -> r.feature2, + "_ts:test-fs" -> r.eventTimestamp + )) + } + ) + } + + "Invalid rows" should "not be ingested and stored to deadletter instead" in new Scope { + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val rows = generateDistinctRows(gen, 100) + + val rowsWithNullEntity = rows.map(_.copy(customer = null)) + + val tempPath = storeAsParquet(rowsWithNullEntity) + val deadletterConfig = config.copy( + featureTable = config.featureTable.copy(offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp"))), + deadLetterPath = Some(generateTempPath("deadletters")) + ) + + OfflinePipeline.createPipeline(sparkSession, deadletterConfig) + + jedis.keys("*").toArray should be (empty) + + sparkSession.read + .parquet(deadletterConfig.deadLetterPath.get) + .count() should be (rows.length) + } + + "Columns from source" should "be mapped according to configuration" in new Scope { + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val rows = generateDistinctRows(gen, 100) + + val tempPath = storeAsParquet(rows) + + val configWithMapping = config.copy( + featureTable = config.featureTable.copy( + entities = Seq(Field("entity", ValueType.Enum.STRING)), + features = Seq( + Field("new_feature1", ValueType.Enum.INT32), + Field("new_feature2", ValueType.Enum.FLOAT) + ), + offline_source = Some(GSSource( + tempPath, + Map( + "entity" -> "customer", + "new_feature1" -> "feature1", + "new_feature2" -> "feature2" + ), + "eventTimestamp"))) + ) + + OfflinePipeline.createPipeline(sparkSession, configWithMapping) + + val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) + + rows.foreach( + r => { + val storedValues = jedis.hgetAll(encodeEntityKey(r, configWithMapping.featureTable)).asScala.toMap + storedValues should beStoredRow(Map( + featureKeyEncoder("new_feature1") -> r.feature1, + featureKeyEncoder("new_feature2") -> r.feature2, + "_ts:test-fs" -> r.eventTimestamp + )) } ) } From 04e9abbc50c3373778c3285d24f2e61eb139dab5 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Tue, 29 Sep 2020 14:03:09 +0800 Subject: [PATCH 08/15] scala styling Signed-off-by: Oleksii Moskalenko --- .scalafmt.conf | 2 + .../scala/feast/ingestion/BasePipeline.scala | 22 +- .../scala/feast/ingestion/FeatureTable.scala | 56 +++-- .../scala/feast/ingestion/IngestionJob.scala | 56 +++-- .../feast/ingestion/IngestionJobConfig.scala | 32 ++- .../feast/ingestion/OfflinePipeline.scala | 40 +++- .../ingestion/sources/bq/BigQueryReader.scala | 24 ++- .../ingestion/sources/file/FileReader.scala | 24 ++- .../stores/redis/DefaultSource.scala | 35 ++- .../stores/redis/HashTypePersistence.scala | 49 ++++- .../stores/redis/RedisSinkRelation.scala | 44 +++- .../stores/redis/SparkRedisConfig.scala | 41 ++-- .../ingestion/utils/TypeConversion.scala | 50 ++++- .../ingestion/validation/RowValidator.scala | 17 +- .../source/RedisSinkMetricSource.scala | 22 +- .../feast/ingestion/OfflinePipelineIT.scala | 204 +++++++++++------- .../test/scala/feast/ingestion/UnitSpec.scala | 24 ++- pom.xml | 51 +++-- 18 files changed, 594 insertions(+), 199 deletions(-) create mode 100644 .scalafmt.conf diff --git a/.scalafmt.conf b/.scalafmt.conf new file mode 100644 index 0000000000..f3c72b8ceb --- /dev/null +++ b/.scalafmt.conf @@ -0,0 +1,2 @@ +align.preset = more +maxColumn = 100 \ No newline at end of file diff --git a/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala index 64fd556592..7cd0c138f9 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import org.apache.spark.SparkConf @@ -17,7 +33,10 @@ trait BasePipeline { jobConfig.metrics match { case Some(c: StatsDConfig) => conf - .set("spark.metrics.conf.*.source.redis.class", "org.apache.spark.metrics.source.RedisSinkMetricSource") + .set( + "spark.metrics.conf.*.source.redis.class", + "org.apache.spark.metrics.source.RedisSinkMetricSource" + ) .set("spark.metrics.conf.*.sink.statsd.class", "org.apache.spark.metrics.sink.StatsdSink") .set("spark.metrics.conf.*.sink.statsd.host", c.host) .set("spark.metrics.conf.*.sink.statsd.port", c.port.toString) @@ -31,6 +50,5 @@ trait BasePipeline { .getOrCreate() } - def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala index 09a436088d..432e5a7f89 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion abstract class Source { @@ -10,18 +26,34 @@ abstract class OfflineSource extends Source abstract class OnlineSource extends Source -case class GSSource(path: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource - -case class BQSource(project: String, dataset: String, table: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OfflineSource - - -case class KafkaSource(bootstrapServers: String, topic: String, override val mapping: Map[String, String], override val timestampColumn: String) extends OnlineSource +case class GSSource( + path: String, + override val mapping: Map[String, String], + override val timestampColumn: String +) extends OfflineSource + +case class BQSource( + project: String, + dataset: String, + table: String, + override val mapping: Map[String, String], + override val timestampColumn: String +) extends OfflineSource + +case class KafkaSource( + bootstrapServers: String, + topic: String, + override val mapping: Map[String, String], + override val timestampColumn: String +) extends OnlineSource case class Field(name: String, `type`: feast.proto.types.ValueProto.ValueType.Enum) -case class FeatureTable(name: String, - project: String, - entities: Seq[Field], - features: Seq[Field], - offline_source: Option[OfflineSource] = None, - online_source: Option[OnlineSource] = None) +case class FeatureTable( + name: String, + project: String, + entities: Seq[Field], + features: Seq[Field], + offline_source: Option[OfflineSource] = None, + online_source: Option[OnlineSource] = None +) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala index c4c2de7321..187852f537 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import feast.proto.types.ValueProto.ValueType @@ -15,26 +31,38 @@ object IngestionJob { .required() .text("Mode to operate ingestion job (offline or online)") - opt[String](name="feature-table-spec") - .action((x, c) => c.copy(featureTable = FeatureTable( - name = "jaeger-car", - project = "default", - entities = Seq(Field(name="customer", `type` = ValueType.Enum.STRING)), - features = Seq( - Field("car_customer_id_avg_customer_distance_cancelled", ValueType.Enum.FLOAT), - Field("car_customer_id_num_completed", ValueType.Enum.FLOAT), - Field("car_customer_id_origin_completed_1", ValueType.Enum.INT32) - ), - offline_source = Some(BQSource("gods-staging", "feast", "default_jaeger_car_customer", Map.empty, "event_timestamp")) - ))) + opt[String](name = "feature-table-spec") + .action((x, c) => + c.copy(featureTable = + FeatureTable( + name = "jaeger-car", + project = "default", + entities = Seq(Field(name = "customer", `type` = ValueType.Enum.STRING)), + features = Seq( + Field("car_customer_id_avg_customer_distance_cancelled", ValueType.Enum.FLOAT), + Field("car_customer_id_num_completed", ValueType.Enum.FLOAT), + Field("car_customer_id_origin_completed_1", ValueType.Enum.INT32) + ), + offline_source = Some( + BQSource( + "gods-staging", + "feast", + "default_jaeger_car_customer", + Map.empty, + "event_timestamp" + ) + ) + ) + ) + ) .required() .text("JSON-encoded FeatureTableSpec object") - opt[String](name="start") + opt[String](name = "start") .action((x, c) => c.copy(startTime = DateTime.parse(x))) .text("Start timestamp for offline ingestion") - opt[String](name="end") + opt[String](name = "end") .action((x, c) => c.copy(endTime = DateTime.parse(x))) .text("End timestamp for offline ingestion") } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala index f9f37c96ff..fc5f3d304b 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import feast.ingestion.Modes.Modes @@ -17,11 +33,11 @@ abstract class MetricConfig case class StatsDConfig(host: String, port: Int) extends MetricConfig case class IngestionJobConfig( - mode: Modes = Modes.Offline, - featureTable: FeatureTable = null, - startTime: DateTime = DateTime.now(), - endTime: DateTime = DateTime.now(), - store: StoreConfig = RedisConfig("localhost:6379"), - metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)), - deadLetterPath: Option[String] = None - ) + mode: Modes = Modes.Offline, + featureTable: FeatureTable = null, + startTime: DateTime = DateTime.now(), + endTime: DateTime = DateTime.now(), + store: StoreConfig = RedisConfig("localhost:6379"), + metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)), + deadLetterPath: Option[String] = None +) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala index b9d27ce853..5589b2fbfe 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import feast.ingestion.sources.bq.BigQueryReader @@ -9,7 +25,8 @@ import org.apache.spark.sql.functions.col object OfflinePipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { val featureTable = config.featureTable - val projection = inputProjection(featureTable.offline_source.get, featureTable.features, featureTable.entities) + val projection = + inputProjection(featureTable.offline_source.get, featureTable.features, featureTable.entities) val validator = new RowValidator(featureTable) val input = config.featureTable.offline_source match { @@ -22,7 +39,10 @@ object OfflinePipeline extends BasePipeline { ) case Some(source: GSSource) => FileReader.createBatchSource( - sparkSession.sqlContext, source, config.startTime, config.endTime + sparkSession.sqlContext, + source, + config.startTime, + config.endTime ) } @@ -31,8 +51,7 @@ object OfflinePipeline extends BasePipeline { val validRows = projected .filter(validator.checkAll) - validRows - .write + validRows.write .format("feast.ingestion.stores.redis") .option("entity_columns", featureTable.entities.map(_.name).mkString(",")) .option("namespace", featureTable.name) @@ -52,17 +71,22 @@ object OfflinePipeline extends BasePipeline { } - private def inputProjection(source: OfflineSource, features: Seq[Field], entities: Seq[Field]): Array[Column] = { + private def inputProjection( + source: OfflineSource, + features: Seq[Field], + entities: Seq[Field] + ): Array[Column] = { val featureColumns = if (source.mapping.nonEmpty) source.mapping else features.map(f => (f.name, f.name)) val timestampColumn = Seq((source.timestampColumn, source.timestampColumn)) - val entitiesColumns = entities.filter(e => !source.mapping.contains(e.name)).map(e => (e.name, e.name)) + val entitiesColumns = + entities.filter(e => !source.mapping.contains(e.name)).map(e => (e.name, e.name)) - (featureColumns ++ entitiesColumns ++ timestampColumn).map { - case (alias, source) => col(source).alias(alias) + (featureColumns ++ entitiesColumns ++ timestampColumn).map { case (alias, source) => + col(source).alias(alias) }.toArray } } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala b/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala index f540a168cc..5dc5a7fddd 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.sources.bq import java.sql.Timestamp @@ -8,8 +24,12 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.functions.col object BigQueryReader { - def createBatchSource(sqlContext: SQLContext, source: BQSource, - start: DateTime, end: DateTime): DataFrame = { + def createBatchSource( + sqlContext: SQLContext, + source: BQSource, + start: DateTime, + end: DateTime + ): DataFrame = { sqlContext.read .format("bigquery") .load(s"${source.project}.${source.dataset}.${source.table}") diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala b/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala index 8c4500437a..490b5e542b 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.sources.file import java.sql.Timestamp @@ -8,8 +24,12 @@ import org.apache.spark.sql.{DataFrame, SQLContext} import org.joda.time.DateTime object FileReader { - def createBatchSource(sqlContext: SQLContext, source: GSSource, - start: DateTime, end: DateTime): DataFrame = { + def createBatchSource( + sqlContext: SQLContext, + source: GSSource, + start: DateTime, + end: DateTime + ): DataFrame = { sqlContext.read .parquet(source.path) .filter(col(source.timestampColumn) >= new Timestamp(start.getMillis)) diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala index d02358c080..d870afc09c 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.stores.redis import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} @@ -5,10 +21,18 @@ import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, Re class RedisRelationProvider extends RelationProvider with CreatableRelationProvider { - override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = ??? - - override def createRelation(sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { - val config = SparkRedisConfig.parse(parameters) + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String] + ): BaseRelation = ??? + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame + ): BaseRelation = { + val config = SparkRedisConfig.parse(parameters) val relation = new RedisSinkRelation(sqlContext, config) relation.insert(data, overwrite = false) @@ -17,5 +41,4 @@ class RedisRelationProvider extends RelationProvider with CreatableRelationProvi } } - -class DefaultSource extends RedisRelationProvider \ No newline at end of file +class DefaultSource extends RedisRelationProvider diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala index b8e2b7dad3..fc5a45cca0 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala @@ -1,6 +1,21 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.stores.redis - import org.apache.spark.sql.Row import org.apache.spark.sql.types._ import redis.clients.jedis.{Pipeline, Response} @@ -11,10 +26,14 @@ import feast.ingestion.utils.TypeConversion import scala.util.hashing.MurmurHash3 class HashTypePersistence(config: SparkRedisConfig) extends Serializable { - def encodeRow(keyColumns: Array[String], timestampField: String, value: Row): Map[Array[Byte], Array[Byte]] = { + def encodeRow( + keyColumns: Array[String], + timestampField: String, + value: Row + ): Map[Array[Byte], Array[Byte]] = { val fields = value.schema.fields.map(_.name) - val types = value.schema.fields.map(f => (f.name, f.dataType)).toMap - val kvMap = value.getValuesMap[Any](fields) + val types = value.schema.fields.map(f => (f.name, f.dataType)).toMap + val kvMap = value.getValuesMap[Any](fields) val values = kvMap .filter { case (_, v) => @@ -29,9 +48,12 @@ class HashTypePersistence(config: SparkRedisConfig) extends Serializable { encodeKey(k) -> encodeValue(v, types(k)) } - val timestamp = Seq(( - timestampField.getBytes, - encodeValue(value.getAs[Timestamp](config.timestampColumn), TimestampType))) + val timestamp = Seq( + ( + timestampField.getBytes, + encodeValue(value.getAs[Timestamp](config.timestampColumn), TimestampType) + ) + ) values ++ timestamp } @@ -45,14 +67,23 @@ class HashTypePersistence(config: SparkRedisConfig) extends Serializable { MurmurHash3.stringHash(fullFeatureReference).toHexString.getBytes } - def save(pipeline: Pipeline, key: String, value: Map[Array[Byte], Array[Byte]], ttl: Int): Unit = { + def save( + pipeline: Pipeline, + key: String, + value: Map[Array[Byte], Array[Byte]], + ttl: Int + ): Unit = { pipeline.hset(key.getBytes, value.asJava) if (ttl > 0) { pipeline.expire(key, ttl) } } - def getTimestamp(pipeline: Pipeline, key: String, timestampField: String): Response[Array[Byte]] = { + def getTimestamp( + pipeline: Pipeline, + key: String, + timestampField: String + ): Response[Array[Byte]] = { pipeline.hget(key.getBytes(), timestampField.getBytes) } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index 369662c2cc..158d9dfca3 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.stores.redis import com.google.protobuf.Timestamp @@ -11,10 +27,10 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.functions.col import org.apache.spark.sql.{DataFrame, Row, SQLContext} - -class RedisSinkRelation(override val sqlContext: SQLContext, - config: SparkRedisConfig) extends BaseRelation - with InsertableRelation with Serializable { +class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisConfig) + extends BaseRelation + with InsertableRelation + with Serializable { private implicit val redisConfig: RedisConfig = { new RedisConfig( new RedisEndpoint(sqlContext.sparkContext.getConf) @@ -38,7 +54,8 @@ class RedisSinkRelation(override val sqlContext: SQLContext, dataToStore.foreachPartition { partition: Iterator[Row] => // grouped iterator to only allocate memory for a portion of rows partition.grouped(config.iteratorGroupingSize).foreach { batch => - val rowsWithKey: Map[String, Row] = compactRowsToLatestTimestamp(batch.map(row => dataKeyId(row) -> row)).toMap + val rowsWithKey: Map[String, Row] = + compactRowsToLatestTimestamp(batch.map(row => dataKeyId(row) -> row)).toMap groupKeysByNode(redisConfig.hosts, rowsWithKey.keysIterator).foreach { case (node, keys) => val conn = node.connect() @@ -48,7 +65,11 @@ class RedisSinkRelation(override val sqlContext: SQLContext, val timestampByKey = timestamps .map(_.asInstanceOf[Array[Byte]]) - .map(Option(_).map(Timestamp.parseFrom).map(t => new java.sql.Timestamp(t.getSeconds * 1000))) + .map( + Option(_) + .map(Timestamp.parseFrom) + .map(t => new java.sql.Timestamp(t.getSeconds * 1000)) + ) .zip(rowsWithKey.keys) .map(_.swap) .toMap @@ -60,7 +81,9 @@ class RedisSinkRelation(override val sqlContext: SQLContext, case Some(t) if !t.before(row.getAs[java.sql.Timestamp](config.timestampColumn)) => () case _ => if (metricSource.nonEmpty) { - val lag = System.currentTimeMillis() - row.getAs[java.sql.Timestamp](config.timestampColumn).getTime + val lag = System.currentTimeMillis() - row + .getAs[java.sql.Timestamp](config.timestampColumn) + .getTime metricSource.get.METRIC_TOTAL_ROWS_INSERTED.inc() metricSource.get.METRIC_ROWS_LAG.update(lag) @@ -81,11 +104,10 @@ class RedisSinkRelation(override val sqlContext: SQLContext, .values .map(_.maxBy(_._2.getAs[java.sql.Timestamp](config.timestampColumn).getTime)) - private def dataKeyId(row: Row): String = { val sortedEntities = config.entityColumns.sorted - val entityKey = sortedEntities.map(row.getAs[Any]).map(_.toString).mkString(":") - val entityPrefix = sortedEntities.mkString("_") + val entityKey = sortedEntities.map(row.getAs[Any]).map(_.toString).mkString(":") + val entityPrefix = sortedEntities.mkString("_") s"${config.projectName}_${entityPrefix}:$entityKey" } @@ -96,6 +118,6 @@ class RedisSinkRelation(override val sqlContext: SQLContext, private lazy val metricSource: Option[RedisSinkMetricSource] = SparkEnv.get.metricsSystem.getSourcesByName("redis_sink") match { case Seq(head) => Some(head.asInstanceOf[RedisSinkMetricSource]) - case _ => None + case _ => None } } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala index 3ace7fe0dd..389607ce99 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala @@ -1,22 +1,37 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.stores.redis - case class SparkRedisConfig( - namespace: String, - projectName: String, - entityColumns: Array[String], - timestampColumn: String, - iteratorGroupingSize: Int = 1000, - timestampPrefix: String = "_ts", - repartitionByEntity: Boolean = true - ) + namespace: String, + projectName: String, + entityColumns: Array[String], + timestampColumn: String, + iteratorGroupingSize: Int = 1000, + timestampPrefix: String = "_ts", + repartitionByEntity: Boolean = true +) object SparkRedisConfig { - val NAMESPACE = "namespace" - val ENTITY_COLUMNS = "entity_columns" - val TS_COLUMN = "timestamp_column" + val NAMESPACE = "namespace" + val ENTITY_COLUMNS = "entity_columns" + val TS_COLUMN = "timestamp_column" val ENTITY_REPARTITION = "entity_repartition" - val PROJECT_NAME = "project_name" + val PROJECT_NAME = "project_name" def parse(parameters: Map[String, String]): SparkRedisConfig = SparkRedisConfig( diff --git a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala index e550496a01..2bbaf70baf 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala @@ -1,21 +1,47 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.utils import java.sql import com.google.protobuf.{Message, Timestamp} import feast.proto.types.ValueProto -import org.apache.spark.sql.types.{DataType, DoubleType, FloatType, IntegerType, LongType, StringType, TimestampType} +import org.apache.spark.sql.types.{ + DataType, + DoubleType, + FloatType, + IntegerType, + LongType, + StringType, + TimestampType +} object TypeConversion { def sqlTypeToProtoValue(value: Any, `type`: DataType): Message = { (`type` match { case IntegerType => ValueProto.Value.newBuilder().setInt32Val(value.asInstanceOf[Int]) - case LongType => ValueProto.Value.newBuilder().setInt64Val(value.asInstanceOf[Long]) - case StringType => ValueProto.Value.newBuilder().setStringVal(value.asInstanceOf[String]) - case DoubleType => ValueProto.Value.newBuilder().setDoubleVal(value.asInstanceOf[Double]) - case FloatType => ValueProto.Value.newBuilder().setFloatVal(value.asInstanceOf[Float]) - case TimestampType => Timestamp.newBuilder() - .setSeconds(value.asInstanceOf[java.sql.Timestamp].getTime / 1000) + case LongType => ValueProto.Value.newBuilder().setInt64Val(value.asInstanceOf[Long]) + case StringType => ValueProto.Value.newBuilder().setStringVal(value.asInstanceOf[String]) + case DoubleType => ValueProto.Value.newBuilder().setDoubleVal(value.asInstanceOf[Double]) + case FloatType => ValueProto.Value.newBuilder().setFloatVal(value.asInstanceOf[Float]) + case TimestampType => + Timestamp + .newBuilder() + .setSeconds(value.asInstanceOf[java.sql.Timestamp].getTime / 1000) }).build } @@ -27,12 +53,14 @@ object TypeConversion { v.getValCase match { case ValueProto.Value.ValCase.INT32_VAL => v.getInt32Val case ValueProto.Value.ValCase.FLOAT_VAL => v.getFloatVal - case ValueProto.Value.ValCase.VAL_NOT_SET => throw new RuntimeException(s"$v not a ValueProto") + case ValueProto.Value.ValCase.VAL_NOT_SET => + throw new RuntimeException(s"$v not a ValueProto") } ) - implicit def timestampAsScala(t: Timestamp): AsScala[java.sql.Timestamp] = new AsScala[java.sql.Timestamp]( - new sql.Timestamp(t.getSeconds * 1000) - ) + implicit def timestampAsScala(t: Timestamp): AsScala[java.sql.Timestamp] = + new AsScala[java.sql.Timestamp]( + new sql.Timestamp(t.getSeconds * 1000) + ) } diff --git a/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala b/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala index 6205d08237..e2ea12ef0a 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.validation import feast.ingestion.FeatureTable @@ -14,4 +30,3 @@ class RowValidator(featureTable: FeatureTable) extends Serializable { def checkAll: Column = allEntitiesPresent && atLeastOneFeatureNotNull } - diff --git a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala index 8b2ebb5911..460ab8c978 100644 --- a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala +++ b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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.metrics.source import com.codahale.metrics.MetricRegistry @@ -7,7 +23,9 @@ class RedisSinkMetricSource extends Source { override val metricRegistry: MetricRegistry = new MetricRegistry - val METRIC_TOTAL_ROWS_INSERTED = metricRegistry.counter(MetricRegistry.name("feast_ingestion_feature_row_ingested_count")) + val METRIC_TOTAL_ROWS_INSERTED = + metricRegistry.counter(MetricRegistry.name("feast_ingestion_feature_row_ingested_count")) - val METRIC_ROWS_LAG = metricRegistry.histogram(MetricRegistry.name("feast_ingestion_feature_row_lag_ms")) + val METRIC_ROWS_LAG = + metricRegistry.histogram(MetricRegistry.name("feast_ingestion_feature_row_lag_ms")) } diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala index 5125c2fd55..0259306977 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala @@ -1,3 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import java.nio.file.{Files, Paths} @@ -24,7 +40,6 @@ import scala.util.hashing.MurmurHash3 case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) - class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) @@ -49,12 +64,13 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { def generateTempPath(last: String) = Paths.get(Files.createTempDirectory("test-dir").toString, last).toString - def storeAsParquet[T <: Product : TypeTag](rows: Seq[T]): String = { + def storeAsParquet[T <: Product: TypeTag](rows: Seq[T]): String = { import self.sparkSession.implicits._ val tempPath = generateTempPath("rows") - sparkSession.createDataset(rows) + sparkSession + .createDataset(rows) .withColumn("date", to_date($"eventTimestamp")) .write .partitionBy("date") @@ -68,14 +84,15 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val jedis = new Jedis("localhost", container.mappedPort(6379)) jedis.flushAll() - def rowGenerator(start: DateTime, end: DateTime, customerGen: Option[Gen[String]] = None) = for { - customer <- customerGen.getOrElse(Gen.asciiPrintableStr) - feature1 <- Gen.choose(0, 100) - feature2 <- Gen.choose[Float](0, 1) - eventTimestamp <- Gen.choose(0, Seconds.secondsBetween(start, end).getSeconds) - .map(start.withMillisOfSecond(0).plusSeconds) - } - yield Row(customer, feature1, feature2, new java.sql.Timestamp(eventTimestamp.getMillis)) + def rowGenerator(start: DateTime, end: DateTime, customerGen: Option[Gen[String]] = None) = + for { + customer <- customerGen.getOrElse(Gen.asciiPrintableStr) + feature1 <- Gen.choose(0, 100) + feature2 <- Gen.choose[Float](0, 1) + eventTimestamp <- Gen + .choose(0, Seconds.secondsBetween(start, end).getSeconds) + .map(start.withMillisOfSecond(0).plusSeconds) + } yield Row(customer, feature1, feature2, new java.sql.Timestamp(eventTimestamp.getMillis)) def generateDistinctRows(gen: Gen[Row], N: Int) = Gen.listOfN(N, gen).sample.get.groupBy(_.customer).map(_._2.head).toSeq @@ -87,14 +104,14 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { entities = Seq(Field("customer", ValueType.Enum.STRING)), features = Seq( Field("feature1", ValueType.Enum.INT32), - Field("feature2", ValueType.Enum.FLOAT), - ), + Field("feature2", ValueType.Enum.FLOAT) + ) ), startTime = DateTime.parse("2020-08-01"), endTime = DateTime.parse("2020-09-01") ) - def encodeEntityKey(row: Row, featureTable: FeatureTable) = { + def encodeEntityKey(row: Row, featureTable: FeatureTable): Array[Byte] = { val entityPrefix = featureTable.entities.map(_.name).mkString("_") s"${featureTable.project}_${entityPrefix}:${row.customer}".getBytes } @@ -109,11 +126,12 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { m compose { (_: Map[Array[Byte], Array[Byte]]) - .map { - case (k, v) => ( + .map { case (k, v) => + ( new String(k), if (new String(k).startsWith("_ts")) - Timestamp.parseFrom(v).asScala else + Timestamp.parseFrom(v).asScala + else ValueProto.Value.parseFrom(v).asScala ) } @@ -122,115 +140,144 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { } "Parquet source file" should "be ingested in redis" in new Scope { - val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) - val rows = generateDistinctRows(gen, 10000) + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val rows = generateDistinctRows(gen, 10000) val tempPath = storeAsParquet(rows) - val configWithOfflineSource = config.copy(featureTable = config.featureTable.copy(offline_source = - Some(GSSource(tempPath, Map.empty, "eventTimestamp")))) + val configWithOfflineSource = config.copy(featureTable = + config.featureTable.copy(offline_source = + Some(GSSource(tempPath, Map.empty, "eventTimestamp")) + ) + ) OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) - rows.foreach( - r => { - val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap - storedValues should beStoredRow(Map( + rows.foreach(r => { + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow( + Map( featureKeyEncoder("feature1") -> r.feature1, featureKeyEncoder("feature2") -> r.feature2, - "_ts:test-fs" -> r.eventTimestamp - )) - } - ) + "_ts:test-fs" -> r.eventTimestamp + ) + ) + }) } "Ingested rows" should "be compacted before storing by timestamp column" in new Scope { val entities = (0 to 10000).map(_.toString) - val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) + val genLatest = rowGenerator( + DateTime.parse("2020-08-15"), + DateTime.parse("2020-09-01"), + Some(Gen.oneOf(entities)) + ) val latest = generateDistinctRows(genLatest, 10000) - val genOld = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-08-14"), Some(Gen.oneOf(entities))) + val genOld = rowGenerator( + DateTime.parse("2020-08-01"), + DateTime.parse("2020-08-14"), + Some(Gen.oneOf(entities)) + ) val old = generateDistinctRows(genOld, 10000) val tempPath = storeAsParquet(latest ++ old) - val configWithOfflineSource = config.copy(featureTable = config.featureTable.copy(offline_source = - Some(GSSource(tempPath, Map.empty, "eventTimestamp")))) + val configWithOfflineSource = config.copy(featureTable = + config.featureTable.copy(offline_source = + Some(GSSource(tempPath, Map.empty, "eventTimestamp")) + ) + ) OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) - latest.foreach( - r => { - val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap - storedValues should beStoredRow(Map( + latest.foreach(r => { + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow( + Map( featureKeyEncoder("feature1") -> r.feature1, featureKeyEncoder("feature2") -> r.feature2, - "_ts:test-fs" -> r.eventTimestamp - )) - } - ) + "_ts:test-fs" -> r.eventTimestamp + ) + ) + }) } "Old rows in ingestion" should "not overwrite more recent rows from storage" in new Scope { val entities = (0 to 10000).map(_.toString) - val genLatest = rowGenerator(DateTime.parse("2020-08-15"), DateTime.parse("2020-09-01"), Some(Gen.oneOf(entities))) + val genLatest = rowGenerator( + DateTime.parse("2020-08-15"), + DateTime.parse("2020-09-01"), + Some(Gen.oneOf(entities)) + ) val latest = generateDistinctRows(genLatest, 10000) val tempPath1 = storeAsParquet(latest) - val config1 = config.copy(featureTable = config.featureTable.copy(offline_source = - Some(GSSource(tempPath1, Map.empty, "eventTimestamp")))) + val config1 = config.copy(featureTable = + config.featureTable.copy(offline_source = + Some(GSSource(tempPath1, Map.empty, "eventTimestamp")) + ) + ) OfflinePipeline.createPipeline(sparkSession, config1) - val genOld = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-08-14"), Some(Gen.oneOf(entities))) + val genOld = rowGenerator( + DateTime.parse("2020-08-01"), + DateTime.parse("2020-08-14"), + Some(Gen.oneOf(entities)) + ) val old = generateDistinctRows(genOld, 10000) val tempPath2 = storeAsParquet(old) - val config2 = config.copy(featureTable = config.featureTable.copy(offline_source = - Some(GSSource(tempPath2, Map.empty, "eventTimestamp")))) + val config2 = config.copy(featureTable = + config.featureTable.copy(offline_source = + Some(GSSource(tempPath2, Map.empty, "eventTimestamp")) + ) + ) OfflinePipeline.createPipeline(sparkSession, config2) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) - latest.foreach( - r => { - val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap - storedValues should beStoredRow(Map( + latest.foreach(r => { + val storedValues = jedis.hgetAll(encodeEntityKey(r, config.featureTable)).asScala.toMap + storedValues should beStoredRow( + Map( featureKeyEncoder("feature1") -> r.feature1, featureKeyEncoder("feature2") -> r.feature2, - "_ts:test-fs" -> r.eventTimestamp - )) - } - ) + "_ts:test-fs" -> r.eventTimestamp + ) + ) + }) } "Invalid rows" should "not be ingested and stored to deadletter instead" in new Scope { - val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) val rows = generateDistinctRows(gen, 100) val rowsWithNullEntity = rows.map(_.copy(customer = null)) val tempPath = storeAsParquet(rowsWithNullEntity) val deadletterConfig = config.copy( - featureTable = config.featureTable.copy(offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp"))), + featureTable = config.featureTable + .copy(offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp"))), deadLetterPath = Some(generateTempPath("deadletters")) ) OfflinePipeline.createPipeline(sparkSession, deadletterConfig) - jedis.keys("*").toArray should be (empty) + jedis.keys("*").toArray should be(empty) sparkSession.read .parquet(deadletterConfig.deadLetterPath.get) - .count() should be (rows.length) + .count() should be(rows.length) } "Columns from source" should "be mapped according to configuration" in new Scope { - val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) + val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) val rows = generateDistinctRows(gen, 100) val tempPath = storeAsParquet(rows) @@ -242,29 +289,34 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { Field("new_feature1", ValueType.Enum.INT32), Field("new_feature2", ValueType.Enum.FLOAT) ), - offline_source = Some(GSSource( - tempPath, - Map( - "entity" -> "customer", - "new_feature1" -> "feature1", - "new_feature2" -> "feature2" - ), - "eventTimestamp"))) + offline_source = Some( + GSSource( + tempPath, + Map( + "entity" -> "customer", + "new_feature1" -> "feature1", + "new_feature2" -> "feature2" + ), + "eventTimestamp" + ) + ) + ) ) OfflinePipeline.createPipeline(sparkSession, configWithMapping) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) - rows.foreach( - r => { - val storedValues = jedis.hgetAll(encodeEntityKey(r, configWithMapping.featureTable)).asScala.toMap - storedValues should beStoredRow(Map( + rows.foreach(r => { + val storedValues = + jedis.hgetAll(encodeEntityKey(r, configWithMapping.featureTable)).asScala.toMap + storedValues should beStoredRow( + Map( featureKeyEncoder("new_feature1") -> r.feature1, featureKeyEncoder("new_feature2") -> r.feature2, - "_ts:test-fs" -> r.eventTimestamp - )) - } - ) + "_ts:test-fs" -> r.eventTimestamp + ) + ) + }) } } diff --git a/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala b/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala index 3fa5320562..25b53cb57c 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala +++ b/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala @@ -1,8 +1,28 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion import org.scalatest._ import matchers._ import org.scalatest.flatspec.AnyFlatSpec -abstract class UnitSpec extends AnyFlatSpec with should.Matchers with - OptionValues with Inside with Inspectors +abstract class UnitSpec + extends AnyFlatSpec + with should.Matchers + with OptionValues + with Inside + with Inspectors diff --git a/pom.xml b/pom.xml index 61dff77a37..882a7c65bb 100644 --- a/pom.xml +++ b/pom.xml @@ -86,6 +86,27 @@ 6.1.2.Final 1.6.6 + + + ${maven.multiModuleProjectDirectory} + false feast.common.auth.providers.http.client @@ -529,26 +550,7 @@ - - - + ${license.content} 1.7 @@ -559,6 +561,15 @@ + + + ${license.content} + + + 2.7.2 + ${parent.basedir}/.scalafmt.conf + + From caae910ba4ecf9b139ef617cbcf21bf686e9519d Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Tue, 29 Sep 2020 15:26:34 +0800 Subject: [PATCH 09/15] integration test stage Signed-off-by: Oleksii Moskalenko --- ingestion-spark/pom.xml | 5 +++-- .../feast/ingestion/stores/redis/RedisSinkRelation.scala | 2 +- .../apache/spark/metrics/source/RedisSinkMetricSource.scala | 6 +++++- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/ingestion-spark/pom.xml b/ingestion-spark/pom.xml index b786660417..7fff70225e 100644 --- a/ingestion-spark/pom.xml +++ b/ingestion-spark/pom.xml @@ -214,9 +214,10 @@ - integration-test + test + integration-test - integration-test + test diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index 158d9dfca3..0983528ea5 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -116,7 +116,7 @@ class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisC } private lazy val metricSource: Option[RedisSinkMetricSource] = - SparkEnv.get.metricsSystem.getSourcesByName("redis_sink") match { + SparkEnv.get.metricsSystem.getSourcesByName(RedisSinkMetricSource.sourceName) match { case Seq(head) => Some(head.asInstanceOf[RedisSinkMetricSource]) case _ => None } diff --git a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala index 460ab8c978..1eea1c634f 100644 --- a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala +++ b/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry class RedisSinkMetricSource extends Source { - override val sourceName: String = "redis_sink" + override val sourceName: String = RedisSinkMetricSource.sourceName override val metricRegistry: MetricRegistry = new MetricRegistry @@ -29,3 +29,7 @@ class RedisSinkMetricSource extends Source { val METRIC_ROWS_LAG = metricRegistry.histogram(MetricRegistry.name("feast_ingestion_feature_row_lag_ms")) } + +object RedisSinkMetricSource { + val sourceName = "redis_sink" +} From c41c7b8987da2420556ad67bde322eedccf7571d Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Tue, 29 Sep 2020 15:30:26 +0800 Subject: [PATCH 10/15] remove version from ingestion-spark pom Signed-off-by: Oleksii Moskalenko --- ingestion-spark/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ingestion-spark/pom.xml b/ingestion-spark/pom.xml index 7fff70225e..13c8fd113c 100644 --- a/ingestion-spark/pom.xml +++ b/ingestion-spark/pom.xml @@ -28,8 +28,7 @@ Feast Spark Ingestion feast-ingestion-spark - 0.7-SNAPSHOT - + 2.12 ${scala.version}.12 From 3e4d04a961d35b67caceec1e7eebef1f15820fbc Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Fri, 2 Oct 2020 12:49:02 +0800 Subject: [PATCH 11/15] refactor job options Signed-off-by: Oleksii Moskalenko --- .../feast/ingestion/IngestionJobConfig.scala | 43 ---- pom.xml | 2 +- {ingestion-spark => spark/ingestion}/pom.xml | 9 +- .../scala/feast/ingestion/BasePipeline.scala | 17 +- .../scala/feast/ingestion/BatchPipeline.scala | 14 +- .../scala/feast/ingestion/IngestionJob.scala | 54 ++--- .../feast/ingestion/IngestionJobConfig.scala | 49 ++++- .../metrics/StatsdReporterWithTags.scala | 201 ++++++++++++++++++ .../ingestion/sources/bq/BigQueryReader.scala | 0 .../ingestion/sources/file/FileReader.scala | 4 +- .../stores/redis/DefaultSource.scala | 0 .../stores/redis/HashTypePersistence.scala | 0 .../stores/redis/RedisSinkRelation.scala | 0 .../stores/redis/SparkRedisConfig.scala | 0 .../ingestion/utils/TypeConversion.scala | 0 .../ingestion/validation/RowValidator.scala | 0 .../metrics/sink/StatsdSinkWithTags.scala | 60 ++++++ .../source/RedisSinkMetricSource.scala | 16 +- .../feast/ingestion/BatchPipelineIT.scala | 62 ++---- .../test/scala/feast/ingestion/UnitSpec.scala | 0 .../ingestion/metrics/StatsReporterSpec.scala | 107 ++++++++++ 21 files changed, 508 insertions(+), 130 deletions(-) delete mode 100644 ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala rename {ingestion-spark => spark/ingestion}/pom.xml (97%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/BasePipeline.scala (77%) rename ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala => spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala (86%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/IngestionJob.scala (58%) rename ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala => spark/ingestion/src/main/scala/feast/ingestion/IngestionJobConfig.scala (56%) create mode 100644 spark/ingestion/src/main/scala/feast/ingestion/metrics/StatsdReporterWithTags.scala rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/sources/file/FileReader.scala (95%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/utils/TypeConversion.scala (100%) rename {ingestion-spark => spark/ingestion}/src/main/scala/feast/ingestion/validation/RowValidator.scala (100%) create mode 100644 spark/ingestion/src/main/scala/org/apache/spark/metrics/sink/StatsdSinkWithTags.scala rename {ingestion-spark => spark/ingestion}/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala (65%) rename ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala => spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala (85%) rename {ingestion-spark => spark/ingestion}/src/test/scala/feast/ingestion/UnitSpec.scala (100%) create mode 100644 spark/ingestion/src/test/scala/feast/ingestion/metrics/StatsReporterSpec.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala b/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala deleted file mode 100644 index fc5f3d304b..0000000000 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJobConfig.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * Copyright 2018-2020 The Feast Authors - * - * Licensed 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 - * - * https://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 feast.ingestion - -import feast.ingestion.Modes.Modes -import org.joda.time.DateTime - -object Modes extends Enumeration { - type Modes = Value - val Offline, Online = Value -} - -abstract class StoreConfig - -case class RedisConfig(connection: String) extends StoreConfig - -abstract class MetricConfig - -case class StatsDConfig(host: String, port: Int) extends MetricConfig - -case class IngestionJobConfig( - mode: Modes = Modes.Offline, - featureTable: FeatureTable = null, - startTime: DateTime = DateTime.now(), - endTime: DateTime = DateTime.now(), - store: StoreConfig = RedisConfig("localhost:6379"), - metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)), - deadLetterPath: Option[String] = None -) diff --git a/pom.xml b/pom.xml index 882a7c65bb..0bcb577671 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ common job-controller common-test - ingestion-spark + spark/ingestion diff --git a/ingestion-spark/pom.xml b/spark/ingestion/pom.xml similarity index 97% rename from ingestion-spark/pom.xml rename to spark/ingestion/pom.xml index 13c8fd113c..e734cb1137 100644 --- a/ingestion-spark/pom.xml +++ b/spark/ingestion/pom.xml @@ -24,11 +24,12 @@ dev.feast feast-parent ${revision} + ../.. Feast Spark Ingestion feast-ingestion-spark - + 2.12 ${scala.version}.12 @@ -151,6 +152,12 @@ 4.1.52.Final + + org.json4s + json4s-ext_${scala.version} + 3.7.0-M6 + + org.scalatest scalatest_${scala.version} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala b/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala similarity index 77% rename from ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala rename to spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala index 7cd0c138f9..a9a7a6223c 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/BasePipeline.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala @@ -30,6 +30,13 @@ trait BasePipeline { .setAppName(s"${jobConfig.mode} IngestionJob for ${jobConfig.featureTable.name}") .setMaster("local") + jobConfig.store match { + case RedisConfig(host, port) => + conf + .set("spark.redis.host", host) + .set("spark.redis.port", port.toString) + } + jobConfig.metrics match { case Some(c: StatsDConfig) => conf @@ -37,11 +44,19 @@ trait BasePipeline { "spark.metrics.conf.*.source.redis.class", "org.apache.spark.metrics.source.RedisSinkMetricSource" ) - .set("spark.metrics.conf.*.sink.statsd.class", "org.apache.spark.metrics.sink.StatsdSink") + .set( + "spark.metrics.conf.*.source.redis.labels", + s"feature_table=${jobConfig.featureTable.name}" + ) + .set( + "spark.metrics.conf.*.sink.statsd.class", + "org.apache.spark.metrics.sink.StatsdSinkWithTags" + ) .set("spark.metrics.conf.*.sink.statsd.host", c.host) .set("spark.metrics.conf.*.sink.statsd.port", c.port.toString) .set("spark.metrics.conf.*.sink.statsd.period", "1") .set("spark.metrics.conf.*.sink.statsd.unit", "seconds") + .set("spark.metrics.namespace", jobConfig.mode.toString) } SparkSession diff --git a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala b/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala similarity index 86% rename from ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala rename to spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala index 5589b2fbfe..ecedda315c 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/OfflinePipeline.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala @@ -22,22 +22,22 @@ import feast.ingestion.validation.RowValidator import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.functions.col -object OfflinePipeline extends BasePipeline { +object BatchPipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { val featureTable = config.featureTable val projection = - inputProjection(featureTable.offline_source.get, featureTable.features, featureTable.entities) + inputProjection(config.source, featureTable.features, featureTable.entities) val validator = new RowValidator(featureTable) - val input = config.featureTable.offline_source match { - case Some(source: BQSource) => + val input = config.source match { + case source: BQSource => BigQueryReader.createBatchSource( sparkSession.sqlContext, source, config.startTime, config.endTime ) - case Some(source: GSSource) => + case source: FileSource => FileReader.createBatchSource( sparkSession.sqlContext, source, @@ -56,7 +56,7 @@ object OfflinePipeline extends BasePipeline { .option("entity_columns", featureTable.entities.map(_.name).mkString(",")) .option("namespace", featureTable.name) .option("project_name", featureTable.project) - .option("timestamp_column", featureTable.offline_source.get.timestampColumn) + .option("timestamp_column", config.source.timestampColumn) .save() config.deadLetterPath match { @@ -72,7 +72,7 @@ object OfflinePipeline extends BasePipeline { } private def inputProjection( - source: OfflineSource, + source: Source, features: Seq[Field], entities: Seq[Field] ): Array[Column] = { diff --git a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala similarity index 58% rename from ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala rename to spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala index 187852f537..6d06dd376d 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/IngestionJob.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala @@ -16,12 +16,17 @@ */ package feast.ingestion -import feast.proto.types.ValueProto.ValueType import org.joda.time.DateTime +import org.json4s._ +import org.json4s.jackson.JsonMethods.{parse => parseJSON} +import org.json4s.ext.JavaEnumNameSerializer + object IngestionJob { import Modes._ implicit val modesRead: scopt.Read[Modes.Value] = scopt.Read.reads(Modes withName _.capitalize) + implicit val formats: Formats = DefaultFormats + + new JavaEnumNameSerializer[feast.proto.types.ValueProto.ValueType.Enum]() val parser = new scopt.OptionParser[IngestionJobConfig]("IngestionJon") { head("feast.ingestion.IngestionJob", "0.8") @@ -31,31 +36,20 @@ object IngestionJob { .required() .text("Mode to operate ingestion job (offline or online)") - opt[String](name = "feature-table-spec") + opt[String](name = "source") .action((x, c) => - c.copy(featureTable = - FeatureTable( - name = "jaeger-car", - project = "default", - entities = Seq(Field(name = "customer", `type` = ValueType.Enum.STRING)), - features = Seq( - Field("car_customer_id_avg_customer_distance_cancelled", ValueType.Enum.FLOAT), - Field("car_customer_id_num_completed", ValueType.Enum.FLOAT), - Field("car_customer_id_origin_completed_1", ValueType.Enum.INT32) - ), - offline_source = Some( - BQSource( - "gods-staging", - "feast", - "default_jaeger_car_customer", - Map.empty, - "event_timestamp" - ) - ) - ) - ) + parseJSON(x).extract[Sources] match { + case Sources(file: Some[FileSource], _, _) => c.copy(source = file.get) + case Sources(_, bq: Some[BQSource], _) => c.copy(source = bq.get) + case Sources(_, _, kafka: Some[KafkaSource]) => c.copy(source = kafka.get) + } ) .required() + .text("JSON-encoded source object (e.g. {\"kafka\":{\"bootstrapServers\":...}}") + + opt[String](name = "feature-table") + .action((x, c) => c.copy(featureTable = parseJSON(x).extract[FeatureTable])) + .required() .text("JSON-encoded FeatureTableSpec object") opt[String](name = "start") @@ -65,6 +59,16 @@ object IngestionJob { opt[String](name = "end") .action((x, c) => c.copy(endTime = DateTime.parse(x))) .text("End timestamp for offline ingestion") + + opt[String](name = "redis") + .action((x, c) => c.copy(store = parseJSON(x).extract[RedisConfig])) + + opt[String](name = "statsd") + .action((x, c) => c.copy(metrics = Some(parseJSON(x).extract[StatsDConfig]))) + + opt[String](name = "deadletter-path") + .action((x, c) => c.copy(deadLetterPath = Some(x))) + } def main(args: Array[String]): Unit = { @@ -72,8 +76,8 @@ object IngestionJob { case Some(config) => config.mode match { case Modes.Offline => - val sparkSession = OfflinePipeline.createSparkSession(config) - OfflinePipeline.createPipeline(sparkSession, config) + val sparkSession = BatchPipeline.createSparkSession(config) + BatchPipeline.createPipeline(sparkSession, config) } case None => println("Parameters can't be parsed") diff --git a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJobConfig.scala similarity index 56% rename from ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala rename to spark/ingestion/src/main/scala/feast/ingestion/IngestionJobConfig.scala index 432e5a7f89..880f00f37f 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/FeatureTable.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJobConfig.scala @@ -16,21 +16,37 @@ */ package feast.ingestion +import feast.ingestion.Modes.Modes +import org.joda.time.DateTime + +object Modes extends Enumeration { + type Modes = Value + val Offline, Online = Value +} + +abstract class StoreConfig + +case class RedisConfig(host: String, port: Int) extends StoreConfig + +abstract class MetricConfig + +case class StatsDConfig(host: String, port: Int) extends MetricConfig + abstract class Source { def mapping: Map[String, String] def timestampColumn: String } -abstract class OfflineSource extends Source +abstract class BatchSource extends Source -abstract class OnlineSource extends Source +abstract class StreamingSource extends Source -case class GSSource( +case class FileSource( path: String, override val mapping: Map[String, String], override val timestampColumn: String -) extends OfflineSource +) extends BatchSource case class BQSource( project: String, @@ -38,14 +54,20 @@ case class BQSource( table: String, override val mapping: Map[String, String], override val timestampColumn: String -) extends OfflineSource +) extends BatchSource case class KafkaSource( bootstrapServers: String, topic: String, override val mapping: Map[String, String], override val timestampColumn: String -) extends OnlineSource +) extends StreamingSource + +case class Sources( + file: Option[FileSource] = None, + bq: Option[BQSource] = None, + kafka: Option[KafkaSource] = None +) case class Field(name: String, `type`: feast.proto.types.ValueProto.ValueType.Enum) @@ -53,7 +75,16 @@ case class FeatureTable( name: String, project: String, entities: Seq[Field], - features: Seq[Field], - offline_source: Option[OfflineSource] = None, - online_source: Option[OnlineSource] = None + features: Seq[Field] +) + +case class IngestionJobConfig( + mode: Modes = Modes.Offline, + featureTable: FeatureTable = null, + source: Source = null, + startTime: DateTime = DateTime.now(), + endTime: DateTime = DateTime.now(), + store: StoreConfig = RedisConfig("localhost", 6379), + metrics: Option[MetricConfig] = Some(StatsDConfig("localhost", 9125)), + deadLetterPath: Option[String] = None ) diff --git a/spark/ingestion/src/main/scala/feast/ingestion/metrics/StatsdReporterWithTags.scala b/spark/ingestion/src/main/scala/feast/ingestion/metrics/StatsdReporterWithTags.scala new file mode 100644 index 0000000000..66b48dd444 --- /dev/null +++ b/spark/ingestion/src/main/scala/feast/ingestion/metrics/StatsdReporterWithTags.scala @@ -0,0 +1,201 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.metrics + +import java.io.IOException +import java.net.{DatagramPacket, DatagramSocket, InetSocketAddress} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.SortedMap +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +import com.codahale.metrics._ +import org.apache.hadoop.net.NetUtils + +import org.apache.spark.internal.Logging + +/** + * @see + * StatsD metric types + */ +object StatsdMetricType { + val COUNTER = "c" + val GAUGE = "g" + val TIMER = "ms" + val Set = "s" +} + +/** + * This is fork of internal spark implementation of StatsdReporter. + * The main difference between this implementation & spark is support of tags inside metric names. + * + * When we extend metric name (by adding some suffix) we must keep tags part in the end of the name. + * Labels are separated from name by # (see tagging extensions of prometheus exporter + * https://github.com/prometheus/statsd_exporter#tagging-extensions) + * E.g., + * + * metric_name#tag=value + count -> metric_name.count#tag=value + */ +class StatsdReporterWithTags( + registry: MetricRegistry, + host: String = "127.0.0.1", + port: Int = 8125, + prefix: String = "", + filter: MetricFilter = MetricFilter.ALL, + rateUnit: TimeUnit = TimeUnit.SECONDS, + durationUnit: TimeUnit = TimeUnit.MILLISECONDS +) extends ScheduledReporter(registry, "statsd-reporter", filter, rateUnit, durationUnit) + with Logging { + + import StatsdMetricType._ + + private val address = new InetSocketAddress(host, port) + private val whitespace = "[\\s]+".r + + override def report( + gauges: SortedMap[String, Gauge[_]], + counters: SortedMap[String, Counter], + histograms: SortedMap[String, Histogram], + meters: SortedMap[String, Meter], + timers: SortedMap[String, Timer] + ): Unit = + Try(new DatagramSocket) match { + case Failure(ioe: IOException) => + logWarning( + "StatsD datagram socket construction failed", + NetUtils.wrapException(host, port, NetUtils.getHostname(), 0, ioe) + ) + case Failure(e) => logWarning("StatsD datagram socket construction failed", e) + case Success(s) => + implicit val socket = s + val localAddress = Try(socket.getLocalAddress).map(_.getHostAddress).getOrElse(null) + val localPort = socket.getLocalPort + Try { + gauges.entrySet.asScala.foreach(e => reportGauge(e.getKey, e.getValue)) + counters.entrySet.asScala.foreach(e => reportCounter(e.getKey, e.getValue)) + histograms.entrySet.asScala.foreach(e => reportHistogram(e.getKey, e.getValue)) + meters.entrySet.asScala.foreach(e => reportMetered(e.getKey, e.getValue)) + timers.entrySet.asScala.foreach(e => reportTimer(e.getKey, e.getValue)) + } recover { + case ioe: IOException => + logDebug( + s"Unable to send packets to StatsD", + NetUtils.wrapException( + address.getHostString, + address.getPort, + localAddress, + localPort, + ioe + ) + ) + case e: Throwable => logDebug(s"Unable to send packets to StatsD at '$host:$port'", e) + } + Try(socket.close()) recover { + case ioe: IOException => + logDebug( + "Error when close socket to StatsD", + NetUtils.wrapException( + address.getHostString, + address.getPort, + localAddress, + localPort, + ioe + ) + ) + case e: Throwable => logDebug("Error when close socket to StatsD", e) + } + } + + private def reportGauge(name: String, gauge: Gauge[_])(implicit socket: DatagramSocket): Unit = + formatAny(gauge.getValue).foreach(v => send(fullName(name), v, GAUGE)) + + private def reportCounter(name: String, counter: Counter)(implicit socket: DatagramSocket): Unit = + send(fullName(name), format(counter.getCount), COUNTER) + + private def reportHistogram(name: String, histogram: Histogram)(implicit + socket: DatagramSocket + ): Unit = { + val snapshot = histogram.getSnapshot + send(fullName(name, "count"), format(histogram.getCount), GAUGE) + send(fullName(name, "max"), format(snapshot.getMax), TIMER) + send(fullName(name, "mean"), format(snapshot.getMean), TIMER) + send(fullName(name, "min"), format(snapshot.getMin), TIMER) + send(fullName(name, "stddev"), format(snapshot.getStdDev), TIMER) + send(fullName(name, "p50"), format(snapshot.getMedian), TIMER) + send(fullName(name, "p75"), format(snapshot.get75thPercentile), TIMER) + send(fullName(name, "p95"), format(snapshot.get95thPercentile), TIMER) + send(fullName(name, "p98"), format(snapshot.get98thPercentile), TIMER) + send(fullName(name, "p99"), format(snapshot.get99thPercentile), TIMER) + send(fullName(name, "p999"), format(snapshot.get999thPercentile), TIMER) + } + + private def reportMetered(name: String, meter: Metered)(implicit socket: DatagramSocket): Unit = { + send(fullName(name, "count"), format(meter.getCount), GAUGE) + send(fullName(name, "m1_rate"), format(convertRate(meter.getOneMinuteRate)), TIMER) + send(fullName(name, "m5_rate"), format(convertRate(meter.getFiveMinuteRate)), TIMER) + send(fullName(name, "m15_rate"), format(convertRate(meter.getFifteenMinuteRate)), TIMER) + send(fullName(name, "mean_rate"), format(convertRate(meter.getMeanRate)), TIMER) + } + + private def reportTimer(name: String, timer: Timer)(implicit socket: DatagramSocket): Unit = { + val snapshot = timer.getSnapshot + send(fullName(name, "max"), format(convertDuration(snapshot.getMax)), TIMER) + send(fullName(name, "mean"), format(convertDuration(snapshot.getMean)), TIMER) + send(fullName(name, "min"), format(convertDuration(snapshot.getMin)), TIMER) + send(fullName(name, "stddev"), format(convertDuration(snapshot.getStdDev)), TIMER) + send(fullName(name, "p50"), format(convertDuration(snapshot.getMedian)), TIMER) + send(fullName(name, "p75"), format(convertDuration(snapshot.get75thPercentile)), TIMER) + send(fullName(name, "p95"), format(convertDuration(snapshot.get95thPercentile)), TIMER) + send(fullName(name, "p98"), format(convertDuration(snapshot.get98thPercentile)), TIMER) + send(fullName(name, "p99"), format(convertDuration(snapshot.get99thPercentile)), TIMER) + send(fullName(name, "p999"), format(convertDuration(snapshot.get999thPercentile)), TIMER) + + reportMetered(name, timer) + } + + private def send(name: String, value: String, metricType: String)(implicit + socket: DatagramSocket + ): Unit = { + val bytes = sanitize(s"$name:$value|$metricType").getBytes(UTF_8) + val packet = new DatagramPacket(bytes, bytes.length, address) + socket.send(packet) + } + + private val nameWithTag = """(\S+)#(\S+)""".r + + private def fullName(name: String, suffixes: String*): String = name match { + case nameWithTag(name, tags) => + MetricRegistry.name(prefix, name +: suffixes: _*) ++ "#" ++ tags + case _ => + MetricRegistry.name(prefix, name +: suffixes: _*) + } + + private def sanitize(s: String): String = whitespace.replaceAllIn(s, "-") + + private def format(v: Any): String = formatAny(v).getOrElse("") + + private def formatAny(v: Any): Option[String] = + v match { + case f: Float => Some("%2.2f".format(f)) + case d: Double => Some("%2.2f".format(d)) + case b: BigDecimal => Some("%2.2f".format(b)) + case n: Number => Some(n.toString) + case _ => None + } +} diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala b/spark/ingestion/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala rename to spark/ingestion/src/main/scala/feast/ingestion/sources/bq/BigQueryReader.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala b/spark/ingestion/src/main/scala/feast/ingestion/sources/file/FileReader.scala similarity index 95% rename from ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala rename to spark/ingestion/src/main/scala/feast/ingestion/sources/file/FileReader.scala index 490b5e542b..e099a87aa3 100644 --- a/ingestion-spark/src/main/scala/feast/ingestion/sources/file/FileReader.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/sources/file/FileReader.scala @@ -18,7 +18,7 @@ package feast.ingestion.sources.file import java.sql.Timestamp -import feast.ingestion.GSSource +import feast.ingestion.FileSource import org.apache.spark.sql.functions.col import org.apache.spark.sql.{DataFrame, SQLContext} import org.joda.time.DateTime @@ -26,7 +26,7 @@ import org.joda.time.DateTime object FileReader { def createBatchSource( sqlContext: SQLContext, - source: GSSource, + source: FileSource, start: DateTime, end: DateTime ): DataFrame = { diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala rename to spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala rename to spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala rename to spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala rename to spark/ingestion/src/main/scala/feast/ingestion/stores/redis/SparkRedisConfig.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala b/spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/utils/TypeConversion.scala rename to spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala diff --git a/ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala b/spark/ingestion/src/main/scala/feast/ingestion/validation/RowValidator.scala similarity index 100% rename from ingestion-spark/src/main/scala/feast/ingestion/validation/RowValidator.scala rename to spark/ingestion/src/main/scala/feast/ingestion/validation/RowValidator.scala diff --git a/spark/ingestion/src/main/scala/org/apache/spark/metrics/sink/StatsdSinkWithTags.scala b/spark/ingestion/src/main/scala/org/apache/spark/metrics/sink/StatsdSinkWithTags.scala new file mode 100644 index 0000000000..5c70da2b86 --- /dev/null +++ b/spark/ingestion/src/main/scala/org/apache/spark/metrics/sink/StatsdSinkWithTags.scala @@ -0,0 +1,60 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.MetricRegistry +import feast.ingestion.metrics.StatsdReporterWithTags +import org.apache.spark.SecurityManager +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.MetricsSystem + +private[spark] class StatsdSinkWithTags( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager +) extends Sink + with Logging { + import StatsdSink._ + + val host = property.getProperty(STATSD_KEY_HOST, STATSD_DEFAULT_HOST) + val port = property.getProperty(STATSD_KEY_PORT, STATSD_DEFAULT_PORT).toInt + + val pollPeriod = property.getProperty(STATSD_KEY_PERIOD, STATSD_DEFAULT_PERIOD).toInt + val pollUnit = + TimeUnit.valueOf(property.getProperty(STATSD_KEY_UNIT, STATSD_DEFAULT_UNIT).toUpperCase) + + val prefix = property.getProperty(STATSD_KEY_PREFIX, STATSD_DEFAULT_PREFIX) + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter = new StatsdReporterWithTags(registry, host, port, prefix) + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + logInfo(s"StatsdSink started with prefix: '$prefix'") + } + + override def stop(): Unit = { + reporter.stop() + logInfo("StatsdSink stopped.") + } + + override def report(): Unit = reporter.report() +} diff --git a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala b/spark/ingestion/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala similarity index 65% rename from ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala rename to spark/ingestion/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala index 1eea1c634f..77c9218a7e 100644 --- a/ingestion-spark/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala +++ b/spark/ingestion/src/main/scala/org/apache/spark/metrics/source/RedisSinkMetricSource.scala @@ -17,17 +17,29 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry +import org.apache.spark.{SparkConf, SparkEnv} class RedisSinkMetricSource extends Source { override val sourceName: String = RedisSinkMetricSource.sourceName override val metricRegistry: MetricRegistry = new MetricRegistry + private val sparkConfig = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf(true)) + + private val metricLabels = sparkConfig.get("spark.metrics.conf.*.source.redis.labels") + + private def nameWithLabels(name: String) = + if (metricLabels.isEmpty) { + name + } else { + s"$name#$metricLabels" + } + val METRIC_TOTAL_ROWS_INSERTED = - metricRegistry.counter(MetricRegistry.name("feast_ingestion_feature_row_ingested_count")) + metricRegistry.counter(nameWithLabels("feast_ingestion_feature_row_ingested_count")) val METRIC_ROWS_LAG = - metricRegistry.histogram(MetricRegistry.name("feast_ingestion_feature_row_lag_ms")) + metricRegistry.histogram(nameWithLabels("feast_ingestion_feature_row_lag_ms")) } object RedisSinkMetricSource { diff --git a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala b/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala similarity index 85% rename from ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala rename to spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala index 0259306977..1fef30d856 100644 --- a/ingestion-spark/src/test/scala/feast/ingestion/OfflinePipelineIT.scala +++ b/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala @@ -40,7 +40,7 @@ import scala.util.hashing.MurmurHash3 case class Row(customer: String, feature1: Int, feature2: Float, eventTimestamp: java.sql.Timestamp) -class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { +class BatchPipelineIT extends UnitSpec with ForAllTestContainer { override val container = GenericContainer("redis:6.0.8", exposedPorts = Seq(6379)) @@ -143,13 +143,11 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val gen = rowGenerator(DateTime.parse("2020-08-01"), DateTime.parse("2020-09-01")) val rows = generateDistinctRows(gen, 10000) val tempPath = storeAsParquet(rows) - val configWithOfflineSource = config.copy(featureTable = - config.featureTable.copy(offline_source = - Some(GSSource(tempPath, Map.empty, "eventTimestamp")) - ) + val configWithOfflineSource = config.copy( + source = FileSource(tempPath, Map.empty, "eventTimestamp") ) - OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + BatchPipeline.createPipeline(sparkSession, configWithOfflineSource) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) @@ -183,13 +181,10 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val old = generateDistinctRows(genOld, 10000) val tempPath = storeAsParquet(latest ++ old) - val configWithOfflineSource = config.copy(featureTable = - config.featureTable.copy(offline_source = - Some(GSSource(tempPath, Map.empty, "eventTimestamp")) - ) - ) + val configWithOfflineSource = + config.copy(source = FileSource(tempPath, Map.empty, "eventTimestamp")) - OfflinePipeline.createPipeline(sparkSession, configWithOfflineSource) + BatchPipeline.createPipeline(sparkSession, configWithOfflineSource) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) @@ -216,13 +211,9 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val latest = generateDistinctRows(genLatest, 10000) val tempPath1 = storeAsParquet(latest) - val config1 = config.copy(featureTable = - config.featureTable.copy(offline_source = - Some(GSSource(tempPath1, Map.empty, "eventTimestamp")) - ) - ) + val config1 = config.copy(source = FileSource(tempPath1, Map.empty, "eventTimestamp")) - OfflinePipeline.createPipeline(sparkSession, config1) + BatchPipeline.createPipeline(sparkSession, config1) val genOld = rowGenerator( DateTime.parse("2020-08-01"), @@ -232,13 +223,9 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val old = generateDistinctRows(genOld, 10000) val tempPath2 = storeAsParquet(old) - val config2 = config.copy(featureTable = - config.featureTable.copy(offline_source = - Some(GSSource(tempPath2, Map.empty, "eventTimestamp")) - ) - ) + val config2 = config.copy(source = FileSource(tempPath2, Map.empty, "eventTimestamp")) - OfflinePipeline.createPipeline(sparkSession, config2) + BatchPipeline.createPipeline(sparkSession, config2) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) @@ -262,12 +249,11 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { val tempPath = storeAsParquet(rowsWithNullEntity) val deadletterConfig = config.copy( - featureTable = config.featureTable - .copy(offline_source = Some(GSSource(tempPath, Map.empty, "eventTimestamp"))), + source = FileSource(tempPath, Map.empty, "eventTimestamp"), deadLetterPath = Some(generateTempPath("deadletters")) ) - OfflinePipeline.createPipeline(sparkSession, deadletterConfig) + BatchPipeline.createPipeline(sparkSession, deadletterConfig) jedis.keys("*").toArray should be(empty) @@ -288,22 +274,20 @@ class OfflinePipelineIT extends UnitSpec with ForAllTestContainer { features = Seq( Field("new_feature1", ValueType.Enum.INT32), Field("new_feature2", ValueType.Enum.FLOAT) - ), - offline_source = Some( - GSSource( - tempPath, - Map( - "entity" -> "customer", - "new_feature1" -> "feature1", - "new_feature2" -> "feature2" - ), - "eventTimestamp" - ) ) + ), + source = FileSource( + tempPath, + Map( + "entity" -> "customer", + "new_feature1" -> "feature1", + "new_feature2" -> "feature2" + ), + "eventTimestamp" ) ) - OfflinePipeline.createPipeline(sparkSession, configWithMapping) + BatchPipeline.createPipeline(sparkSession, configWithMapping) val featureKeyEncoder: String => String = encodeFeatureKey(config.featureTable) diff --git a/ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala b/spark/ingestion/src/test/scala/feast/ingestion/UnitSpec.scala similarity index 100% rename from ingestion-spark/src/test/scala/feast/ingestion/UnitSpec.scala rename to spark/ingestion/src/test/scala/feast/ingestion/UnitSpec.scala diff --git a/spark/ingestion/src/test/scala/feast/ingestion/metrics/StatsReporterSpec.scala b/spark/ingestion/src/test/scala/feast/ingestion/metrics/StatsReporterSpec.scala new file mode 100644 index 0000000000..1ae61724ed --- /dev/null +++ b/spark/ingestion/src/test/scala/feast/ingestion/metrics/StatsReporterSpec.scala @@ -0,0 +1,107 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.metrics + +import java.net.{DatagramPacket, DatagramSocket, SocketTimeoutException} +import java.util +import java.util.Collections + +import com.codahale.metrics.{Gauge, Histogram, MetricRegistry, UniformReservoir} +import feast.ingestion.UnitSpec + +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +class StatsReporterSpec extends UnitSpec { + class SimpleServer { + val socket = new DatagramSocket() + socket.setSoTimeout(100) + + def port: Int = socket.getLocalPort + + def receive: Array[String] = { + val messages: ArrayBuffer[String] = ArrayBuffer() + var finished = false + + do { + val buf = new Array[Byte](65535) + val p = new DatagramPacket(buf, buf.length) + try { + socket.receive(p) + } catch { + case _: SocketTimeoutException => { + finished = true + } + } + messages += new String(p.getData, 0, p.getLength) + } while (!finished) + + messages.toArray + } + } + + trait Scope { + val server = new SimpleServer + val reporter = new StatsdReporterWithTags( + new MetricRegistry, + "127.0.0.1", + server.port + ) + + def gauge[A](v: A): Gauge[A] = new Gauge[A] { + override def getValue: A = v + } + + def histogram(values: Seq[Int]): Histogram = { + val hist = new Histogram(new UniformReservoir) + values.foreach(hist.update) + hist + } + } + + "Statsd reporter" should "send simple gauge unmodified" in new Scope { + reporter.report( + gauges = new util.TreeMap( + Map( + "test" -> gauge(0) + ).asJava + ), + counters = Collections.emptySortedMap(), + histograms = Collections.emptySortedMap(), + meters = Collections.emptySortedMap(), + timers = Collections.emptySortedMap() + ) + + server.receive should contain("test:0|g") + } + + "Statsd reporter" should "keep tags part in the name's end" in new Scope { + reporter.report( + gauges = Collections.emptySortedMap(), + counters = Collections.emptySortedMap(), + histograms = new util.TreeMap( + Map( + "test#fs=name" -> histogram((1 to 100)) + ).asJava + ), + meters = Collections.emptySortedMap(), + timers = Collections.emptySortedMap() + ) + + server.receive should contain("test.p95#fs=name:95.95|ms") + } +} From 70c1c6fe1452397a8f1640fd3c9ff532ced724b5 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 5 Oct 2020 11:04:20 +0800 Subject: [PATCH 12/15] clean up dependencies + some api docs Signed-off-by: Oleksii Moskalenko --- spark/ingestion/pom.xml | 104 +++++++++++------- .../scala/feast/ingestion/BasePipeline.scala | 2 +- .../scala/feast/ingestion/BatchPipeline.scala | 22 +++- .../scala/feast/ingestion/IngestionJob.scala | 3 +- .../stores/redis/DefaultSource.scala | 12 +- .../stores/redis/HashTypePersistence.scala | 9 +- .../ingestion/stores/redis/Persistence.scala | 41 +++++++ .../stores/redis/RedisSinkRelation.scala | 17 ++- .../ingestion/utils/TypeConversion.scala | 79 +++++++++++-- 9 files changed, 219 insertions(+), 70 deletions(-) create mode 100644 spark/ingestion/src/main/scala/feast/ingestion/stores/redis/Persistence.scala diff --git a/spark/ingestion/pom.xml b/spark/ingestion/pom.xml index e734cb1137..195e829cdf 100644 --- a/spark/ingestion/pom.xml +++ b/spark/ingestion/pom.xml @@ -41,25 +41,23 @@ - - dev.feast - datatypes-java - ${project.version} - - - - - - - - - - - - - - + + dev.feast + datatypes-java + ${project.version} + + + * + * + + + + + com.google.protobuf + protobuf-java + 3.12.2 + org.scala-lang @@ -67,22 +65,31 @@ ${scala.fullVersion} + + org.scala-lang.modules + scala-collection-compat_${scala.version} + 2.2.0 + + org.apache.spark spark-core_${scala.version} ${spark.version} + provided org.apache.spark spark-streaming_${scala.version} ${spark.version} + provided org.apache.spark spark-sql_${scala.version} ${spark.version} + provided @@ -91,18 +98,12 @@ 3.0.16 - - - - - - - - - - - - + + org.apache.spark + spark-sql-kafka-0-10_${scala.version} + ${spark.version} + provided + com.github.scopt @@ -110,22 +111,11 @@ 3.7.1 - - - com.thesamet.scalapb - sparksql-scalapb_${scala.version} - 0.10.4 - - com.google.cloud.spark spark-bigquery_${scala.version} 0.17.2 - - - - com.google.cloud - google-cloud-bigquery + provided @@ -176,6 +166,7 @@ com.dimafeng testcontainers-scala-scalatest_${scala.version} 0.38.3 + test @@ -258,6 +249,35 @@ + + maven-shade-plugin + + + package + + shade + + + + + com.google.protobuf + com.google.protobuf.vendor + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + diff --git a/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala b/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala index a9a7a6223c..34c667b7e6 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/BasePipeline.scala @@ -54,7 +54,7 @@ trait BasePipeline { ) .set("spark.metrics.conf.*.sink.statsd.host", c.host) .set("spark.metrics.conf.*.sink.statsd.port", c.port.toString) - .set("spark.metrics.conf.*.sink.statsd.period", "1") + .set("spark.metrics.conf.*.sink.statsd.period", "30") .set("spark.metrics.conf.*.sink.statsd.unit", "seconds") .set("spark.metrics.namespace", jobConfig.mode.toString) } diff --git a/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala b/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala index ecedda315c..ef463c1c11 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/BatchPipeline.scala @@ -22,6 +22,14 @@ import feast.ingestion.validation.RowValidator import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.functions.col +/** + * Batch Ingestion Flow: + * 1. Read from source (BQ | File) + * 2. Map source columns to FeatureTable's schema + * 3. Validate + * 4. Store valid rows in redis + * 5. Store invalid rows in parquet format at `deadletter` destination + */ object BatchPipeline extends BasePipeline { override def createPipeline(sparkSession: SparkSession, config: IngestionJobConfig): Unit = { val featureTable = config.featureTable @@ -71,19 +79,23 @@ object BatchPipeline extends BasePipeline { } + /** + * Build column projection using custom mapping with fallback to feature|entity names. + */ private def inputProjection( source: Source, features: Seq[Field], entities: Seq[Field] ): Array[Column] = { - val featureColumns = - if (source.mapping.nonEmpty) - source.mapping - else features.map(f => (f.name, f.name)) + val featureColumns = features + .filter(f => !source.mapping.contains(f.name)) + .map(f => (f.name, f.name)) ++ source.mapping val timestampColumn = Seq((source.timestampColumn, source.timestampColumn)) val entitiesColumns = - entities.filter(e => !source.mapping.contains(e.name)).map(e => (e.name, e.name)) + entities + .filter(e => !source.mapping.contains(e.name)) + .map(e => (e.name, e.name)) (featureColumns ++ entitiesColumns ++ timestampColumn).map { case (alias, source) => col(source).alias(alias) diff --git a/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala index 6d06dd376d..14e2603982 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/IngestionJob.scala @@ -29,7 +29,8 @@ object IngestionJob { new JavaEnumNameSerializer[feast.proto.types.ValueProto.ValueType.Enum]() val parser = new scopt.OptionParser[IngestionJobConfig]("IngestionJon") { - head("feast.ingestion.IngestionJob", "0.8") + // ToDo: read version from Manifest + head("feast.ingestion.IngestionJob", "0.8-SNAPSHOT") opt[Modes]("mode") .action((x, c) => c.copy(mode = x)) diff --git a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala index d870afc09c..ec09a0bed1 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/DefaultSource.scala @@ -19,13 +19,11 @@ package feast.ingestion.stores.redis import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider} -class RedisRelationProvider extends RelationProvider with CreatableRelationProvider { - - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String] - ): BaseRelation = ??? - +/** + * Entrypoint to Redis Storage. Implements only `CreatableRelationProvider` since it's only possible write to Redis. + * Here we parse configuration from spark parameters & provide SparkRedisConfig to `RedisSinkRelation` + */ +class RedisRelationProvider extends CreatableRelationProvider { override def createRelation( sqlContext: SQLContext, mode: SaveMode, diff --git a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala index fc5a45cca0..f1f3f8a5db 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/HashTypePersistence.scala @@ -25,7 +25,14 @@ import com.google.protobuf.Timestamp import feast.ingestion.utils.TypeConversion import scala.util.hashing.MurmurHash3 -class HashTypePersistence(config: SparkRedisConfig) extends Serializable { +/** + * Use Redis hash type as storage layout. Every feature is stored as separate entry in Hash. + * Also additional `timestamp` column is stored per FeatureTable to track update time. + * + * Keys are hashed as murmur3(`featureTableName` : `featureName`). + * Values are serialized with protobuf (`ValueProto`). + */ +class HashTypePersistence(config: SparkRedisConfig) extends Persistence with Serializable { def encodeRow( keyColumns: Array[String], timestampField: String, diff --git a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/Persistence.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/Persistence.scala new file mode 100644 index 0000000000..1be24737b0 --- /dev/null +++ b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/Persistence.scala @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-2020 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.stores.redis + +import org.apache.spark.sql.Row +import redis.clients.jedis.{Pipeline, Response} + +trait Persistence { + def encodeRow( + keyColumns: Array[String], + timestampField: String, + value: Row + ): Map[Array[Byte], Array[Byte]] + + def save( + pipeline: Pipeline, + key: String, + value: Map[Array[Byte], Array[Byte]], + ttl: Int + ): Unit + + def getTimestamp( + pipeline: Pipeline, + key: String, + timestampField: String + ): Response[Array[Byte]] +} diff --git a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala index 0983528ea5..0c228beedc 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/stores/redis/RedisSinkRelation.scala @@ -27,6 +27,15 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.functions.col import org.apache.spark.sql.{DataFrame, Row, SQLContext} +/** + * High-level writer to Redis. Relies on `Persistence` implementation for actual storage layout. + * Here we define general flow: + * + * 1. Deduplicate rows within one batch (group by key and get only latest (by timestamp)) + * 2. Read last-stored timestamp from Redis + * 3. Check if current timestamp is more recent than already saved one + * 4. Save to storage if it's the case + */ class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisConfig) extends BaseRelation with InsertableRelation @@ -43,9 +52,10 @@ class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisC override def schema: StructType = ??? - val persistence = new HashTypePersistence(config) + val persistence: Persistence = new HashTypePersistence(config) override def insert(data: DataFrame, overwrite: Boolean): Unit = { + // repartition for deduplication val dataToStore = if (config.repartitionByEntity) data.repartition(config.entityColumns.map(col): _*) @@ -54,11 +64,13 @@ class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisC dataToStore.foreachPartition { partition: Iterator[Row] => // grouped iterator to only allocate memory for a portion of rows partition.grouped(config.iteratorGroupingSize).foreach { batch => + // group by key and keep only latest row per each key val rowsWithKey: Map[String, Row] = compactRowsToLatestTimestamp(batch.map(row => dataKeyId(row) -> row)).toMap groupKeysByNode(redisConfig.hosts, rowsWithKey.keysIterator).foreach { case (node, keys) => val conn = node.connect() + // retrieve latest stored timestamp per key val timestamps = mapWithPipeline(conn, keys) { (pipeline, key) => persistence.getTimestamp(pipeline, key, timestampField) } @@ -104,6 +116,9 @@ class RedisSinkRelation(override val sqlContext: SQLContext, config: SparkRedisC .values .map(_.maxBy(_._2.getAs[java.sql.Timestamp](config.timestampColumn).getTime)) + /** + * Key is built from entities columns values with prefix of entities columns names. + */ private def dataKeyId(row: Row): String = { val sortedEntities = config.entityColumns.sorted val entityKey = sortedEntities.map(row.getAs[Any]).map(_.toString).mkString(":") diff --git a/spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala b/spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala index 2bbaf70baf..0846b95ce9 100644 --- a/spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala +++ b/spark/ingestion/src/main/scala/feast/ingestion/utils/TypeConversion.scala @@ -18,17 +18,11 @@ package feast.ingestion.utils import java.sql -import com.google.protobuf.{Message, Timestamp} +import com.google.protobuf.{ByteString, Message, Timestamp} import feast.proto.types.ValueProto -import org.apache.spark.sql.types.{ - DataType, - DoubleType, - FloatType, - IntegerType, - LongType, - StringType, - TimestampType -} +import org.apache.spark.sql.types._ + +import scala.collection.JavaConverters._ object TypeConversion { def sqlTypeToProtoValue(value: Any, `type`: DataType): Message = { @@ -38,6 +32,55 @@ object TypeConversion { case StringType => ValueProto.Value.newBuilder().setStringVal(value.asInstanceOf[String]) case DoubleType => ValueProto.Value.newBuilder().setDoubleVal(value.asInstanceOf[Double]) case FloatType => ValueProto.Value.newBuilder().setFloatVal(value.asInstanceOf[Float]) + case StringType => ValueProto.Value.newBuilder().setStringVal(value.asInstanceOf[String]) + case ArrayType(t: IntegerType, _) => + ValueProto.Value + .newBuilder() + .setInt32ListVal( + ValueProto.Int32List.newBuilder + .addAllVal(value.asInstanceOf[Array[Integer]].toSeq.asJava) + ) + case ArrayType(t: LongType, _) => + ValueProto.Value + .newBuilder() + .setInt64ListVal( + ValueProto.Int64List.newBuilder + .addAllVal(value.asInstanceOf[Array[Long]].toSeq.map(java.lang.Long.valueOf).asJava) + ) + case ArrayType(t: BooleanType, _) => + ValueProto.Value + .newBuilder() + .setBoolListVal( + ValueProto.BoolList.newBuilder.addAllVal( + value.asInstanceOf[Array[Boolean]].toSeq.map(java.lang.Boolean.valueOf).asJava + ) + ) + case ArrayType(t: FloatType, _) => + ValueProto.Value + .newBuilder() + .setFloatListVal( + ValueProto.FloatList.newBuilder + .addAllVal(value.asInstanceOf[Array[Float]].toSeq.map(java.lang.Float.valueOf).asJava) + ) + case ArrayType(t: DoubleType, _) => + ValueProto.Value + .newBuilder() + .setDoubleListVal( + ValueProto.DoubleList.newBuilder.addAllVal( + value.asInstanceOf[Array[Double]].toSeq.map(java.lang.Double.valueOf).asJava + ) + ) + case ArrayType(t: ByteType, _) => + ValueProto.Value + .newBuilder() + .setBytesVal(ByteString.copyFrom(value.asInstanceOf[Array[Byte]])) + case ArrayType(t: StringType, _) => + ValueProto.Value + .newBuilder() + .setStringListVal( + ValueProto.StringList.newBuilder + .addAllVal(value.asInstanceOf[Array[String]].toSeq.asJava) + ) case TimestampType => Timestamp .newBuilder() @@ -51,8 +94,20 @@ object TypeConversion { implicit def protoValueAsScala(v: ValueProto.Value): AsScala[Any] = new AsScala[Any]( v.getValCase match { - case ValueProto.Value.ValCase.INT32_VAL => v.getInt32Val - case ValueProto.Value.ValCase.FLOAT_VAL => v.getFloatVal + case ValueProto.Value.ValCase.INT32_VAL => v.getInt32Val + case ValueProto.Value.ValCase.INT64_VAL => v.getInt64Val + case ValueProto.Value.ValCase.FLOAT_VAL => v.getFloatVal + case ValueProto.Value.ValCase.BOOL_VAL => v.getBoolVal + case ValueProto.Value.ValCase.DOUBLE_VAL => v.getDoubleVal + case ValueProto.Value.ValCase.STRING_VAL => v.getStringVal + case ValueProto.Value.ValCase.BYTES_VAL => v.getBytesVal + case ValueProto.Value.ValCase.INT32_LIST_VAL => v.getInt32ListVal.getValList + case ValueProto.Value.ValCase.INT64_LIST_VAL => v.getInt64ListVal.getValList + case ValueProto.Value.ValCase.FLOAT_LIST_VAL => v.getFloatListVal.getValList + case ValueProto.Value.ValCase.DOUBLE_LIST_VAL => v.getDoubleListVal.getValList + case ValueProto.Value.ValCase.BOOL_LIST_VAL => v.getBoolListVal.getValList + case ValueProto.Value.ValCase.STRING_LIST_VAL => v.getStringListVal.getValList + case ValueProto.Value.ValCase.BYTES_LIST_VAL => v.getBytesListVal.getValList case ValueProto.Value.ValCase.VAL_NOT_SET => throw new RuntimeException(s"$v not a ValueProto") } From 3c83d100b446cb52fc0180f944ae4b4f9d7bd5fd Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 5 Oct 2020 12:14:46 +0800 Subject: [PATCH 13/15] extend mapping test Signed-off-by: Oleksii Moskalenko --- .../src/test/scala/feast/ingestion/BatchPipelineIT.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala b/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala index 1fef30d856..8887026a84 100644 --- a/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala +++ b/spark/ingestion/src/test/scala/feast/ingestion/BatchPipelineIT.scala @@ -273,15 +273,14 @@ class BatchPipelineIT extends UnitSpec with ForAllTestContainer { entities = Seq(Field("entity", ValueType.Enum.STRING)), features = Seq( Field("new_feature1", ValueType.Enum.INT32), - Field("new_feature2", ValueType.Enum.FLOAT) + Field("feature2", ValueType.Enum.FLOAT) ) ), source = FileSource( tempPath, Map( "entity" -> "customer", - "new_feature1" -> "feature1", - "new_feature2" -> "feature2" + "new_feature1" -> "feature1" ), "eventTimestamp" ) @@ -297,7 +296,7 @@ class BatchPipelineIT extends UnitSpec with ForAllTestContainer { storedValues should beStoredRow( Map( featureKeyEncoder("new_feature1") -> r.feature1, - featureKeyEncoder("new_feature2") -> r.feature2, + featureKeyEncoder("feature2") -> r.feature2, "_ts:test-fs" -> r.eventTimestamp ) ) From c883f08d297ccd567691452b89ceeae0371f0718 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Mon, 5 Oct 2020 15:47:28 +0800 Subject: [PATCH 14/15] add shade plugin version & group Signed-off-by: Oleksii Moskalenko --- spark/ingestion/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/spark/ingestion/pom.xml b/spark/ingestion/pom.xml index 195e829cdf..a044ec9d60 100644 --- a/spark/ingestion/pom.xml +++ b/spark/ingestion/pom.xml @@ -250,7 +250,9 @@ + org.apache.maven.plugins maven-shade-plugin + 3.2.4 package From c724d0e0af327a37f4653ff84e0471f2b21b6a43 Mon Sep 17 00:00:00 2001 From: Oleksii Moskalenko Date: Tue, 6 Oct 2020 11:47:57 +0800 Subject: [PATCH 15/15] disable buildkit on docker build Signed-off-by: Oleksii Moskalenko --- .github/workflows/complete.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/complete.yml b/.github/workflows/complete.yml index 354b51f087..c57255bacb 100644 --- a/.github/workflows/complete.yml +++ b/.github/workflows/complete.yml @@ -12,7 +12,6 @@ jobs: GITHUB_PR_SHA: ${{ github.event.pull_request.head.sha }} REGISTRY: gcr.io/kf-feast MAVEN_CACHE: gs://feast-templocation-kf-feast/.m2.2020-08-19.tar - DOCKER_BUILDKIT: '1' steps: - uses: actions/checkout@v2 - uses: GoogleCloudPlatform/github-actions/setup-gcloud@master