From b826568ee20a7cc5db6ddc4bc52c6fb96fb9602d Mon Sep 17 00:00:00 2001 From: AbeleMM Date: Tue, 7 Jul 2020 00:42:31 +0200 Subject: [PATCH 1/3] Added changes from Gitlab & 1Up plugins --- build.sbt | 90 +- .../main/scala/org/codefeedr/Properties.scala | 3 - .../org/codefeedr/buffer/KafkaBuffer.scala | 27 +- .../buffer/serialization/AbstractSerde.scala | 8 +- .../buffer/serialization/BsonSerde.scala | 6 +- .../buffer/serialization/JSONSerde.scala | 10 +- .../buffer/serialization/KryoSerde.scala | 8 +- .../buffer/serialization/Serializer.scala | 4 +- .../codefeedr/pipeline/PipelineBuilder.scala | 5 +- .../pipeline/PluginReleasesSource.scala | 129 +++ .../scala/org/codefeedr/pipeline/Stage.scala | 25 +- .../codefeedr/stages/kafka/KafkaOutput.scala | 6 +- .../stages/utilities/DefaultTypeMapper.scala | 26 + .../codefeedr/buffer/KafkaBufferTest.scala | 4 +- .../pipeline/DirectedAcyclicGraphTest.scala | 6 +- .../org/codefeedr/pipeline/PipelineTest.scala | 16 +- .../stages/kafka/KafkaInputOutputTest.scala | 3 +- .../cargo/operators/CargoReleasesSource.scala | 177 ++++ .../plugins/cargo/operators/JsonParser.scala | 447 ++++++++++ .../plugins/cargo/protocol/Protocol.scala | 355 ++++++++ .../cargo/stages/CargoReleasesStage.scala | 36 + .../operators/CargoReleasesSourceTests.scala | 53 ++ .../cargo/operators/JsonParserTests.scala | 282 +++++++ .../cargo/protocol/ProtocolTests.scala | 168 ++++ .../resources/CargoReleasesSnapshot.scala | 6 + .../cargo/resources/CrateSnapshot.scala | 6 + .../stages/CargoReleasesStageTests.scala | 54 ++ .../ClearlyDefinedReleasesSource.scala | 150 ++++ .../clearlydefined/protocol/Protocol.scala | 554 ++++++++++++ .../stages/ClearlyDefinedReleasesStage.scala | 41 + .../ClearlyDefinedReleasesSourceTests.scala | 90 ++ .../protocol/ProtocolTests.scala | 257 ++++++ .../ClearlyDefinedPackageSnapshot.scala | 93 +++ .../ClearlyDefinedReleasesStageTests.scala | 55 ++ .../stages/GHTAbstractEventStage.scala | 3 +- .../ghtorrent/stages/GHTCommitStage.scala | 3 +- .../ghtorrent/stages/GHTInputStage.scala | 7 +- .../util/GHTorrentRabbitMQSource.scala | 2 +- .../plugins/json/JsonExitStage.scala | 28 + .../plugins/json/JsonTransformStage.scala | 35 + .../maven/operators/MavenReleasesSource.scala | 117 +++ .../operators/RetrieveProjectAsync.scala | 76 ++ .../plugins/maven/protocol/Protocol.scala | 378 +++++++++ .../maven/stages/MavenReleasesExtStage.scala | 40 + .../maven/stages/MavenReleasesStage.scala | 40 + .../plugins/maven/util/MavenService.scala | 344 ++++++++ .../operators/MavenReleasesSourceTest.scala | 30 + .../plugins/maven/protocol/ProtocolTest.scala | 160 ++++ .../stages/MavenReleasesExtStageTest.scala | 63 ++ .../maven/stages/MavenReleasesStageTest.scala | 53 ++ .../plugins/maven/util/MavenServiceTest.scala | 239 ++++++ .../mongodb/stages/MongoInputOutputTest.scala | 12 +- .../npm/operators/NpmReleasesSource.scala | 143 ++++ .../npm/operators/RetrieveProjectAsync.scala | 73 ++ .../plugins/npm/protocol/Protocol.scala | 260 ++++++ .../npm/stages/NpmReleasesExtStage.scala | 60 ++ .../plugins/npm/stages/NpmReleasesStage.scala | 42 + .../plugins/npm/util/NpmService.scala | 244 ++++++ .../src/test/resources/test-data/bslet.json | 288 +++++++ .../test/resources/test-data/firetheNun.json | 112 +++ .../test-data/fitfont-haskeywords.json | 790 ++++++++++++++++++ .../src/test/resources/test-data/tiny.json | 120 +++ .../src/test/resources/test-data/ts2php.json | 371 ++++++++ .../test/resources/test-data/unpublished.json | 37 + .../npm/operators/NpmReleasesSourceTest.scala | 75 ++ .../plugins/npm/protocol/ProtocolTest.scala | 368 ++++++++ .../npm/stages/NpmReleasesExtStageTest.scala | 83 ++ .../npm/stages/NpmReleasesStageTest.scala | 67 ++ .../plugins/npm/util/AllfieldTest.scala | 85 ++ .../plugins/npm/util/NpmServiceTest.scala | 336 ++++++++ .../pypi/operators/PyPiReleasesSource.scala | 94 +-- .../pypi/stages/PyPiReleaseExtStage.scala | 11 +- .../pypi/stages/PyPiReleasesStage.scala | 20 +- .../stages/RabbitMQInputOutputTest.scala | 14 +- .../schema_exposure/RedisSchemaExposer.scala | 2 +- .../util}/schema_exposure/SchemaExposer.scala | 4 +- .../ZookeeperSchemaExposer.scala | 26 +- .../RedisSchemaExposerTest.scala | 12 +- .../schema_exposure/SchemaExposerTest.scala | 14 +- .../ZookeeperSchemaExposerTest.scala | 8 +- project/assembly.sbt | 1 - project/plugins.sbt | 1 + 82 files changed, 8400 insertions(+), 191 deletions(-) create mode 100644 codefeedr-core/src/main/scala/org/codefeedr/pipeline/PluginReleasesSource.scala create mode 100644 codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSource.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/JsonParser.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/protocol/Protocol.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStage.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSourceTests.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/JsonParserTests.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/protocol/ProtocolTests.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CargoReleasesSnapshot.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CrateSnapshot.scala create mode 100644 codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStageTests.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSource.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/protocol/Protocol.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSourceTests.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/protocol/ProtocolTests.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/resources/ClearlyDefinedPackageSnapshot.scala create mode 100644 codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStageTests.scala create mode 100644 codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonExitStage.scala create mode 100644 codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonTransformStage.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSource.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/RetrieveProjectAsync.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/util/MavenService.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSourceTest.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/protocol/ProtocolTest.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStageTest.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStageTest.scala create mode 100644 codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/util/MavenServiceTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSource.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/RetrieveProjectAsync.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/util/NpmService.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/bslet.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/firetheNun.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/fitfont-haskeywords.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/tiny.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/ts2php.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/unpublished.json create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSourceTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/protocol/ProtocolTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStageTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStageTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/AllfieldTest.scala create mode 100644 codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/NpmServiceTest.scala rename {codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util}/schema_exposure/RedisSchemaExposer.scala (97%) rename {codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util}/schema_exposure/SchemaExposer.scala (95%) rename {codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util}/schema_exposure/ZookeeperSchemaExposer.scala (86%) rename {codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util}/schema_exposure/RedisSchemaExposerTest.scala (84%) rename {codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util}/schema_exposure/SchemaExposerTest.scala (92%) rename {codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization => codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util}/schema_exposure/ZookeeperSchemaExposerTest.scala (85%) delete mode 100644 project/assembly.sbt diff --git a/build.sbt b/build.sbt index 8ac1cba7..2ad6b143 100644 --- a/build.sbt +++ b/build.sbt @@ -39,6 +39,7 @@ parallelExecution in Test := false val projectPrefix = "codefeedr-" val pluginPrefix = projectPrefix + "plugin-" +val utilPrefix = projectPrefix + "util-" lazy val root = (project in file(".")) .settings(settings ++ noPublishSettings) @@ -48,7 +49,14 @@ lazy val root = (project in file(".")) pluginGitHub, pluginRabbitMQ, pluginGHTorrent, - pluginPypi) + pluginPypi, + pluginCargo, + pluginClearlyDefined, + pluginJSON, + pluginMaven, + pluginnpm, + utilSchemaExposure + ) lazy val core = (project in file("codefeedr-core")) .settings( @@ -88,9 +96,13 @@ lazy val core = (project in file("codefeedr-core")) dependencies.embeddedRedis, // Embedded kafka for integration tests - dependencies.embeddedKafka + dependencies.embeddedKafka, + + // ZK schema registration + dependencies.avro4s ) ) + .dependsOn(utilSchemaExposure) lazy val pluginMongodb = (project in file("codefeedr-plugins/codefeedr-mongodb")) .settings( @@ -162,6 +174,70 @@ lazy val pluginPypi = (project in file("codefeedr-plugins/codefeedr-pypi")) ) ).dependsOn(core) +lazy val pluginCargo = (project in file("codefeedr-plugins/codefeedr-cargo")) + .settings( + name := pluginPrefix + "cargo", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + dependencies.flinkTablePlanner, + dependencies.spray + ) + ).dependsOn(core) + +lazy val pluginClearlyDefined = (project in file("codefeedr-plugins/codefeedr-clearlydefined")) + .settings( + name := pluginPrefix + "clearlydefined", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + dependencies.flinkTablePlanner + ) + ).dependsOn(core, pluginMaven) + +lazy val pluginJSON = (project in file("codefeedr-plugins/codefeedr-json")) + .settings( + name := pluginPrefix + "json", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + ) + ).dependsOn(core) + +lazy val pluginMaven = (project in file("codefeedr-plugins/codefeedr-maven")) + .settings( + name := pluginPrefix + "maven", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + dependencies.flinkTablePlanner + ) + ).dependsOn(core) + +lazy val pluginnpm = (project in file("codefeedr-plugins/codefeedr-npm")) + .settings( + name := pluginPrefix + "npm", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + dependencies.flinkTablePlanner + ) + ).dependsOn(core) + +lazy val utilSchemaExposure = (project in file("codefeedr-util/schema-exposure")) + .settings( + name := utilPrefix + "schema-exposure", + settings, + assemblySettings, + libraryDependencies ++= commonDependencies ++ Seq( + dependencies.avro, + dependencies.zookeeper, + dependencies.redis, + dependencies.embeddedKafka, + dependencies.embeddedRedis + ) + ) + lazy val dependencies = new { val flinkVersion = "1.9.1" @@ -206,6 +282,11 @@ lazy val dependencies = //val embeddedRabbitMQ = "io.arivera.oss" %% "embedded-rabbitmq" % "1.3.0" % Test val avro = "org.apache.avro" % "avro" % "1.8.2" + val avro4s = "com.sksamuel.avro4s" %% "avro4s-core" % "3.1.0" + + val flinkTablePlanner = "org.apache.flink" %% "flink-table-planner" % flinkVersion + + val spray = "io.spray" %% "spray-json" % "1.3.4" } lazy val commonDependencies = Seq( @@ -230,9 +311,9 @@ lazy val commonSettings = Seq( test in assembly := {}, scalacOptions ++= compilerOptions, resolvers ++= Seq( - "confluent" at "http://packages.confluent.io/maven/", + "confluent" at "https://packages.confluent.io/maven/", "Apache Development Snapshot Repository" at "https://repository.apache.org/content/repositories/snapshots/", - "Artima Maven Repository" at "http://repo.artima.com/releases", + "Artima Maven Repository" at "https://repo.artima.com/releases", Resolver.mavenLocal, Resolver.jcenterRepo ), @@ -294,4 +375,3 @@ Global / cancelable := true // exclude Scala library from assembly assembly / assemblyOption := (assembly / assemblyOption).value.copy(includeScala = false) - diff --git a/codefeedr-core/src/main/scala/org/codefeedr/Properties.scala b/codefeedr-core/src/main/scala/org/codefeedr/Properties.scala index de255c56..49761a73 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/Properties.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/Properties.scala @@ -26,9 +26,6 @@ object Properties { // Conversion implicits implicit def stringToBoolean(str: String): Boolean = str.toBoolean implicit def booleanToString(bool: Boolean): String = bool.toString - - - } /** Object containing configuration properties. */ diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/KafkaBuffer.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/KafkaBuffer.scala index bc7b932d..c983f305 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/KafkaBuffer.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/KafkaBuffer.scala @@ -18,23 +18,31 @@ */ package org.codefeedr.buffer -import java.util.{Optional, Properties} +import java.util.Properties import org.apache.avro.reflect.ReflectData import org.apache.flink.streaming.api.functions.sink.SinkFunction import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner -import org.apache.flink.streaming.connectors.kafka.{FlinkKafkaConsumer, FlinkKafkaProducer} +import org.apache.flink.streaming.connectors.kafka.{ + FlinkKafkaConsumer, + FlinkKafkaProducer +} import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig, NewTopic} import org.apache.logging.log4j.scala.Logging -import org.codefeedr.buffer.serialization.schema_exposure.{RedisSchemaExposer, SchemaExposer, ZookeeperSchemaExposer} import org.codefeedr.pipeline.Pipeline import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.reflect.runtime.universe._ import org.codefeedr.Properties._ +import org.codefeedr.util.schema_exposure.{ + RedisSchemaExposer, + SchemaExposer, + ZookeeperSchemaExposer +} + +import scala.language.implicitConversions /** Holds Kafka property names. */ object KafkaBuffer { @@ -110,7 +118,7 @@ class KafkaBuffer[T <: Serializable with AnyRef: ClassTag: TypeTag]( val START_TIMESTAMP = 0x0 //SEMANTIC - val SEMANTIC : Semantic = Semantic.AT_LEAST_ONCE + val SEMANTIC: Semantic = Semantic.AT_LEAST_ONCE } /** Get a Kafka Consumer as source for a stage. @@ -169,11 +177,16 @@ class KafkaBuffer[T <: Serializable with AnyRef: ClassTag: TypeTag]( .getOrElse[String](KafkaBuffer.BROKER, KafkaBufferDefaults.BROKER)) // Check preferred partitioning - val semantic = properties.getOrElse[Semantic](KafkaBuffer.SEMANTIC, KafkaBufferDefaults.SEMANTIC)(stringToSemantic) + val semantic = properties.getOrElse[Semantic]( + KafkaBuffer.SEMANTIC, + KafkaBufferDefaults.SEMANTIC)(stringToSemantic) // Create Kafka producer. val producer = - new FlinkKafkaProducer[T](topic, getSerializer(topic), getKafkaProperties, semantic) + new FlinkKafkaProducer[T](topic, + getSerializer(topic), + getKafkaProperties, + semantic) producer.setWriteTimestampToKafka(true) producer diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/AbstractSerde.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/AbstractSerde.scala index a6d8d3f2..c2524a24 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/AbstractSerde.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/AbstractSerde.scala @@ -18,7 +18,10 @@ */ package org.codefeedr.buffer.serialization -import org.apache.flink.api.common.serialization.{AbstractDeserializationSchema, SerializationSchema} +import org.apache.flink.api.common.serialization.{ + AbstractDeserializationSchema, + SerializationSchema +} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema @@ -33,7 +36,8 @@ abstract class AbstractSerde[T <: Serializable: ClassTag](topic: String = "") extends AbstractDeserializationSchema[T]( TypeExtractor.createTypeInfo( classTag[T].runtimeClass.asInstanceOf[Class[T]])) - with SerializationSchema[T] with KafkaSerializationSchema[T] { + with SerializationSchema[T] + with KafkaSerializationSchema[T] { // Get type of class val inputClassType: Class[T] = classTag[T].runtimeClass.asInstanceOf[Class[T]] diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/BsonSerde.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/BsonSerde.scala index 9348b1b4..c2d6f125 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/BsonSerde.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/BsonSerde.scala @@ -35,7 +35,8 @@ import scala.reflect.runtime.universe._ * * @tparam T Type of the SerDe. */ -class BsonSerde[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String = "") +class BsonSerde[T <: Serializable with AnyRef: TypeTag: ClassTag]( + topic: String = "") extends AbstractSerde[T](topic) { // Implicitly and lazily define the serialization to JSON. @@ -79,6 +80,7 @@ class BsonSerde[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String object BsonSerde { /** Creates new BSON Serde. */ - def apply[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String = ""): BsonSerde[T] = + def apply[T <: Serializable with AnyRef: TypeTag: ClassTag]( + topic: String = ""): BsonSerde[T] = new BsonSerde[T](topic) } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/JSONSerde.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/JSONSerde.scala index 0211fb2f..15694bc5 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/JSONSerde.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/JSONSerde.scala @@ -33,7 +33,8 @@ import scala.reflect.runtime.universe._ * * @tparam T Type of the SerDe. */ -class JSONSerde[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String = "") +class JSONSerde[T <: Serializable with AnyRef: TypeTag: ClassTag]( + topic: String = "") extends AbstractSerde[T](topic) { // Implicitly and lazily define the serialization to JSON. @@ -61,7 +62,9 @@ class JSONSerde[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String /** Serialize as Kafka Producer Record. * @return ProducerRecord. */ - override def serialize(element: T, timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = { + override def serialize( + element: T, + timestamp: lang.Long): ProducerRecord[Array[Byte], Array[Byte]] = { new ProducerRecord(topic, serialize(element)) } } @@ -70,6 +73,7 @@ class JSONSerde[T <: Serializable with AnyRef: TypeTag: ClassTag](topic: String object JSONSerde { /** Creates new JSON Serde. */ - def apply[T <: Serializable with AnyRef: ClassTag: TypeTag](topic: String = ""): JSONSerde[T] = + def apply[T <: Serializable with AnyRef: ClassTag: TypeTag]( + topic: String = ""): JSONSerde[T] = new JSONSerde[T](topic) } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/KryoSerde.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/KryoSerde.scala index e2653841..79aa3d89 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/KryoSerde.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/KryoSerde.scala @@ -30,7 +30,8 @@ import scala.reflect.runtime.universe._ * * @tparam T Type of the SerDe. */ -class KryoSerde[T <: Serializable: TypeTag: ClassTag](topic: String = "") extends AbstractSerde[T](topic) { +class KryoSerde[T <: Serializable: TypeTag: ClassTag](topic: String = "") + extends AbstractSerde[T](topic) { // Lazily retrieve kryo instance. private lazy val kryo: KryoBase = getKryo @@ -70,7 +71,7 @@ class KryoSerde[T <: Serializable: TypeTag: ClassTag](topic: String = "") extend * @return ProducerRecord. */ override def serialize(element: T, timestamp: lang.Long) = { - val buffer : Array[Byte] = new Array[Byte](KryoSerde.BUFFER_SIZE) + val buffer: Array[Byte] = new Array[Byte](KryoSerde.BUFFER_SIZE) val output = new Output(buffer) kryo.writeObject(output, element) @@ -83,6 +84,7 @@ object KryoSerde { val BUFFER_SIZE = 4096 /** Creates new Kryo Serde. */ - def apply[T <: Serializable: ClassTag: TypeTag](topic: String = ""): KryoSerde[T] = + def apply[T <: Serializable: ClassTag: TypeTag]( + topic: String = ""): KryoSerde[T] = new KryoSerde[T](topic) } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/Serializer.scala b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/Serializer.scala index ca877153..3112a38b 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/Serializer.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/Serializer.scala @@ -55,7 +55,9 @@ object Serializer extends Enumeration { * @tparam T the type which has to be serialized/deserialized. * @return the serde instance. */ - def getSerde[T <: Serializable with AnyRef: ClassTag: TypeTag](name: String, topic: String = "") = + def getSerde[T <: Serializable with AnyRef: ClassTag: TypeTag](name: String, + topic: String = + "") = name match { case "JSON" => JSONSerde[T](topic) case "BSON" => BsonSerde[T](topic) diff --git a/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PipelineBuilder.scala b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PipelineBuilder.scala index 540f90ce..c52d5ff6 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PipelineBuilder.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PipelineBuilder.scala @@ -24,7 +24,10 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy import org.apache.flink.runtime.state.StateBackend import org.apache.flink.runtime.state.memory.MemoryStateBackend import org.apache.flink.streaming.api.{CheckpointingMode, TimeCharacteristic} -import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} +import org.apache.flink.streaming.api.scala.{ + DataStream, + StreamExecutionEnvironment +} import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.Semantic import org.apache.logging.log4j.scala.Logging import org.codefeedr.Properties diff --git a/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PluginReleasesSource.scala b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PluginReleasesSource.scala new file mode 100644 index 00000000..23860f06 --- /dev/null +++ b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/PluginReleasesSource.scala @@ -0,0 +1,129 @@ +package org.codefeedr.pipeline + +import org.apache.flink.api.common.accumulators.LongCounter +import org.apache.flink.api.common.state.{ListState, ListStateDescriptor} +import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.state.{ + FunctionInitializationContext, + FunctionSnapshotContext +} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.functions.source.{ + RichSourceFunction, + SourceFunction +} + +import scala.collection.JavaConverters._ +import scala.reflect.{ClassTag, _} + +abstract class PluginSourceConfig { + def pollingInterval: Int + def maxNumberOfRuns: Int +} + +abstract class PluginReleasesSource[T: ClassTag](config: PluginSourceConfig) + extends RichSourceFunction[T] + with CheckpointedFunction { + + /** + * Indication of operation status, i.e. running or not + */ + protected var isRunning: Boolean = false + + /** + * @return whether this source is running or not + */ + def getIsRunning: Boolean = isRunning + + /** + * Counter to indicate the number of polls left to poll the update stream + */ + protected var runsLeft: Int = 0 + + /** + * Accumulator for the amount of processed releases. + */ + protected val releasesProcessed = new LongCounter() + + /** + * Checkpointed last release processed + */ + protected var lastItem: Option[T] = None + + /** + * Keeps track of a checkpointed state of releases + */ + @transient + protected var checkpointedState: ListState[T] = _ + def getCheckpointedstate: ListState[T] = checkpointedState + + /** Opens this source. */ + override def open(parameters: Configuration): Unit = { + isRunning = true + runsLeft = config.maxNumberOfRuns + } + + /** Closes this source. */ + override def cancel(): Unit = { + isRunning = false + } + + /** + * Wait a certain amount of times the polling interval + * + * @param times Times the polling interval should be waited + */ + def waitPollingInterval(times: Int = 1): Unit = { + Thread.sleep(times * config.pollingInterval) + } + + /** + * Reduces runsLeft by 1 + */ + def decreaseRunsLeft(): Unit = { + if (runsLeft > 0) { + runsLeft -= 1 + } + } + + /** Make a snapshot of the current state. */ + override def snapshotState(context: FunctionSnapshotContext): Unit = { + if (lastItem.isDefined) { + checkpointedState.clear() + checkpointedState.add(lastItem.get) + } + } + + /** + * Cycles through the polling of the plugin's run method; Does generic functionality for all plugins + * @param ctx context + */ + def runPlugin( + ctx: SourceFunction.SourceContext[T], + validSortedItems: Seq[T] + ): Unit = { + releasesProcessed.add(validSortedItems.size) + if (validSortedItems.nonEmpty) { + lastItem = Some(validSortedItems.last) + } + + // Wait until the next poll + waitPollingInterval() + } + + /** Initializes state by reading from a checkpoint or creating an empty one. */ + override def initializeState(context: FunctionInitializationContext): Unit = { + val descriptor = new ListStateDescriptor[T]( + "last_element", + classTag[T].runtimeClass.asInstanceOf[Class[T]] + ) + + checkpointedState = context.getOperatorStateStore.getListState(descriptor) + + if (context.isRestored) { + checkpointedState.get().asScala.foreach { x => + lastItem = Some(x) + } + } + } +} diff --git a/codefeedr-core/src/main/scala/org/codefeedr/pipeline/Stage.scala b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/Stage.scala index 752d3dcd..fb6fe94e 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/pipeline/Stage.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/pipeline/Stage.scala @@ -18,15 +18,17 @@ */ package org.codefeedr.pipeline +import org.apache.avro.Schema import org.apache.flink.streaming.api.functions.sink.SinkFunction import org.apache.flink.streaming.api.scala.{ DataStream, StreamExecutionEnvironment } import org.codefeedr.Properties -import org.codefeedr.buffer.BufferFactory +import org.codefeedr.buffer.{BufferFactory, KafkaBuffer} +import org.codefeedr.util.schema_exposure.ZookeeperSchemaExposer -import scala.reflect.{ClassTag, classTag} +import scala.reflect.ClassTag import scala.reflect.runtime.universe._ /** The context of this stage. @@ -82,6 +84,14 @@ protected[codefeedr] abstract class Stage[ */ def setUp(pipeline: Pipeline): Unit = { this.pipeline = pipeline + + val schema = getSchema + val zkAddress = + this.pipeline.bufferProperties.get(KafkaBuffer.ZOOKEEPER).orNull + if (schema != null && zkAddress != null) { + val zk = new ZookeeperSchemaExposer(zkAddress) + zk.put(schema, id) + } } /** Transforms the stage from its input type to its output type. @@ -195,4 +205,15 @@ protected[codefeedr] abstract class Stage[ */ def inList: StageList = new StageList().add(this) + + /** + * Override to enable exporting output type schema to ZooKeeper. + * @note Ensure availability of all implicits required by Avro4s. + * One option is to either include all the components of + * org.codefeedr.stages.utilities.DefaultTypeMapper or an object extending it. + * @example import org.codefeedr.stages.utilities.DefaultTypeMapper._ + * AvroSchema[OutputCaseClass] + * @return Schema type as generated by Avro4s + */ + def getSchema: Schema = null } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/stages/kafka/KafkaOutput.scala b/codefeedr-core/src/main/scala/org/codefeedr/stages/kafka/KafkaOutput.scala index 6216d4b3..c9ce8bbf 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/stages/kafka/KafkaOutput.scala +++ b/codefeedr-core/src/main/scala/org/codefeedr/stages/kafka/KafkaOutput.scala @@ -47,6 +47,10 @@ class KafkaOutput[T <: Serializable with AnyRef: ClassTag: TypeTag]( //add producer as sink override def main(source: DataStream[T]): Unit = { - source.addSink(new FlinkKafkaProducer[T](topic, serde, properties)) + source.addSink( + new FlinkKafkaProducer[T](topic, + serde, + properties, + FlinkKafkaProducer.Semantic.NONE)) } } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala b/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala new file mode 100644 index 00000000..607babde --- /dev/null +++ b/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala @@ -0,0 +1,26 @@ +package org.codefeedr.stages.utilities + +import java.util.Date + +import com.sksamuel.avro4s._ +import org.apache.avro.Schema + +object DefaultTypeMapper { + + /** + * Generate an Avro Schema for java.util.Date. + * + * @param isRowtime determines if Date field is used as a rowtime attribute + */ + class DateSchemaFor(val isRowtime: Boolean = false) extends SchemaFor[Date] { + + override def schema(fieldMapper: FieldMapper): Schema = { + val sc = Schema.create(Schema.Type.STRING) + if (isRowtime) { + sc.addProp("isRowtime", isRowtime) + } + sc + } + } + +} diff --git a/codefeedr-core/src/test/scala/org/codefeedr/buffer/KafkaBufferTest.scala b/codefeedr-core/src/test/scala/org/codefeedr/buffer/KafkaBufferTest.scala index 86cc2021..04848f02 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/buffer/KafkaBufferTest.scala +++ b/codefeedr-core/src/test/scala/org/codefeedr/buffer/KafkaBufferTest.scala @@ -37,7 +37,7 @@ import org.codefeedr.testUtils.{JobFinishedException, SimpleSourceStage} import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import redis.embedded.RedisServer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ class KafkaBufferTest extends FunSuite @@ -186,7 +186,7 @@ object StringCollectSink { result = new util.ArrayList[String]() } - def asList: List[String] = result.toList + def asList: List[String] = result.asScala.toList } class StringCollectSink extends SinkFunction[StringType] { diff --git a/codefeedr-core/src/test/scala/org/codefeedr/pipeline/DirectedAcyclicGraphTest.scala b/codefeedr-core/src/test/scala/org/codefeedr/pipeline/DirectedAcyclicGraphTest.scala index 89ee620c..f4691f51 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/pipeline/DirectedAcyclicGraphTest.scala +++ b/codefeedr-core/src/test/scala/org/codefeedr/pipeline/DirectedAcyclicGraphTest.scala @@ -40,15 +40,15 @@ class DirectedAcyclicGraphTest extends FunSuite { class StringTypeStringTypeNothing extends OutputStage2[StringType, StringType] { override def main(source: DataStream[StringType], - secondSource: DataStream[StringType]): Unit = null + secondSource: DataStream[StringType]): Unit = Unit } class StringTypeNothing extends OutputStage[StringType] { - override def main(source: DataStream[StringType]) = null + override def main(source: DataStream[StringType]) = Unit } class IntTypeNothing extends OutputStage[IntType] { - override def main(source: DataStream[IntType]) = null + override def main(source: DataStream[IntType]) = Unit } class NothingIntType extends InputStage[IntType] { diff --git a/codefeedr-core/src/test/scala/org/codefeedr/pipeline/PipelineTest.scala b/codefeedr-core/src/test/scala/org/codefeedr/pipeline/PipelineTest.scala index eafb0ff4..cc251f2f 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/pipeline/PipelineTest.scala +++ b/codefeedr-core/src/test/scala/org/codefeedr/pipeline/PipelineTest.scala @@ -21,27 +21,17 @@ import java.util import com.github.sebruck.EmbeddedRedis import net.manub.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig} -import org.apache.flink.streaming.api.scala.{ - DataStream, - StreamExecutionEnvironment -} +import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} import org.codefeedr.buffer.{Buffer, BufferType, KafkaBuffer, KafkaTopic} import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.flink.api.scala._ import org.apache.flink.runtime.client.JobExecutionException import org.apache.flink.streaming.api.functions.sink.SinkFunction import org.codefeedr.buffer.serialization.Serializer -import org.codefeedr.buffer.serialization.schema_exposure.{ - RedisSchemaExposer, - ZookeeperSchemaExposer -} import org.codefeedr.stages.{InputStage, OutputStage2} -import org.codefeedr.stages.utilities.{ - JsonPrinterOutput, - StringInput, - StringType -} +import org.codefeedr.stages.utilities.{JsonPrinterOutput, StringInput, StringType} import org.codefeedr.testUtils._ +import org.codefeedr.util.schema_exposure.{RedisSchemaExposer, ZookeeperSchemaExposer} import redis.embedded.RedisServer import scala.collection.JavaConverters._ diff --git a/codefeedr-core/src/test/scala/org/codefeedr/stages/kafka/KafkaInputOutputTest.scala b/codefeedr-core/src/test/scala/org/codefeedr/stages/kafka/KafkaInputOutputTest.scala index 894a5093..e788bb04 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/stages/kafka/KafkaInputOutputTest.scala +++ b/codefeedr-core/src/test/scala/org/codefeedr/stages/kafka/KafkaInputOutputTest.scala @@ -34,7 +34,6 @@ import org.codefeedr.testUtils.JobFinishedException import org.scalatest.{BeforeAndAfterAll, FunSuite} import scala.collection.JavaConverters._ -import scala.collection.JavaConversions._ class KafkaInputOutputTest extends FunSuite @@ -114,7 +113,7 @@ class KafkaInputOutputTest object KafkaStringCollectSink { var result = new util.ArrayList[String]() //mutable list - def asList: List[String] = result.toList + def asList: List[String] = result.asScala.toList } class KafkaStringCollectSink(amount: Int) extends SinkFunction[StringType] { diff --git a/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSource.scala b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSource.scala new file mode 100644 index 00000000..16d3c8e4 --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSource.scala @@ -0,0 +1,177 @@ +package org.codefeedr.plugins.cargo.operators + +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.codefeedr.pipeline.{PluginReleasesSource, PluginSourceConfig} +import org.codefeedr.stages.utilities.{HttpRequester, RequestException} +import org.codefeedr.plugins.cargo.protocol.Protocol.{ + CrateFromPoll, + CrateRelease +} +import scalaj.http.Http +import spray.json._ + +case class CargoSourceConfig(pollingInterval: Int = 10000, + maxNumberOfRuns: Int = -1, + timeout: Int = 32) + extends PluginSourceConfig + +/** + * Important to note in retrieving data from the stream of crates in Cargo is the following: + * - The stream URL is https://crates.io/api/v1/summary + * - A list of 10 of the most recent crates can be found there under the name "new_crates" + * - This information per crate is minimal, so the id/name is taken and used in a separate URL + * - This URL is https://crates.io/api/v1/crates/{name} + * @param config the cargo source configuration, has pollingInterval and maxNumberOfRuns fields + */ +class CargoReleasesSource(config: CargoSourceConfig = CargoSourceConfig()) + extends PluginReleasesSource[CrateRelease](config) { + + /** url for the stream of updated and new crates */ + val url = "https://crates.io/api/v1/summary" + + /** + * Main fetcher of new items in the Crates.io package source + * @param ctx context + */ + override def run(ctx: SourceFunction.SourceContext[CrateRelease]): Unit = { + val lock = ctx.getCheckpointLock + + /** While is running or #runs left. */ + while (isRunning && runsLeft != 0) { + lock.synchronized { // Synchronize to the checkpoint lock. + try { + // Polls the RSS feed + val rssAsString: String = getRSSAsString.get + // Parse into polled micro-crates + val polledItems: Seq[CrateFromPoll] = getPolledCrates(rssAsString) + // Collect only new crates and sort them based on updated date + val validSortedItems: Seq[CrateFromPoll] = + sortAndDropDuplicates(polledItems) + // Parses the polled crates into full crates + val items: Seq[CrateRelease] = parseNewCrates(validSortedItems) + // Decrease runs left + super.decreaseRunsLeft() + // Add a timestamp to the item + items.foreach(x => + ctx.collectWithTimestamp(x, x.crate.updated_at.getTime)) + // Call the parent run + super.runPlugin(ctx, items) + } catch { + case _: Throwable => + } + } + } + } + + /** + * Drops items that already have been collected and sorts them based on times + * + * @param items Potential items to be collected + * @return Valid sorted items + */ + def sortAndDropDuplicates(items: Seq[CrateFromPoll]): Seq[CrateFromPoll] = { + items + .filter((x: CrateFromPoll) => { + if (lastItem.isDefined) + lastItem.get.crate.updated_at.before(x.updated_at) + else + true + }) + .sortWith((x: CrateFromPoll, y: CrateFromPoll) => + x.updated_at.before(y.updated_at)) + } + + /** + * Requests the RSS feed and returns its body as a string. + * Will keep trying with increasing intervals if it doesn't succeed + * + * @return Body of requested RSS feed + */ + @throws[RequestException] + def getRSSAsString: Option[String] = { + try { + Some(new HttpRequester().retrieveResponse(Http(url)).body) + } catch { + case _: Throwable => None + } + } + + /** + * Gets the body response of a specific crate as String + * @param crateName Name of the crate + * @return Http Response body + */ + def getRSSFromCrate(crateName: String): Option[String] = { + try { + Some( + new HttpRequester() + .retrieveResponse( + Http("https://crates.io/api/v1/crates/".concat(crateName)) + ) + .body + ) + } catch { + case _: Throwable => None + } + } + + /** + * Processes the html body from the Cargo feed into a list of polled Crates + * @param rssString html body of the webrequest to the releases feed + * @return A list of polled crates, consisting of 10 new crates and 10 updated crates + */ + def getPolledCrates(rssString: String): Seq[CrateFromPoll] = { + try { + // Parse the big release string as a Json object + val json: JsObject = rssString.parseJson.asJsObject + + // Retrieve 2x10 JsObjects of Crates + val newCrates: Vector[JsObject] = + JsonParser.getNewOrUpdatedCratesFromSummary(json, "new_crates").get + val updatedCrates: Vector[JsObject] = + JsonParser.getNewOrUpdatedCratesFromSummary(json, "just_updated").get + + // Translate 2x10 JSObjects into CrateFromPolls + for (crate <- newCrates ++ updatedCrates) yield { + val crateId: String = + JsonParser.getStringFieldFromCrate(crate, "id").get + val crateUpdated = + JsonParser.getDateFieldFromCrate(crate, "updated_at").get + CrateFromPoll(crateId, crateUpdated) + } + } catch { + // If the string cannot be parsed return an empty list + case _: Throwable => + printf( + "Failed parsing the RSSString in the CargoReleasesSource.scala file" + ) + Nil + } + } + + /** + * Parses a string that contains polled crates into a list of CrateReleases + * + * @param newCrates sequence of polled crates to turn into extended releases + * @return Sequence of RSS items in type CrateRelease + */ + def parseNewCrates(newCrates: Seq[CrateFromPoll]): Seq[CrateRelease] = { + try { + for (crate <- newCrates) yield { + // Get the html body of the crate + val crateRSS: String = getRSSFromCrate(crate.id).get + // Turn the html body into a json object + val crateJson: JsObject = crateRSS.parseJson.asJsObject + // Turn the json object into a CrateRelease object + JsonParser.parseCrateJsonToCrateRelease(crateJson).get + } + } catch { + // If the string cannot be parsed return an empty list + case _: Throwable => + printf( + "Failed parsing the RSSString in the CargoReleasesSource.scala file" + ) + Nil + } + } +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/JsonParser.scala b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/JsonParser.scala new file mode 100644 index 00000000..c82c0909 --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/operators/JsonParser.scala @@ -0,0 +1,447 @@ +package org.codefeedr.plugins.cargo.operators + +import java.sql.Timestamp +import java.text.SimpleDateFormat +import java.util.{Calendar, Date} + +import org.codefeedr.plugins.cargo.protocol.Protocol._ +import spray.json.{JsArray, JsBoolean, JsNumber, JsObject, JsString, JsValue} + +object JsonParser { + //TODO, The last 3 S's might create bugs + val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX") + + /** + * Gets the total number of crates in the Cargo package source currently available + * @param jsObject body of the Crate.io summary page parsed into JsObject + * @param field string which denotes either 'num_crates' or 'num_downloads' to retrieve the corresponding value + * @return Number of crates + */ + def getNumCratesOrDownloadsFromSummary(jsObject: JsObject, + field: String): Option[Int] = { + try { + val numCrates: Option[JsValue] = jsObject.fields.get(field) + + Some( + numCrates.get + .asInstanceOf[JsNumber] + .value + .toInt) + } catch { + case _: Throwable => None + } + } + + /** + * Retrieves the ten most recently updated or brand new Crates from the summary page + * @param jsObject Summary page parsed into a JsObject (crates.io/api/v1/summary) + * @param field "new_crates" or "just_updated" for the type of crates to retrieve + * @return A Vector consisting of 10 JsObjects which are the most recently updated or new crates + */ + def getNewOrUpdatedCratesFromSummary( + jsObject: JsObject, + field: String): Option[Vector[JsObject]] = { + try { + val updatedCrates: Option[JsValue] = jsObject.fields.get(field) + + Some( + updatedCrates.get + .asInstanceOf[JsArray] + .elements + .distinct + .map(x => x.asJsObject())) + } catch { + case _: Throwable => None + } + } + + /** + * Method for parsing the CrateRelease Json into a CrateRelease object + * Can be seen as the 'parent' method of this class + * @param jsObject object to be parsed + * @return + */ + def parseCrateJsonToCrateRelease(jsObject: JsObject): Option[CrateRelease] = { + try { + val crate: JsObject = jsObject.fields("crate").asJsObject() + val versions: List[JsValue] = + jsObject.fields("versions").asInstanceOf[JsArray].elements.toList + val keywords: List[JsValue] = + jsObject.fields("keywords").asInstanceOf[JsArray].elements.toList + val categories: List[JsValue] = + jsObject.fields("categories").asInstanceOf[JsArray].elements.toList + + val crateObject: Crate = this.parseCrateJsonToCrateObject(crate).get + val versionsObject: List[CrateVersion] = versions.map({ x => + this.parseCrateVersionsJsonToCrateVersionsObject(x.asJsObject).get + }) + val keywordsObject: List[CrateKeyword] = keywords.map( + x => + this + .parseCrateKeywordsJsonToCrateKeywordsObject( + x.asInstanceOf[JsObject]) + .get) + val categoriesObject: List[CrateCategory] = categories.map( + x => + this + .parseCrateCategoryJsonToCrateCategoryObject( + x.asInstanceOf[JsObject]) + .get) + + Some( + CrateRelease(crateObject, + versionsObject, + keywordsObject, + categoriesObject)) + } catch { + case e: Throwable => + printf( + "\nparsing from Json to CrateRelease failed with info: " + e.toString) + None + } + } + + /** + * Method for parsing the Crate field in a CrateRelease object + * @param jsObject object to be parsed + * @return + */ + def parseCrateJsonToCrateObject(jsObject: JsObject): Option[Crate] = { + try { + val id: String = this.getStringFieldFromCrate(jsObject, "id").get + val name: String = this.getStringFieldFromCrate(jsObject, "name").get + val updated_at: Timestamp = + this.getDateFieldFromCrate(jsObject, "updated_at").get + val versions: List[Int] = + this.getListOfIntsFieldFromCrate(jsObject, "versions").get + val keywords: List[String] = + this.getListOfStringsFieldFromCrate(jsObject, "keywords").get + val categories: List[String] = + this.getListOfStringsFieldFromCrate(jsObject, "categories").get + val created_at: Timestamp = + this.getDateFieldFromCrate(jsObject, "created_at").get + val downloads: Int = this.getIntFieldFromCrate(jsObject, "downloads").get + val recent_downloads: Option[Int] = + this.getIntFieldFromCrate(jsObject, "recent_downloads") + val max_version: String = + this.getStringFieldFromCrate(jsObject, "max_version").get + val description: String = + this.getStringFieldFromCrate(jsObject, "description").get + val homepage: Option[String] = + this.getStringFieldFromCrate(jsObject, "homepage") + val documentation: Option[String] = + this.getStringFieldFromCrate(jsObject, "documentation") + val repository: Option[String] = + this.getStringFieldFromCrate(jsObject, "repository") + val links: CrateLinks = + this + .parseCrateLinksJsonToCrateLinksObject( + jsObject.fields("links").asJsObject()) + .get + val exact_match: Boolean = + this.getBoolFieldFromCrate(jsObject, "exact_match").get + + Some( + Crate( + id, + name, + updated_at, + versions, + keywords, + categories, + created_at, + downloads, + recent_downloads, + max_version, + description, + homepage, + documentation, + repository, + links, + exact_match + ) + ) + } catch { + case e: Throwable => + printf("\nparsing from Json to Crate failed with info: " + e.toString) + None + } + } + + /** + * Method for parsing the CrateVersion fields in a CrateRelease object + * @param jsObject object to be parsed + * @return + */ + def parseCrateVersionsJsonToCrateVersionsObject( + jsObject: JsObject): Option[CrateVersion] = { + try { + val id: Int = this.getIntFieldFromCrate(jsObject, "id").get + val crate: String = this.getStringFieldFromCrate(jsObject, "crate").get + val num: String = this.getStringFieldFromCrate(jsObject, "num").get + val dl_path: String = + this.getStringFieldFromCrate(jsObject, "dl_path").get + val readme_path: String = + this.getStringFieldFromCrate(jsObject, "readme_path").get + val updated_at: Timestamp = + this.getDateFieldFromCrate(jsObject, "updated_at").get + val created_at: Timestamp = + this.getDateFieldFromCrate(jsObject, "created_at").get + val downloads: Int = this.getIntFieldFromCrate(jsObject, "downloads").get + val features + : CrateVersionFeatures = new CrateVersionFeatures // this is always empty + val yanked: Boolean = this.getBoolFieldFromCrate(jsObject, "yanked").get + val license: String = + this.getStringFieldFromCrate(jsObject, "license").get + val links: CrateVersionLinks = + this + .parseCrateVersionLinksJsonToCrateVersionLinksObject( + jsObject.fields("links").asJsObject()) + .get + val crate_size: Option[Int] = + this.getIntFieldFromCrate(jsObject, "crate_size") + val published_by: Option[CrateVersionPublishedBy] = try { + this.parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject( + jsObject.fields("published_by").asJsObject()) + } catch { + case _: Throwable => None + } + + Some( + CrateVersion(id, + crate, + num, + dl_path, + readme_path, + updated_at, + created_at, + downloads, + features, + yanked, + license, + links, + crate_size, + published_by) + ) + } catch { + case _: Throwable => + printf("\nparsing from Json to CrateVersion object failed") + None + } + } + + /** + * Method for parsing the Links field in a Crate object + * @param jsObject object to be parsed + * @return + */ + def parseCrateLinksJsonToCrateLinksObject( + jsObject: JsObject): Option[CrateLinks] = { + try { + val version_downloads: String = + this.getStringFieldFromCrate(jsObject, "version_downloads").get + val versions: Option[String] = + this.getStringFieldFromCrate(jsObject, "versions") + val owners: String = this.getStringFieldFromCrate(jsObject, "owners").get + val owner_team: String = + this.getStringFieldFromCrate(jsObject, "owner_team").get + val owner_user: String = + this.getStringFieldFromCrate(jsObject, "owner_user").get + val reverse_dependencies: String = + this.getStringFieldFromCrate(jsObject, "reverse_dependencies").get + + Some( + CrateLinks(version_downloads, + versions, + owners, + owner_team, + owner_user, + reverse_dependencies) + ) + } catch { + case _: Throwable => + printf("\nparsing crate links failed.") + None + } + } + + /** + * Method for parsing the CrateVersionLinks field in a CrateVersion object + * @param jsObject object to be parsed + * @return + */ + def parseCrateVersionLinksJsonToCrateVersionLinksObject( + jsObject: JsObject): Option[CrateVersionLinks] = { + try { + val dependencies: String = + this.getStringFieldFromCrate(jsObject, "dependencies").get + val version_downloads: String = + this.getStringFieldFromCrate(jsObject, "version_downloads").get + val authors: String = + this.getStringFieldFromCrate(jsObject, "authors").get + + Some(CrateVersionLinks(dependencies, version_downloads, authors)) + } catch { + case _: Throwable => + printf("\nparsing crate version links returned a None somehow.") + None + } + } + + /** + * Method for parsing the CrateVersionPublishedBy field in a CrateVersion object + * @param jsObject object to be parsed + * @return + */ + def parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject( + jsObject: JsObject): Option[CrateVersionPublishedBy] = { + try { + val id: Int = this.getIntFieldFromCrate(jsObject, "id").get + val login: String = this.getStringFieldFromCrate(jsObject, "login").get + val name: Option[String] = this.getStringFieldFromCrate(jsObject, "name") + val avatar: String = this.getStringFieldFromCrate(jsObject, "avatar").get + val url: String = this.getStringFieldFromCrate(jsObject, "url").get + + Some(CrateVersionPublishedBy(id, login, name, avatar, url)) + } catch { + case _: Throwable => + printf("\nparsing crate version links returned a None somehow.") + None + } + } + + /** + * Method for parsing the CrateKeyword field in a Crate object + * @param jsObject object to be parsed + * @return + */ + def parseCrateKeywordsJsonToCrateKeywordsObject( + jsObject: JsObject): Option[CrateKeyword] = { + try { + val id: String = this.getStringFieldFromCrate(jsObject, "id").get + val keyword: String = + this.getStringFieldFromCrate(jsObject, "keyword").get + val created_at: String = + this.getStringFieldFromCrate(jsObject, "created_at").get + val crates_cnt = this.getIntFieldFromCrate(jsObject, "crates_cnt").get + + Some(CrateKeyword(id, keyword, created_at, crates_cnt)) + } catch { + case _: Throwable => + printf("\nparsing the crate keyword returned a None somehow.") + None + } + } + + /** + * Method for parsing the CrateCategory field in a Crate object + * @param jsObject object to be parsed + * @return + */ + def parseCrateCategoryJsonToCrateCategoryObject( + jsObject: JsObject): Option[CrateCategory] = { + try { + val id: String = this.getStringFieldFromCrate(jsObject, "id").get + val category: String = + this.getStringFieldFromCrate(jsObject, "category").get + val slug: String = this.getStringFieldFromCrate(jsObject, "slug").get + val description: String = + this.getStringFieldFromCrate(jsObject, "description").get + val created_at: String = + this.getStringFieldFromCrate(jsObject, "created_at").get + val crates_cnt: Int = + this.getIntFieldFromCrate(jsObject, "crates_cnt").get + + Some( + CrateCategory(id, category, slug, description, created_at, crates_cnt)) + } catch { + case _: Throwable => + printf("\nparsing the crate category returned a None somehow.") + None + } + } + + /** Below methods are used to parse single fields in Json to their respective types */ + def getStringFieldFromCrate(jsObject: JsObject, + field: String): Option[String] = { + try { + Some(jsObject.fields(field).asInstanceOf[JsString].value) + } catch { + case _: Throwable => + None + } + } + + def getIntFieldFromCrate(jsObject: JsObject, field: String): Option[Int] = { + try { + Some(jsObject.fields(field).asInstanceOf[JsNumber].value.toInt) + } catch { + case _: Throwable => + None + } + } + + def getBoolFieldFromCrate(jsObject: JsObject, + field: String): Option[Boolean] = { + try { + Some(jsObject.fields(field).asInstanceOf[JsBoolean].value) + } catch { + case _: Throwable => + None + } + } + + def getDateFieldFromCrate(jsObject: JsObject, + field: String): Option[Timestamp] = { + try { + val fieldValue: String = + jsObject.fields(field).asInstanceOf[JsString].value + val dateField: Date = dateFormat.parse(fieldValue) + val cal = Calendar.getInstance + + cal.setTime(dateField) + val time = new Timestamp(cal.getTimeInMillis) + + Some(time) + } catch { + case _: Throwable => + println("Failed parsing the Date") + None + } + } + + def getListOfIntsFieldFromCrate(jsObject: JsObject, + field: String): Option[List[Int]] = { + try { + Some( + jsObject + .fields(field) + .asInstanceOf[JsArray] + .elements + .toList + .map(x => x.asInstanceOf[JsNumber]) + .map(x => x.value) + .map(x => x.toInt)) + } catch { + case _: Throwable => + None + } + } + + def getListOfStringsFieldFromCrate(jsObject: JsObject, + field: String): Option[List[String]] = { + try { + Some( + jsObject + .fields(field) + .asInstanceOf[JsArray] + .elements + .toList + .map(x => x.asInstanceOf[JsString]) + .map(x => x.value)) + } catch { + case _: Throwable => + None + } + } +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/protocol/Protocol.scala new file mode 100644 index 00000000..0a506aca --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/protocol/Protocol.scala @@ -0,0 +1,355 @@ +package org.codefeedr.plugins.cargo.protocol + +import java.sql.Timestamp + +object Protocol extends Enumeration { + + case class CrateFromPoll(id: String, updated_at: Timestamp) + + case class CrateRelease( + crate: Crate, + versions: List[CrateVersion], + keywords: List[CrateKeyword], + categories: List[CrateCategory] + ) + + class CrateReleasePojo extends Serializable { + var crate: CratePojo = _ + var versions: List[CrateVersionPojo] = _ + var keywords: List[CrateKeywordPojo] = _ + var categories: List[CrateCategoryPojo] = _ + } + + object CrateReleasePojo { + def fromCrateRelease(crateRelease: CrateRelease): CrateReleasePojo = { + val pojo = new CrateReleasePojo + + pojo.crate = CratePojo.fromCrate(crateRelease.crate) + + // Map the CrateVersions + pojo.versions = crateRelease.versions.map(x => { + CrateVersionPojo.fromCrateVersion(x) + }) + + // Map the CrateKeywords + pojo.keywords = crateRelease.keywords.map(x => { + CrateKeywordPojo.fromCrateKeyword(x) + }) + + // Map the CrateCategories + pojo.categories = crateRelease.categories.map(x => { + CrateCategoryPojo.fromCrateCategory(x) + }) + pojo + } + } + + case class Crate( + id: String, + name: String, + updated_at: Timestamp, + versions: List[Int], + keywords: List[String], + categories: List[String], + //badges: Option[List[String]], //Unimportant information / Too extensive, See natvis-pdbs + created_at: Timestamp, + downloads: Int, // Assuming no more than 2B downloads + recent_downloads: Option[Int], + max_version: String, + description: String, + homepage: Option[String], + documentation: Option[String], + repository: Option[String], + links: CrateLinks, + exact_match: Boolean + ) + + class CratePojo extends Serializable { + var id: String = _ + var name: String = _ + var updated_at: Timestamp = _ + var versions: List[Int] = _ + var keywords: List[String] = _ + var categories: List[String] = _ + var created_at: Timestamp = _ + var downloads: Int = _ + var recent_downloads: Int = _ + var max_version: String = _ + var description: String = _ + var homepage: String = _ + var documentation: String = _ + var repository: String = _ + var links: CrateLinksPojo = _ + var exact_match: Boolean = _ + } + + object CratePojo { + def fromCrate(crate: Crate): CratePojo = { + val pojo = new CratePojo + pojo.id = crate.id + pojo.name = crate.name + pojo.updated_at = crate.updated_at + pojo.versions = crate.versions + pojo.keywords = crate.keywords + pojo.categories = crate.categories + pojo.created_at = crate.created_at + pojo.downloads = crate.downloads + if (crate.recent_downloads.isDefined) { + pojo.recent_downloads = crate.recent_downloads.get + } + pojo.max_version = crate.max_version + pojo.description = crate.description + pojo.homepage = crate.homepage.orNull + pojo.documentation = crate.documentation.orNull + pojo.repository = crate.repository.orNull + pojo.links = CrateLinksPojo.fromCrateLinks(crate.links) + pojo.exact_match = crate.exact_match + pojo + } + } + + case class CrateLinks( + version_downloads: String, + versions: Option[String], + owners: String, + owner_team: String, + owner_user: String, + reverse_dependencies: String + ) + + class CrateLinksPojo extends Serializable { + var version_downloads: String = _ + var versions: String = _ + var owners: String = _ + var owner_team: String = _ + var owner_user: String = _ + var reverse_dependencies: String = _ + } + + class CrateLinksPojoExt extends CrateLinksPojo { + var crateId: String = _ + } + + object CrateLinksPojo { + def fromCrateLinks(crateLinks: CrateLinks): CrateLinksPojo = { + val pojo = new CrateLinksPojo + pojo.version_downloads = crateLinks.version_downloads + pojo.versions = crateLinks.versions.orNull + pojo.owners = crateLinks.owners + pojo.owner_team = crateLinks.owner_team + pojo.owner_user = crateLinks.owner_user + pojo.reverse_dependencies = crateLinks.reverse_dependencies + pojo + } + } + + case class CrateVersion( + id: Int, + crate: String, + num: String, + dl_path: String, + readme_path: String, + updated_at: Timestamp, + created_at: Timestamp, + downloads: Int, + features: CrateVersionFeatures, + yanked: Boolean, + license: String, + links: CrateVersionLinks, + crate_size: Option[Int], + published_by: Option[CrateVersionPublishedBy] + ) + + class CrateVersionPojo extends Serializable { + var id: Int = _ + var crate: String = _ + var num: String = _ + var dl_path: String = _ + var readme_path: String = _ + var updated_at: Timestamp = _ + var created_at: Timestamp = _ + var downloads: Int = _ + var features: CrateVersionFeaturesPojo = _ + var yanked: Boolean = _ + var license: String = _ + var links: CrateVersionLinksPojo = _ + var crate_size: Int = _ + var published_by: CrateVersionPublishedByPojo = _ + } + + object CrateVersionPojo { + def fromCrateVersion(crateVersion: CrateVersion): CrateVersionPojo = { + val pojo = new CrateVersionPojo + pojo.id = crateVersion.id + pojo.crate = crateVersion.crate + pojo.num = crateVersion.num + pojo.dl_path = crateVersion.dl_path + pojo.readme_path = crateVersion.readme_path + pojo.updated_at = crateVersion.updated_at + pojo.created_at = crateVersion.created_at + pojo.downloads = crateVersion.downloads + pojo.features = CrateVersionFeaturesPojo.fromCrateVersionFeatures() + pojo.yanked = crateVersion.yanked + pojo.license = crateVersion.license + pojo.links = + CrateVersionLinksPojo.fromCrateVersionLinks(crateVersion.links) + if (crateVersion.crate_size.isDefined) { + pojo.crate_size = crateVersion.crate_size.get + } + if (crateVersion.published_by.isDefined) { + pojo.published_by = + CrateVersionPublishedByPojo.fromCrateVersionPublishedBy( + crateVersion.published_by.get + ) + } + pojo + } + } + + // checked ~5 repos this is all JSON {}, which means empty complex object... + // At least one parameter necessary for REPL compatibility. + case class CrateVersionFeatures(s: String = null) + + class CrateVersionFeaturesPojo extends Serializable {} + + class CrateVersionFeaturesPojoExt extends CrateVersionFeaturesPojo { + // unique key = id + crate of parent object CrateVersionPojo + var versionId: Int = _ + var crate: String = _ + } + + object CrateVersionFeaturesPojo { + def fromCrateVersionFeatures(): CrateVersionFeaturesPojo = { + new CrateVersionFeaturesPojo + } + } + + case class CrateVersionLinks( + dependencies: String, + version_downloads: String, + authors: String + ) + + class CrateVersionLinksPojo extends Serializable { + var dependencies: String = _ + var version_downloads: String = _ + var authors: String = _ + } + + class CrateVersionLinksPojoExt extends CrateVersionLinksPojo { + // unique key = id + crate of parent object CrateVersionPojo + var versionId: Int = _ + var crate: String = _ + } + + object CrateVersionLinksPojo { + def fromCrateVersionLinks( + crateVersionLinks: CrateVersionLinks + ): CrateVersionLinksPojo = { + val pojo = new CrateVersionLinksPojo + pojo.dependencies = crateVersionLinks.dependencies + pojo.version_downloads = crateVersionLinks.version_downloads + pojo.authors = crateVersionLinks.authors + pojo + } + } + + case class CrateVersionPublishedBy( + id: Int, + login: String, + name: Option[String], + avatar: String, + url: String + ) + + class CrateVersionPublishedByPojo extends Serializable { + var id: Int = _ + var login: String = _ + var name: String = _ + var avatar: String = _ + var url: String = _ + } + + class CrateVersionPublishedByPojoExt extends CrateVersionPublishedByPojo { + var versionId: Int = _ + var crate: String = _ + } + + object CrateVersionPublishedByPojo { + def fromCrateVersionPublishedBy( + crateVersionPublishedBy: CrateVersionPublishedBy + ): CrateVersionPublishedByPojo = { + val pojo = new CrateVersionPublishedByPojo + pojo.id = crateVersionPublishedBy.id + pojo.login = crateVersionPublishedBy.login + pojo.name = crateVersionPublishedBy.name.orNull + pojo.avatar = crateVersionPublishedBy.avatar + pojo.url = crateVersionPublishedBy.url + pojo + } + } + + case class CrateKeyword( + id: String, + keyword: String, + created_at: String, + crates_cnt: Int + ) + + class CrateKeywordPojo extends Serializable { + var id: String = _ + var keyword: String = _ + var created_at: String = _ + var crates_cnt: Int = _ + } + + class CrateKeywordPojoExt extends CrateKeywordPojo { + var crate: String = _ + } + + object CrateKeywordPojo { + def fromCrateKeyword(crateKeyword: CrateKeyword): CrateKeywordPojo = { + val pojo = new CrateKeywordPojo + pojo.id = crateKeyword.id + pojo.keyword = crateKeyword.keyword + pojo.created_at = crateKeyword.created_at + pojo.crates_cnt = crateKeyword.crates_cnt + pojo + } + } + + case class CrateCategory( + id: String, + category: String, + slug: String, + description: String, + created_at: String, + crates_cnt: Int + ) + + class CrateCategoryPojo extends Serializable { + var id: String = _ + var category: String = _ + var slug: String = _ + var description: String = _ + var created_at: String = _ + var crates_cnt: Int = _ + } + + class CrateCategoryPojoExt extends CrateCategoryPojo { + var crate: String = _ + } + + object CrateCategoryPojo { + def fromCrateCategory(crateCategory: CrateCategory): CrateCategoryPojo = { + val pojo = new CrateCategoryPojo + pojo.id = crateCategory.id + pojo.category = crateCategory.category + pojo.slug = crateCategory.slug + pojo.description = crateCategory.description + pojo.created_at = crateCategory.created_at + pojo.crates_cnt = crateCategory.crates_cnt + pojo + } + } +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStage.scala b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStage.scala new file mode 100644 index 00000000..e3b7d128 --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/main/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStage.scala @@ -0,0 +1,36 @@ +package org.codefeedr.plugins.cargo.stages + +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.Context +import org.codefeedr.stages.InputStage +import org.codefeedr.plugins.cargo.operators.{ + CargoReleasesSource, + CargoSourceConfig +} +import org.codefeedr.plugins.cargo.protocol.Protocol.CrateRelease + +import scala.language.higherKinds + +/** fetches real-time releases from Cargo */ +class CargoReleasesStage(stageId: String = "cargo_releases_min", + sourceConfig: CargoSourceConfig = + CargoSourceConfig(500, -1)) + extends InputStage[CrateRelease](Some(stageId)) { + + /** Fetches [[CrateRelease]] from real-time Cargo feed. + * + * @param context The context to add the source to. + * @return The stream of type [[CrateRelease]]. + */ + override def main(context: Context): DataStream[CrateRelease] = { + implicit val typeInfo: TypeInformation[CrateRelease] = + TypeInformation.of(classOf[CrateRelease]) + context.env + .addSource(new CargoReleasesSource(sourceConfig))(typeInfo) + } + + override def getSchema: Schema = AvroSchema[CrateRelease] +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSourceTests.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSourceTests.scala new file mode 100644 index 00000000..e4275e03 --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/CargoReleasesSourceTests.scala @@ -0,0 +1,53 @@ +package org.codefeedr.plugins.cargo.operators + +import org.apache.flink.runtime.state.FunctionSnapshotContext +import org.scalatest.{BeforeAndAfter, FunSuite} + +class CargoReleasesSourceTests extends FunSuite with BeforeAndAfter { + + val crs: CargoReleasesSource = new CargoReleasesSource() + + before { + // First assure that the url field is set to the crates io summary stream + assert(crs.url.equals("https://crates.io/api/v1/summary")) + } + + test("getRSSAsString works on /api/v1/summary") { + // Retrieve the RSS information in string format + val cargoSummary = crs.getRSSAsString + + // Assert string type and correct start + assert(cargoSummary.get.startsWith("{\"num_downloads\":")) + } + + test("getRSSFromCrate works on example 'shtola-markdown'") { + val shtolaCrate = crs.getRSSFromCrate("shtola-markdown") + + // Assert string type and correctly retrieved start of body + assert(shtolaCrate.get.startsWith("{\"crate\":{\"id\":\"shtola-markdown\",\"name\":\"shtola-markdown\",\"updated_at\":\"")) + } + + test("snapshotState no lastItem") { + // Arrange + val ctx = new FunctionSnapshotContext { + override def getCheckpointId: Long = 300 + + override def getCheckpointTimestamp: Long = 200 + } + + // Act + crs.snapshotState(ctx) + + // Assert + assert(crs.getCheckpointedstate == null) + } + + test("cancel success") { + // Act + crs.cancel() + + // Assert + assert(!crs.getIsRunning) + } + +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/JsonParserTests.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/JsonParserTests.scala new file mode 100644 index 00000000..9e367101 --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/operators/JsonParserTests.scala @@ -0,0 +1,282 @@ +package org.codefeedr.plugins.cargo.operators + +import java.util.Date + +import org.scalatest.FunSuite +import org.codefeedr.plugins.cargo.protocol.Protocol.CrateLinks +import org.codefeedr.plugins.cargo.resources.{CargoReleasesSnapshot, CrateSnapshot} +import spray.json._ + +class JsonParserTests extends FunSuite { + + val cargoReleaseSnapshot :JsObject = CargoReleasesSnapshot.snapshot.parseJson.asJsObject() + val crateSnapshot :JsObject = CrateSnapshot.snapshot.parseJson.asJsObject() + + test("getNumCratesFromSummary success") { + // Act + val numCrates :Option[Int] = JsonParser.getNumCratesOrDownloadsFromSummary(cargoReleaseSnapshot, "num_crates") + + // Assert + assert(numCrates.get == 32478) + } + + test("getNumCratesFromSummary throws with wrong input") { + // Act + val numCrates :Option[Int] = JsonParser.getNumCratesOrDownloadsFromSummary(crateSnapshot, "num_crates") + + // Assert + assert(numCrates.isEmpty) + } + + test("getNumDownloadsFromSummary success") { + // Act + val numDownloads :Option[Int] = JsonParser.getNumCratesOrDownloadsFromSummary(cargoReleaseSnapshot, "num_downloads") + + // Assert + assert(numDownloads.get == 1820298166) + } + + test("getNumDownloadsFromSummary throws with wrong input") { + // Act + val numDownloads :Option[Int] = JsonParser.getNumCratesOrDownloadsFromSummary(crateSnapshot, "num_downloads") + + // Assert + assert(numDownloads.isEmpty) + } + + test("getNewCratesFromSummary success") { + // Act + val newCrates :Option[Vector[JsObject]] = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "new_crates") + + assert(newCrates.get.size == 10) + } + + test("getNewCratesFromSummary throws with wrong input") { + // Act + val newCrates :Option[Vector[JsObject]] = JsonParser.getNewOrUpdatedCratesFromSummary(crateSnapshot, "new_crates") + + assert(newCrates.isEmpty) + } + + test("getIntFieldFromCrate success") { + // Arrange (first crate has id "dtd") + val firstCrate = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "new_crates").get(0) + + // Act + val downloads: Int = JsonParser.getIntFieldFromCrate(firstCrate, "downloads").get + + assert(downloads.isInstanceOf[Int]) + } + + test("getIntFieldFromCrate fails with non-Int field") { + // Act + val name = JsonParser.getIntFieldFromCrate(cargoReleaseSnapshot, "name") + + // Assert that name is a None type + assert(name.isEmpty) + } + + test("getStringFieldFromCrate fails when json input is not a crates") { + // Act + val id :Option[String] = JsonParser.getStringFieldFromCrate(cargoReleaseSnapshot, "id") + + assert(id.isEmpty) + } + + test("getStringFieldFromCrate success") { + // Arrange (first crate has id "dtd") + val firstCrate = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "new_crates").get(0) + + // Act + val id :Option[String] = JsonParser.getStringFieldFromCrate(firstCrate, "id") + + assert(id.get.equals("dtd")) + } + + test("getDateFieldFromCrate success") { + // Arrange + val firstCrate = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "new_crates").get(0) + + // Act + val created_at: Option[Date] = JsonParser.getDateFieldFromCrate(firstCrate, "created_at") + + assert(created_at.isDefined) + } + + test("getDateFieldFromCrate fails with wrong input") { + // Arrange + val firstCrate = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "new_crates").get(0) + + // Act + val created_at: Option[Date] = JsonParser.getDateFieldFromCrate(firstCrate, "id") + + assert(created_at.isEmpty) + } + + test("getUpdatedCratesFromSummary success") { + // Act + val updatedCrates = JsonParser.getNewOrUpdatedCratesFromSummary(cargoReleaseSnapshot, "just_updated") + + // Assert + assert(updatedCrates.isDefined) + } + + test("getUpdatedCratesFromSummary fails with wrong input") { + // Act + val updatedCrates = JsonParser.getNewOrUpdatedCratesFromSummary(crateSnapshot, "just_updated") + + // Assert + assert(updatedCrates.isEmpty) + } + + test("parseCrateJsonToCrateObject success") { + // Arrange + val crateJson = crateSnapshot.fields("crate").asJsObject() + + // Act + val crate = JsonParser.parseCrateJsonToCrateObject(crateJson) + + assert(crate.isDefined) + assert(crate.get.id.isInstanceOf[String]) + assert(crate.get.versions.isInstanceOf[List[Int]]) + assert(crate.get.keywords.isInstanceOf[List[String]]) + assert(crate.get.categories.isInstanceOf[List[String]]) + assert(crate.get.created_at.isInstanceOf[Date]) + assert(crate.get.downloads.isInstanceOf[Int]) + assert(crate.get.recent_downloads.isDefined) + assert(crate.get.max_version.isInstanceOf[String]) + assert(crate.get.description.isInstanceOf[String]) + assert(crate.get.homepage.isEmpty) + assert(crate.get.documentation.isDefined) + assert(crate.get.repository.isDefined) + assert(crate.get.links.isInstanceOf[CrateLinks]) + assert(crate.get.exact_match.isInstanceOf[Boolean]) + } + + test("parseCrateJsonToCrateObject fails with wrong input") { + // Act + val crate = JsonParser.parseCrateJsonToCrateObject(crateSnapshot) + + // Assert + assert(crate.isEmpty) + } + + test("parseCrateVersionsJsonToCrateVersionsObject success") { + // Arrange + val versionsJson = crateSnapshot.fields("versions").asInstanceOf[JsArray].elements.head.asJsObject() + + // Act + val version = JsonParser.parseCrateVersionsJsonToCrateVersionsObject(versionsJson) + + // assert + assert(version.isDefined) + } + + test("parseCrateVersionsJsonToCrateVersionsObject fails with wrong input") { + // Act + val version = JsonParser.parseCrateVersionsJsonToCrateVersionsObject(crateSnapshot) + + // Assert + assert(version.isEmpty) + } + + test("parseCrateLinksJsonToCrateLinksObject success") { + // Arrange + val crateLinksJson = crateSnapshot.fields("crate").asJsObject().fields("links").asJsObject() + + // Act + val crateLinks = JsonParser.parseCrateLinksJsonToCrateLinksObject(crateLinksJson) + + // Assert + assert(crateLinks.isDefined) + } + + test("parseCrateLinksJsonToCrateLinksObject fails with wrong input") { + // Act + val crateLinks = JsonParser.parseCrateLinksJsonToCrateLinksObject(crateSnapshot) + + // Assert + assert(crateLinks.isEmpty) + } + + test("parseCrateVersionLinksJsonToCrateVersionLinksObject success") { + // Arrange + val crateVersionLinksJson = crateSnapshot.fields("versions").asInstanceOf[JsArray] + .elements.head.asJsObject().fields("links").asJsObject() + + // Act + val crateVersionLinks = JsonParser.parseCrateVersionLinksJsonToCrateVersionLinksObject(crateVersionLinksJson) + + // Assert + assert(crateVersionLinks.isDefined) + } + + test("parseCrateVersionLinksJsonToCrateVersionLinksObject fails with wrong input"){ + // Act + val crateVersionLinks = JsonParser.parseCrateVersionLinksJsonToCrateVersionLinksObject(crateSnapshot) + + // Assert + assert(crateVersionLinks.isEmpty) + } + + test("parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject success") { + // Arrange + val crateVersionPublishedByJson = crateSnapshot.fields("versions").asInstanceOf[JsArray] + .elements.head.asJsObject().fields("published_by").asJsObject() + + // Act + val crateVersionPublishedBy = JsonParser.parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject(crateVersionPublishedByJson) + + // Assert + assert(crateVersionPublishedBy.isDefined) + } + + test("parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject fails with wrong input") { + // Act + val crateVersionPublishedBy = JsonParser.parseCrateVersionPublishedByJsonToCrateVersionPublishedByObject(crateSnapshot) + + // Assert + assert(crateVersionPublishedBy.isEmpty) + } + + test("parseCrateKeywordsJsonToCrateKeywordsObject fails with wrong input") { + // Act + val crateKeywords = JsonParser.parseCrateKeywordsJsonToCrateKeywordsObject(crateSnapshot) + + // Assert + assert(crateKeywords.isEmpty) + } + + test("parseCrateCategoryJsonToCrateCategoryObject fails with wrong input") { + // Act + val crateKeywords = JsonParser.parseCrateCategoryJsonToCrateCategoryObject(crateSnapshot) + + // Assert + assert(crateKeywords.isEmpty) + } + + test("parseCrateJsonToCrateRelease fails with wrong input") { + // Act + val crateRelease = JsonParser.parseCrateJsonToCrateRelease(cargoReleaseSnapshot) + + // Assert + assert(crateRelease.isEmpty) + } + + test("getListOfIntsFieldFromCrate fails with wrong input") { + // Act + val listOfInts = JsonParser.getListOfIntsFieldFromCrate(crateSnapshot, "bogus") + + // Assert + assert(listOfInts.isEmpty) + } + + test("getListOfStringsFieldFromCrate fails with wrong input") { + // Act + val listOfStrings = JsonParser.getListOfStringsFieldFromCrate(crateSnapshot, "bogus") + + // Assert + assert(listOfStrings.isEmpty) + } + +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/protocol/ProtocolTests.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/protocol/ProtocolTests.scala new file mode 100644 index 00000000..99362c8d --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/protocol/ProtocolTests.scala @@ -0,0 +1,168 @@ +package org.codefeedr.plugins.cargo.protocol + +import java.sql.Timestamp + +import org.scalatest.FunSuite +import org.codefeedr.plugins.cargo.protocol.Protocol._ + +class ProtocolTests extends FunSuite { + + /** Instantiate object with full fields */ + val crateVersionFeatures: CrateVersionFeatures = CrateVersionFeatures() + val crateVersionLinks: CrateVersionLinks = CrateVersionLinks("dependencies", "version_downloads", "authors") + val crateVersionPublishedBy: CrateVersionPublishedBy = CrateVersionPublishedBy(1, "login", Some("name"), "avatar", "url") + val crateVersion: CrateVersion = CrateVersion(1, "crate", "num", "dl_path", "readme_path", + new Timestamp(0), new Timestamp(0), 100, crateVersionFeatures, yanked = true, "license", crateVersionLinks, + Some(20), Some(crateVersionPublishedBy)) + val crateKeyword: CrateKeyword = CrateKeyword("id", "keyword", "created_at", 1) + val crateCategory: CrateCategory = CrateCategory("id", "category", "slug", "description", "created_at", 1) + val crateLinks: CrateLinks = CrateLinks("version_downloads", Some("versions"), "owners", + "owner_team", "owner_user", "reverse_dependencies") + val crate: Crate = Crate("id", "name", new Timestamp(0), List(1, 2), List("keyword1", "keyword2"), + List("category1", "category2"), new Timestamp(0), 100, Some(4), "max_version", + "description", Some("homepage"), Some("documentation"), Some("repository"), crateLinks, exact_match = true) + + val crateRelease: CrateRelease = CrateRelease(crate, List(crateVersion, crateVersion), List(crateKeyword, crateKeyword), + List(crateCategory, crateCategory)) + + /** Instantiate object with empty fields */ + val crateVersionPublishedByEmpty: CrateVersionPublishedBy = CrateVersionPublishedBy(1, "login", None, "avatar", "url") + val crateLinksEmpty: CrateLinks = CrateLinks("version_downloads", None, "owners", + "owner_team", "owner_user", "reverse_dependencies") + val crateEmpty: Crate = Crate("id", "name", new Timestamp(0), List(1, 2), List("keyword1", "keyword2"), + List("category1", "category2"), new Timestamp(0), 100, None, "max_version", + "description", None, None, None, crateLinksEmpty, exact_match = true) + val crateVersionEmpty: CrateVersion = CrateVersion(1, "crate", "num", "dl_path", "readme_path", + new Timestamp(0), new Timestamp(0), 100, crateVersionFeatures, yanked = true, "license", crateVersionLinks, + None, None) + + val crateReleaseEmpty: CrateRelease = CrateRelease(crateEmpty, List(crateVersionEmpty, crateVersionEmpty), + List(crateKeyword, crateKeyword), List(crateCategory, crateCategory)) + + test("CrateReleasePojo convert with all fields success") { + val pojo = CrateReleasePojo.fromCrateRelease(crateRelease) + + // Assert certain fields, e.g. Option[Int] crate.recent_downloads + assert(pojo.crate.recent_downloads == 4) + assert(pojo.crate.links.versions.equals("versions")) + assert(pojo.versions.head.published_by.login.equals("login")) + assert(pojo.keywords.head.keyword.equals("keyword")) + assert(pojo.categories.head.category.equals("category")) + } + + test("CrateReleasePojo convert with None fields success") { + val pojo = CrateReleasePojo.fromCrateRelease(crateReleaseEmpty) + + // Assert certain None fields + assert(pojo.crate.recent_downloads == 0) + assert(pojo.crate.links.versions == null) + assert(pojo.versions.head.published_by == null) + // Assert a regular field + assert(pojo.crate.name.equals("name")) + } + + test("CratePojo convert with all fields success") { + val pojo = CratePojo.fromCrate(crate) + + // Assert certain fields + assert(pojo.id.equals("id")) + assert(pojo.versions.head == 1) + assert(pojo.repository.equals("repository")) + } + + test("CratePojo convert with None fields success") { + val pojo = CratePojo.fromCrate(crateEmpty) + + // Assert certain None fields + assert(pojo.recent_downloads == 0) + assert(pojo.homepage == null) + assert(pojo.documentation == null) + assert(pojo.repository == null) + // Assert a regular field + assert(pojo.exact_match) + } + + test("CrateLinks convert with all fields success") { + val pojo = CrateLinksPojo.fromCrateLinks(crateLinks) + + // Assert certain fields + assert(pojo.version_downloads.equals("version_downloads")) + assert(pojo.versions.equals("versions")) + assert(pojo.owner_user.equals("owner_user")) + } + + test("CrateLinks convert with None fields success") { + val pojo = CrateLinksPojo.fromCrateLinks(crateLinksEmpty) + + // Assert None field + assert(pojo.versions == null) + // Assert regular field + assert(pojo.owner_team.equals("owner_team")) + } + + test("CrateVersion convert with all fields success") { + val pojo = CrateVersionPojo.fromCrateVersion(crateVersion) + + // Assert certain fields + assert(pojo.id == 1) + assert(pojo.yanked) + assert(pojo.license.equals("license")) + } + + test("CrateVersionFeatures convert success") { + val pojo = CrateVersionFeaturesPojo.fromCrateVersionFeatures() + + // Assert type correct + assert(pojo.isInstanceOf[CrateVersionFeaturesPojo]) + } + + test("CrateVersionLinks with all fields success") { + val pojo = CrateVersionLinksPojo.fromCrateVersionLinks(crateVersionLinks) + + // Assert certain fields + assert(pojo.dependencies.equals("dependencies")) + assert(pojo.version_downloads.equals("version_downloads")) + assert(pojo.authors.equals("authors")) + } + + test("CrateVersionPublishedBy with all fields success") { + val pojo = CrateVersionPublishedByPojo.fromCrateVersionPublishedBy(crateVersionPublishedBy) + + // Assert certain fields + assert(pojo.id == 1) + assert(pojo.login.equals("login")) + assert(pojo.name.equals("name")) + assert(pojo.avatar.equals("avatar")) + assert(pojo.url.equals("url")) + } + + test("CrateVersionPublishedBy with None fields") { + val pojo = CrateVersionPublishedByPojo.fromCrateVersionPublishedBy(crateVersionPublishedByEmpty) + + // Assert None field + assert(pojo.name == null) + } + + test("CrateKeyword with all fields success") { + val pojo = CrateKeywordPojo.fromCrateKeyword(crateKeyword) + + // Assert fields + assert(pojo.id.equals("id")) + assert(pojo.keyword.equals("keyword")) + assert(pojo.created_at.equals("created_at")) + assert(pojo.crates_cnt == 1) + } + + test("CrateCategory with all fields success") { + val pojo = CrateCategoryPojo.fromCrateCategory(crateCategory) + + // Assert fields + assert(pojo.id.equals("id")) + assert(pojo.category.equals("category")) + assert(pojo.slug.equals("slug")) + assert(pojo.description.equals("description")) + assert(pojo.created_at.equals("created_at")) + assert(pojo.crates_cnt == 1) + } + +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CargoReleasesSnapshot.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CargoReleasesSnapshot.scala new file mode 100644 index 00000000..a3fe6eab --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CargoReleasesSnapshot.scala @@ -0,0 +1,6 @@ +package org.codefeedr.plugins.cargo.resources + +object CargoReleasesSnapshot { + // snapshot taken from https://crates.io/api/v1/summary at 11/27/2019 11:25AM + val snapshot = """{"num_downloads":1820298166,"num_crates":32478,"new_crates":[{"id":"dtd","name":"dtd","updated_at":"2019-11-27T09:55:26.851957+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:55:26.851957+00:00","downloads":2,"recent_downloads":null,"max_version":"0.1.0","description":"A parser for XML DTD files","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/dtd/downloads","versions":"/api/v1/crates/dtd/versions","owners":"/api/v1/crates/dtd/owners","owner_team":"/api/v1/crates/dtd/owner_team","owner_user":"/api/v1/crates/dtd/owner_user","reverse_dependencies":"/api/v1/crates/dtd/reverse_dependencies"},"exact_match":false},{"id":"fetch_configuration","name":"fetch_configuration","updated_at":"2019-11-27T09:52:32.840911+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:52:32.840911+00:00","downloads":3,"recent_downloads":null,"max_version":"0.1.0","description":"Library to fetch configuration from config file.","homepage":null,"documentation":null,"repository":"https://github.com/PankajChaudhary5/fetch_configuration.git","links":{"version_downloads":"/api/v1/crates/fetch_configuration/downloads","versions":"/api/v1/crates/fetch_configuration/versions","owners":"/api/v1/crates/fetch_configuration/owners","owner_team":"/api/v1/crates/fetch_configuration/owner_team","owner_user":"/api/v1/crates/fetch_configuration/owner_user","reverse_dependencies":"/api/v1/crates/fetch_configuration/reverse_dependencies"},"exact_match":false},{"id":"randomx-rs","name":"randomx-rs","updated_at":"2019-11-27T09:48:55.652617+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:48:55.652617+00:00","downloads":2,"recent_downloads":null,"max_version":"0.1.1","description":"Rust bindings for the RandomX Proof-of-Work","homepage":"https://tari.com","documentation":null,"repository":"https://github.com/tari-project/randomx-rs","links":{"version_downloads":"/api/v1/crates/randomx-rs/downloads","versions":"/api/v1/crates/randomx-rs/versions","owners":"/api/v1/crates/randomx-rs/owners","owner_team":"/api/v1/crates/randomx-rs/owner_team","owner_user":"/api/v1/crates/randomx-rs/owner_user","reverse_dependencies":"/api/v1/crates/randomx-rs/reverse_dependencies"},"exact_match":false},{"id":"shepherd","name":"shepherd","updated_at":"2019-11-27T09:35:00.035849+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:35:00.035849+00:00","downloads":3,"recent_downloads":null,"max_version":"0.1.0","description":"A distributed video encoder that splits files into chunks to encode them on\nmultiple machines in parallel.\n","homepage":null,"documentation":null,"repository":"https://github.com/martindisch/shepherd","links":{"version_downloads":"/api/v1/crates/shepherd/downloads","versions":"/api/v1/crates/shepherd/versions","owners":"/api/v1/crates/shepherd/owners","owner_team":"/api/v1/crates/shepherd/owner_team","owner_user":"/api/v1/crates/shepherd/owner_user","reverse_dependencies":"/api/v1/crates/shepherd/reverse_dependencies"},"exact_match":false},{"id":"minigrep0444","name":"minigrep0444","updated_at":"2019-11-27T09:25:31.884380+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:25:31.884380+00:00","downloads":2,"recent_downloads":null,"max_version":"0.1.0","description":"A fun game where you guess what number the computer has chosen.","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/minigrep0444/downloads","versions":"/api/v1/crates/minigrep0444/versions","owners":"/api/v1/crates/minigrep0444/owners","owner_team":"/api/v1/crates/minigrep0444/owner_team","owner_user":"/api/v1/crates/minigrep0444/owner_user","reverse_dependencies":"/api/v1/crates/minigrep0444/reverse_dependencies"},"exact_match":false},{"id":"ogl33","name":"ogl33","updated_at":"2019-11-27T09:31:05.674236+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:25:01.774728+00:00","downloads":3,"recent_downloads":null,"max_version":"0.1.2","description":"OpenGL 3.3 loader/bindings","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/ogl33/downloads","versions":"/api/v1/crates/ogl33/versions","owners":"/api/v1/crates/ogl33/owners","owner_team":"/api/v1/crates/ogl33/owner_team","owner_user":"/api/v1/crates/ogl33/owner_user","reverse_dependencies":"/api/v1/crates/ogl33/reverse_dependencies"},"exact_match":false},{"id":"libcvss","name":"libcvss","updated_at":"2019-11-27T09:07:38.661999+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:07:38.661999+00:00","downloads":2,"recent_downloads":null,"max_version":"1.0.0-rc.2","description":"A Rust implementation of the CVSS specification.","homepage":"https://crates.io/crates/libcvss","documentation":"https://docs.rs/crate/libcvss/","repository":"https://gitlab.inria.fr/celbaz/libcvss","links":{"version_downloads":"/api/v1/crates/libcvss/downloads","versions":"/api/v1/crates/libcvss/versions","owners":"/api/v1/crates/libcvss/owners","owner_team":"/api/v1/crates/libcvss/owner_team","owner_user":"/api/v1/crates/libcvss/owner_user","reverse_dependencies":"/api/v1/crates/libcvss/reverse_dependencies"},"exact_match":false},{"id":"fairy-macros","name":"fairy-macros","updated_at":"2019-11-27T06:25:15.584254+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T06:25:15.584254+00:00","downloads":9,"recent_downloads":null,"max_version":"0.0.0","description":"Reserved","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/fairy-macros/downloads","versions":"/api/v1/crates/fairy-macros/versions","owners":"/api/v1/crates/fairy-macros/owners","owner_team":"/api/v1/crates/fairy-macros/owner_team","owner_user":"/api/v1/crates/fairy-macros/owner_user","reverse_dependencies":"/api/v1/crates/fairy-macros/reverse_dependencies"},"exact_match":false},{"id":"fairy","name":"fairy","updated_at":"2019-11-27T06:25:08.836317+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T06:25:08.836317+00:00","downloads":9,"recent_downloads":null,"max_version":"0.0.0","description":"Reserved","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/fairy/downloads","versions":"/api/v1/crates/fairy/versions","owners":"/api/v1/crates/fairy/owners","owner_team":"/api/v1/crates/fairy/owner_team","owner_user":"/api/v1/crates/fairy/owner_user","reverse_dependencies":"/api/v1/crates/fairy/reverse_dependencies"},"exact_match":false},{"id":"sqlx-sqlite","name":"sqlx-sqlite","updated_at":"2019-11-27T06:25:00.214630+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T06:25:00.214630+00:00","downloads":9,"recent_downloads":null,"max_version":"0.0.0","description":"Reserved","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/sqlx-sqlite/downloads","versions":"/api/v1/crates/sqlx-sqlite/versions","owners":"/api/v1/crates/sqlx-sqlite/owners","owner_team":"/api/v1/crates/sqlx-sqlite/owner_team","owner_user":"/api/v1/crates/sqlx-sqlite/owner_user","reverse_dependencies":"/api/v1/crates/sqlx-sqlite/reverse_dependencies"},"exact_match":false}],"most_downloaded":[{"id":"rand","name":"rand","updated_at":"2019-09-16T20:09:24.860231+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-02-03T06:17:14.147783+00:00","downloads":23834706,"recent_downloads":null,"max_version":"0.7.2","description":"Random number generators and other randomness functionality.\n","homepage":"https://crates.io/crates/rand","documentation":"https://rust-random.github.io/rand/","repository":"https://github.com/rust-random/rand","links":{"version_downloads":"/api/v1/crates/rand/downloads","versions":"/api/v1/crates/rand/versions","owners":"/api/v1/crates/rand/owners","owner_team":"/api/v1/crates/rand/owner_team","owner_user":"/api/v1/crates/rand/owner_user","reverse_dependencies":"/api/v1/crates/rand/reverse_dependencies"},"exact_match":false},{"id":"libc","name":"libc","updated_at":"2019-10-18T08:17:25.326162+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-01-15T20:22:13.100871+00:00","downloads":20374094,"recent_downloads":null,"max_version":"0.2.65","description":"Raw FFI bindings to platform libraries like libc.\n","homepage":"https://github.com/rust-lang/libc","documentation":"http://doc.rust-lang.org/libc","repository":"https://github.com/rust-lang/libc","links":{"version_downloads":"/api/v1/crates/libc/downloads","versions":"/api/v1/crates/libc/versions","owners":"/api/v1/crates/libc/owners","owner_team":"/api/v1/crates/libc/owner_team","owner_user":"/api/v1/crates/libc/owner_user","reverse_dependencies":"/api/v1/crates/libc/reverse_dependencies"},"exact_match":false},{"id":"bitflags","name":"bitflags","updated_at":"2019-10-13T23:21:24.597863+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-01-15T08:35:24.901276+00:00","downloads":18834902,"recent_downloads":null,"max_version":"1.2.1","description":"A macro to generate structures which behave like bitflags.\n","homepage":"https://github.com/bitflags/bitflags","documentation":"https://docs.rs/bitflags","repository":"https://github.com/bitflags/bitflags","links":{"version_downloads":"/api/v1/crates/bitflags/downloads","versions":"/api/v1/crates/bitflags/versions","owners":"/api/v1/crates/bitflags/owners","owner_team":"/api/v1/crates/bitflags/owner_team","owner_user":"/api/v1/crates/bitflags/owner_user","reverse_dependencies":"/api/v1/crates/bitflags/reverse_dependencies"},"exact_match":false},{"id":"syn","name":"syn","updated_at":"2019-11-08T19:29:50.485524+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2016-09-07T15:22:40.491981+00:00","downloads":17257094,"recent_downloads":null,"max_version":"1.0.8","description":"Parser for Rust source code","homepage":null,"documentation":"https://docs.rs/syn","repository":"https://github.com/dtolnay/syn","links":{"version_downloads":"/api/v1/crates/syn/downloads","versions":"/api/v1/crates/syn/versions","owners":"/api/v1/crates/syn/owners","owner_team":"/api/v1/crates/syn/owner_team","owner_user":"/api/v1/crates/syn/owner_user","reverse_dependencies":"/api/v1/crates/syn/reverse_dependencies"},"exact_match":false},{"id":"lazy_static","name":"lazy_static","updated_at":"2019-08-26T00:55:06.276184+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-11-23T01:18:10.993977+00:00","downloads":17159108,"recent_downloads":null,"max_version":"1.4.0","description":"A macro for declaring lazily evaluated statics in Rust.","homepage":null,"documentation":"https://docs.rs/lazy_static","repository":"https://github.com/rust-lang-nursery/lazy-static.rs","links":{"version_downloads":"/api/v1/crates/lazy_static/downloads","versions":"/api/v1/crates/lazy_static/versions","owners":"/api/v1/crates/lazy_static/owners","owner_team":"/api/v1/crates/lazy_static/owner_team","owner_user":"/api/v1/crates/lazy_static/owner_user","reverse_dependencies":"/api/v1/crates/lazy_static/reverse_dependencies"},"exact_match":false},{"id":"log","name":"log","updated_at":"2019-07-28T21:04:08.549810+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-12-13T22:10:19.944550+00:00","downloads":16819457,"recent_downloads":null,"max_version":"0.4.8","description":"A lightweight logging facade for Rust\n","homepage":null,"documentation":"https://docs.rs/log","repository":"https://github.com/rust-lang/log","links":{"version_downloads":"/api/v1/crates/log/downloads","versions":"/api/v1/crates/log/versions","owners":"/api/v1/crates/log/owners","owner_team":"/api/v1/crates/log/owner_team","owner_user":"/api/v1/crates/log/owner_user","reverse_dependencies":"/api/v1/crates/log/reverse_dependencies"},"exact_match":false},{"id":"serde","name":"serde","updated_at":"2019-11-25T00:16:10.820610+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-12-05T20:20:39.487502+00:00","downloads":15828742,"recent_downloads":null,"max_version":"1.0.103","description":"A generic serialization/deserialization framework","homepage":"https://serde.rs","documentation":"https://docs.serde.rs/serde/","repository":"https://github.com/serde-rs/serde","links":{"version_downloads":"/api/v1/crates/serde/downloads","versions":"/api/v1/crates/serde/versions","owners":"/api/v1/crates/serde/owners","owner_team":"/api/v1/crates/serde/owner_team","owner_user":"/api/v1/crates/serde/owner_user","reverse_dependencies":"/api/v1/crates/serde/reverse_dependencies"},"exact_match":false},{"id":"quote","name":"quote","updated_at":"2019-08-18T02:57:39.430996+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2016-09-03T05:42:05.484129+00:00","downloads":15285829,"recent_downloads":null,"max_version":"1.0.2","description":"Quasi-quoting macro quote!(...)","homepage":null,"documentation":"https://docs.rs/quote/","repository":"https://github.com/dtolnay/quote","links":{"version_downloads":"/api/v1/crates/quote/downloads","versions":"/api/v1/crates/quote/versions","owners":"/api/v1/crates/quote/owners","owner_team":"/api/v1/crates/quote/owner_team","owner_user":"/api/v1/crates/quote/owner_user","reverse_dependencies":"/api/v1/crates/quote/reverse_dependencies"},"exact_match":false},{"id":"regex-syntax","name":"regex-syntax","updated_at":"2019-09-03T16:52:52.627723+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-05-27T23:19:16.839117+00:00","downloads":14094189,"recent_downloads":null,"max_version":"0.6.12","description":"A regular expression parser.","homepage":"https://github.com/rust-lang/regex","documentation":"https://docs.rs/regex-syntax","repository":"https://github.com/rust-lang/regex","links":{"version_downloads":"/api/v1/crates/regex-syntax/downloads","versions":"/api/v1/crates/regex-syntax/versions","owners":"/api/v1/crates/regex-syntax/owners","owner_team":"/api/v1/crates/regex-syntax/owner_team","owner_user":"/api/v1/crates/regex-syntax/owner_user","reverse_dependencies":"/api/v1/crates/regex-syntax/reverse_dependencies"},"exact_match":false},{"id":"regex","name":"regex","updated_at":"2019-09-04T10:47:34.929514+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-12-13T22:10:11.303311+00:00","downloads":13942643,"recent_downloads":null,"max_version":"1.3.1","description":"An implementation of regular expressions for Rust. This implementation uses\nfinite automata and guarantees linear time matching on all inputs.\n","homepage":"https://github.com/rust-lang/regex","documentation":"https://docs.rs/regex","repository":"https://github.com/rust-lang/regex","links":{"version_downloads":"/api/v1/crates/regex/downloads","versions":"/api/v1/crates/regex/versions","owners":"/api/v1/crates/regex/owners","owner_team":"/api/v1/crates/regex/owner_team","owner_user":"/api/v1/crates/regex/owner_user","reverse_dependencies":"/api/v1/crates/regex/reverse_dependencies"},"exact_match":false}],"most_recently_downloaded":[{"id":"rand","name":"rand","updated_at":"2019-09-16T20:09:24.860231+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-02-03T06:17:14.147783+00:00","downloads":23834706,"recent_downloads":null,"max_version":"0.7.2","description":"Random number generators and other randomness functionality.\n","homepage":"https://crates.io/crates/rand","documentation":"https://rust-random.github.io/rand/","repository":"https://github.com/rust-random/rand","links":{"version_downloads":"/api/v1/crates/rand/downloads","versions":"/api/v1/crates/rand/versions","owners":"/api/v1/crates/rand/owners","owner_team":"/api/v1/crates/rand/owner_team","owner_user":"/api/v1/crates/rand/owner_user","reverse_dependencies":"/api/v1/crates/rand/reverse_dependencies"},"exact_match":false},{"id":"rand_core","name":"rand_core","updated_at":"2019-08-28T11:51:48.451487+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2017-09-14T12:02:18.709038+00:00","downloads":13291508,"recent_downloads":null,"max_version":"0.5.1","description":"Core random number generator traits and tools for implementation.\n","homepage":"https://crates.io/crates/rand_core","documentation":"https://rust-random.github.io/rand/rand_core/","repository":"https://github.com/rust-random/rand","links":{"version_downloads":"/api/v1/crates/rand_core/downloads","versions":"/api/v1/crates/rand_core/versions","owners":"/api/v1/crates/rand_core/owners","owner_team":"/api/v1/crates/rand_core/owner_team","owner_user":"/api/v1/crates/rand_core/owner_user","reverse_dependencies":"/api/v1/crates/rand_core/reverse_dependencies"},"exact_match":false},{"id":"syn","name":"syn","updated_at":"2019-11-08T19:29:50.485524+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2016-09-07T15:22:40.491981+00:00","downloads":17257094,"recent_downloads":null,"max_version":"1.0.8","description":"Parser for Rust source code","homepage":null,"documentation":"https://docs.rs/syn","repository":"https://github.com/dtolnay/syn","links":{"version_downloads":"/api/v1/crates/syn/downloads","versions":"/api/v1/crates/syn/versions","owners":"/api/v1/crates/syn/owners","owner_team":"/api/v1/crates/syn/owner_team","owner_user":"/api/v1/crates/syn/owner_user","reverse_dependencies":"/api/v1/crates/syn/reverse_dependencies"},"exact_match":false},{"id":"proc-macro2","name":"proc-macro2","updated_at":"2019-10-19T21:52:53.663568+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2017-07-06T01:01:58.064887+00:00","downloads":11749781,"recent_downloads":null,"max_version":"1.0.6","description":"A stable implementation of the upcoming new `proc_macro` API. Comes with an\noption, off by default, to also reimplement itself in terms of the upstream\nunstable API.\n","homepage":"https://github.com/alexcrichton/proc-macro2","documentation":"https://docs.rs/proc-macro2","repository":"https://github.com/alexcrichton/proc-macro2","links":{"version_downloads":"/api/v1/crates/proc-macro2/downloads","versions":"/api/v1/crates/proc-macro2/versions","owners":"/api/v1/crates/proc-macro2/owners","owner_team":"/api/v1/crates/proc-macro2/owner_team","owner_user":"/api/v1/crates/proc-macro2/owner_user","reverse_dependencies":"/api/v1/crates/proc-macro2/reverse_dependencies"},"exact_match":false},{"id":"quote","name":"quote","updated_at":"2019-08-18T02:57:39.430996+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2016-09-03T05:42:05.484129+00:00","downloads":15285829,"recent_downloads":null,"max_version":"1.0.2","description":"Quasi-quoting macro quote!(...)","homepage":null,"documentation":"https://docs.rs/quote/","repository":"https://github.com/dtolnay/quote","links":{"version_downloads":"/api/v1/crates/quote/downloads","versions":"/api/v1/crates/quote/versions","owners":"/api/v1/crates/quote/owners","owner_team":"/api/v1/crates/quote/owner_team","owner_user":"/api/v1/crates/quote/owner_user","reverse_dependencies":"/api/v1/crates/quote/reverse_dependencies"},"exact_match":false},{"id":"unicode-xid","name":"unicode-xid","updated_at":"2019-07-25T13:26:11.852698+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-04-27T01:12:08.811826+00:00","downloads":13936377,"recent_downloads":null,"max_version":"0.2.0","description":"Determine whether characters have the XID_Start\nor XID_Continue properties according to\nUnicode Standard Annex #31.\n","homepage":"https://github.com/unicode-rs/unicode-xid","documentation":"https://unicode-rs.github.io/unicode-xid","repository":"https://github.com/unicode-rs/unicode-xid","links":{"version_downloads":"/api/v1/crates/unicode-xid/downloads","versions":"/api/v1/crates/unicode-xid/versions","owners":"/api/v1/crates/unicode-xid/owners","owner_team":"/api/v1/crates/unicode-xid/owner_team","owner_user":"/api/v1/crates/unicode-xid/owner_user","reverse_dependencies":"/api/v1/crates/unicode-xid/reverse_dependencies"},"exact_match":false},{"id":"libc","name":"libc","updated_at":"2019-10-18T08:17:25.326162+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-01-15T20:22:13.100871+00:00","downloads":20374094,"recent_downloads":null,"max_version":"0.2.65","description":"Raw FFI bindings to platform libraries like libc.\n","homepage":"https://github.com/rust-lang/libc","documentation":"http://doc.rust-lang.org/libc","repository":"https://github.com/rust-lang/libc","links":{"version_downloads":"/api/v1/crates/libc/downloads","versions":"/api/v1/crates/libc/versions","owners":"/api/v1/crates/libc/owners","owner_team":"/api/v1/crates/libc/owner_team","owner_user":"/api/v1/crates/libc/owner_user","reverse_dependencies":"/api/v1/crates/libc/reverse_dependencies"},"exact_match":false},{"id":"lazy_static","name":"lazy_static","updated_at":"2019-08-26T00:55:06.276184+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-11-23T01:18:10.993977+00:00","downloads":17159108,"recent_downloads":null,"max_version":"1.4.0","description":"A macro for declaring lazily evaluated statics in Rust.","homepage":null,"documentation":"https://docs.rs/lazy_static","repository":"https://github.com/rust-lang-nursery/lazy-static.rs","links":{"version_downloads":"/api/v1/crates/lazy_static/downloads","versions":"/api/v1/crates/lazy_static/versions","owners":"/api/v1/crates/lazy_static/owners","owner_team":"/api/v1/crates/lazy_static/owner_team","owner_user":"/api/v1/crates/lazy_static/owner_user","reverse_dependencies":"/api/v1/crates/lazy_static/reverse_dependencies"},"exact_match":false},{"id":"serde","name":"serde","updated_at":"2019-11-25T00:16:10.820610+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2014-12-05T20:20:39.487502+00:00","downloads":15828742,"recent_downloads":null,"max_version":"1.0.103","description":"A generic serialization/deserialization framework","homepage":"https://serde.rs","documentation":"https://docs.serde.rs/serde/","repository":"https://github.com/serde-rs/serde","links":{"version_downloads":"/api/v1/crates/serde/downloads","versions":"/api/v1/crates/serde/versions","owners":"/api/v1/crates/serde/owners","owner_team":"/api/v1/crates/serde/owner_team","owner_user":"/api/v1/crates/serde/owner_user","reverse_dependencies":"/api/v1/crates/serde/reverse_dependencies"},"exact_match":false},{"id":"bitflags","name":"bitflags","updated_at":"2019-10-13T23:21:24.597863+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2015-01-15T08:35:24.901276+00:00","downloads":18834902,"recent_downloads":null,"max_version":"1.2.1","description":"A macro to generate structures which behave like bitflags.\n","homepage":"https://github.com/bitflags/bitflags","documentation":"https://docs.rs/bitflags","repository":"https://github.com/bitflags/bitflags","links":{"version_downloads":"/api/v1/crates/bitflags/downloads","versions":"/api/v1/crates/bitflags/versions","owners":"/api/v1/crates/bitflags/owners","owner_team":"/api/v1/crates/bitflags/owner_team","owner_user":"/api/v1/crates/bitflags/owner_user","reverse_dependencies":"/api/v1/crates/bitflags/reverse_dependencies"},"exact_match":false}],"just_updated":[{"id":"bme680","name":"bme680","updated_at":"2019-11-27T10:11:36.413761+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2018-05-25T21:10:02.690185+00:00","downloads":1409,"recent_downloads":null,"max_version":"0.5.0","description":"A pure Rust implementation for the BME680 environmental sensor.","homepage":null,"documentation":"https://docs.rs/bme680","repository":"https://github.com/marcelbuesing/bme680-hal","links":{"version_downloads":"/api/v1/crates/bme680/downloads","versions":"/api/v1/crates/bme680/versions","owners":"/api/v1/crates/bme680/owners","owner_team":"/api/v1/crates/bme680/owner_team","owner_user":"/api/v1/crates/bme680/owner_user","reverse_dependencies":"/api/v1/crates/bme680/reverse_dependencies"},"exact_match":false},{"id":"rerast","name":"rerast","updated_at":"2019-11-27T10:04:04.848324+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2017-10-31T07:50:37.856862+00:00","downloads":8957,"recent_downloads":null,"max_version":"0.1.75","description":"An AST (abstract syntax tree) based search replace tool for code written in Rust","homepage":null,"documentation":null,"repository":"https://github.com/google/rerast","links":{"version_downloads":"/api/v1/crates/rerast/downloads","versions":"/api/v1/crates/rerast/versions","owners":"/api/v1/crates/rerast/owners","owner_team":"/api/v1/crates/rerast/owner_team","owner_user":"/api/v1/crates/rerast/owner_user","reverse_dependencies":"/api/v1/crates/rerast/reverse_dependencies"},"exact_match":false},{"id":"safe-nd","name":"safe-nd","updated_at":"2019-11-27T09:56:38.363447+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-05-20T13:15:06.605852+00:00","downloads":7186,"recent_downloads":null,"max_version":"0.6.0","description":"Safe Network Data Types","homepage":"https://maidsafe.net","documentation":null,"repository":"https://github.com/maidsafe/safe-nd","links":{"version_downloads":"/api/v1/crates/safe-nd/downloads","versions":"/api/v1/crates/safe-nd/versions","owners":"/api/v1/crates/safe-nd/owners","owner_team":"/api/v1/crates/safe-nd/owner_team","owner_user":"/api/v1/crates/safe-nd/owner_user","reverse_dependencies":"/api/v1/crates/safe-nd/reverse_dependencies"},"exact_match":false},{"id":"polona-dl","name":"polona-dl","updated_at":"2019-11-27T09:56:05.080062+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-08-22T23:06:30.125555+00:00","downloads":79,"recent_downloads":null,"max_version":"0.1.1","description":"Narzędzie linii poleceń do ściągania obrazów z bazy zbiorów Biblioteki Narodowej Polona.","homepage":null,"documentation":null,"repository":"https://github.com/jakubadamw/polona_dl","links":{"version_downloads":"/api/v1/crates/polona-dl/downloads","versions":"/api/v1/crates/polona-dl/versions","owners":"/api/v1/crates/polona-dl/owners","owner_team":"/api/v1/crates/polona-dl/owner_team","owner_user":"/api/v1/crates/polona-dl/owner_user","reverse_dependencies":"/api/v1/crates/polona-dl/reverse_dependencies"},"exact_match":false},{"id":"odbc","name":"odbc","updated_at":"2019-11-27T09:42:51.718280+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2016-02-19T13:51:24.446669+00:00","downloads":17671,"recent_downloads":null,"max_version":"0.16.0","description":"ODBC wrapper for idiomatic Rust","homepage":"https://github.com/Koka/odbc-rs","documentation":"http://koka.github.io/odbc-rs/odbc/","repository":"https://github.com/Koka/odbc-rs","links":{"version_downloads":"/api/v1/crates/odbc/downloads","versions":"/api/v1/crates/odbc/versions","owners":"/api/v1/crates/odbc/owners","owner_team":"/api/v1/crates/odbc/owner_team","owner_user":"/api/v1/crates/odbc/owner_user","reverse_dependencies":"/api/v1/crates/odbc/reverse_dependencies"},"exact_match":false},{"id":"ogl33","name":"ogl33","updated_at":"2019-11-27T09:31:05.674236+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-11-27T09:25:01.774728+00:00","downloads":3,"recent_downloads":null,"max_version":"0.1.2","description":"OpenGL 3.3 loader/bindings","homepage":null,"documentation":null,"repository":null,"links":{"version_downloads":"/api/v1/crates/ogl33/downloads","versions":"/api/v1/crates/ogl33/versions","owners":"/api/v1/crates/ogl33/owners","owner_team":"/api/v1/crates/ogl33/owner_team","owner_user":"/api/v1/crates/ogl33/owner_user","reverse_dependencies":"/api/v1/crates/ogl33/reverse_dependencies"},"exact_match":false},{"id":"cplex-sys","name":"cplex-sys","updated_at":"2019-11-27T09:19:30.736281+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2017-08-01T12:26:21.852155+00:00","downloads":1474,"recent_downloads":null,"max_version":"0.6.0","description":"Low level bindings to the Cplex C-API","homepage":"https://chiselapp.com/user/fifr/repository/cplex-sys","documentation":"http://fifr.spdns.de/doc/cplex_sys","repository":"https://chiselapp.com/user/fifr/repository/cplex-sys","links":{"version_downloads":"/api/v1/crates/cplex-sys/downloads","versions":"/api/v1/crates/cplex-sys/versions","owners":"/api/v1/crates/cplex-sys/owners","owner_team":"/api/v1/crates/cplex-sys/owner_team","owner_user":"/api/v1/crates/cplex-sys/owner_user","reverse_dependencies":"/api/v1/crates/cplex-sys/reverse_dependencies"},"exact_match":false},{"id":"backoff-futures","name":"backoff-futures","updated_at":"2019-11-27T08:57:06.005298+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-08-15T14:07:07.691274+00:00","downloads":288,"recent_downloads":null,"max_version":"0.2.0","description":"A retry and backoff mechanism for `std::future::Future`","homepage":null,"documentation":null,"repository":"https://github.com/jakubadamw/backoff-futures","links":{"version_downloads":"/api/v1/crates/backoff-futures/downloads","versions":"/api/v1/crates/backoff-futures/versions","owners":"/api/v1/crates/backoff-futures/owners","owner_team":"/api/v1/crates/backoff-futures/owner_team","owner_user":"/api/v1/crates/backoff-futures/owner_user","reverse_dependencies":"/api/v1/crates/backoff-futures/reverse_dependencies"},"exact_match":false},{"id":"staticvec","name":"staticvec","updated_at":"2019-11-27T08:33:17.433001+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-07-30T00:09:10.697232+00:00","downloads":1736,"recent_downloads":null,"max_version":"0.4.7","description":"Implements a fixed-capacity stack-allocated Vec alternative backed by an array, using const generics.","homepage":null,"documentation":"https://docs.rs/staticvec/","repository":"https://github.com/slightlyoutofphase/staticvec","links":{"version_downloads":"/api/v1/crates/staticvec/downloads","versions":"/api/v1/crates/staticvec/versions","owners":"/api/v1/crates/staticvec/owners","owner_team":"/api/v1/crates/staticvec/owner_team","owner_user":"/api/v1/crates/staticvec/owner_user","reverse_dependencies":"/api/v1/crates/staticvec/reverse_dependencies"},"exact_match":false},{"id":"opennode-client","name":"opennode-client","updated_at":"2019-11-27T08:27:20.152711+00:00","versions":null,"keywords":null,"categories":null,"badges":null,"created_at":"2019-10-15T07:40:02.769207+00:00","downloads":40,"recent_downloads":null,"max_version":"1.0.0","description":"Client for the Opennode v1 HTTP API","homepage":null,"documentation":"https://docs.rs/opennode-client/","repository":"https://github.com/edouardparis/opennode-rs.git","links":{"version_downloads":"/api/v1/crates/opennode-client/downloads","versions":"/api/v1/crates/opennode-client/versions","owners":"/api/v1/crates/opennode-client/owners","owner_team":"/api/v1/crates/opennode-client/owner_team","owner_user":"/api/v1/crates/opennode-client/owner_user","reverse_dependencies":"/api/v1/crates/opennode-client/reverse_dependencies"},"exact_match":false}],"popular_keywords":[{"id":"cli","keyword":"cli","created_at":"2014-12-13T23:17:17.651770+00:00","crates_cnt":783},{"id":"ffi","keyword":"ffi","created_at":"2014-11-21T14:53:31.372632+00:00","crates_cnt":721},{"id":"windows","keyword":"windows","created_at":"2015-05-23T17:13:20.460427+00:00","crates_cnt":546},{"id":"api","keyword":"api","created_at":"2014-11-29T17:51:55.405929+00:00","crates_cnt":544},{"id":"web","keyword":"web","created_at":"2014-11-29T13:11:08.855945+00:00","crates_cnt":515},{"id":"google","keyword":"google","created_at":"2015-02-27T09:40:12.784768+00:00","crates_cnt":470},{"id":"parser","keyword":"parser","created_at":"2014-11-14T20:00:40.785931+00:00","crates_cnt":429},{"id":"winsdk","keyword":"winsdk","created_at":"2015-01-26T18:05:11.957648+00:00","crates_cnt":393},{"id":"protocol","keyword":"protocol","created_at":"2014-11-20T20:14:31.702804+00:00","crates_cnt":373},{"id":"crypto","keyword":"crypto","created_at":"2014-11-24T02:30:36.413823+00:00","crates_cnt":325}],"popular_categories":[{"id":"development-tools","category":"Development tools","slug":"development-tools","description":"Crates that provide developer-facing features such as testing, debugging, linting, performance profiling, autocompletion, formatting, and more.","created_at":"2016-12-30T23:42:37.777395+00:00","crates_cnt":1265},{"id":"no-std","category":"No standard library","slug":"no-std","description":"Crates that are able to function without the Rust standard library.\n","created_at":"2017-02-10T01:52:09.447906+00:00","crates_cnt":1201},{"id":"command-line-utilities","category":"Command line utilities","slug":"command-line-utilities","description":"Applications to run at the command line.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":1164},{"id":"api-bindings","category":"API bindings","slug":"api-bindings","description":"Idiomatic wrappers of specific APIs for convenient access from Rust. Includes HTTP API wrappers as well. Non-idiomatic or unsafe bindings can be found in External FFI bindings.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":870},{"id":"data-structures","category":"Data structures","slug":"data-structures","description":"Rust implementations of particular ways of organizing data suited for specific purposes.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":774},{"id":"network-programming","category":"Network programming","slug":"network-programming","description":"Crates dealing with higher-level network protocols such as FTP, HTTP, or SSH, or lower-level network protocols such as TCP or UDP.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":744},{"id":"web-programming","category":"Web programming","slug":"web-programming","description":"Crates to create applications for the web.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":712},{"id":"embedded","category":"Embedded development","slug":"embedded","description":"Crates that are primarily useful on embedded devices or without an operating system.\n","created_at":"2017-02-05T21:31:23.444608+00:00","crates_cnt":644},{"id":"algorithms","category":"Algorithms","slug":"algorithms","description":"Rust implementations of core algorithms such as hashing, sorting, searching, and more.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":598},{"id":"asynchronous","category":"Asynchronous","slug":"asynchronous","description":"Crates to help you deal with events independently of the main program flow, using techniques like futures, promises, waiting, or eventing.","created_at":"2017-01-17T19:13:05.112025+00:00","crates_cnt":595}]}""" +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CrateSnapshot.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CrateSnapshot.scala new file mode 100644 index 00000000..8942aa2e --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/resources/CrateSnapshot.scala @@ -0,0 +1,6 @@ +package org.codefeedr.plugins.cargo.resources + +object CrateSnapshot { + // Taken from https://crates.io/api/v1/crates/json on 27/11/2019 11:42PM + val snapshot = """{"crate":{"id":"json","name":"json","updated_at":"2019-09-06T16:43:25.950377+00:00","versions":[174509,168892,154478,77794,70917,70910,67655,60919,58613,58053,47860,42931,41398,38809,38454,38336,36807,36734,31597,31124,30599,30354,30119,29810,29702,29655,29558,29389,29388,29377,29291,29207,29203,29127,29037,29005,28948,28917,28886,28858,28821,28798,28739,28689,28627,28624,28607,28513,28508,28506],"keywords":[],"categories":[],"badges":[],"created_at":"2016-06-12T10:45:12.398700+00:00","downloads":1126453,"recent_downloads":202669,"max_version":"0.12.0","description":"JSON implementation in Rust","homepage":null,"documentation":"https://docs.rs/json/","repository":"https://github.com/maciejhirsz/json-rust","links":{"version_downloads":"/api/v1/crates/json/downloads","versions":null,"owners":"/api/v1/crates/json/owners","owner_team":"/api/v1/crates/json/owner_team","owner_user":"/api/v1/crates/json/owner_user","reverse_dependencies":"/api/v1/crates/json/reverse_dependencies"},"exact_match":false},"versions":[{"id":174509,"crate":"json","num":"0.12.0","dl_path":"/api/v1/crates/json/0.12.0/download","readme_path":"/api/v1/crates/json/0.12.0/readme","updated_at":"2019-09-06T16:43:25.950377+00:00","created_at":"2019-09-06T16:43:25.950377+00:00","downloads":47964,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.12.0/dependencies","version_downloads":"/api/v1/crates/json/0.12.0/downloads","authors":"/api/v1/crates/json/0.12.0/authors"},"crate_size":46747,"published_by":{"id":3670,"login":"maciejhirsz","name":"Maciej Hirsz","avatar":"https://avatars1.githubusercontent.com/u/1096222?v=4","url":"https://github.com/maciejhirsz"}},{"id":168892,"crate":"json","num":"0.11.15","dl_path":"/api/v1/crates/json/0.11.15/download","readme_path":"/api/v1/crates/json/0.11.15/readme","updated_at":"2019-08-12T10:43:45.112815+00:00","created_at":"2019-08-12T10:43:45.112815+00:00","downloads":65126,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.15/dependencies","version_downloads":"/api/v1/crates/json/0.11.15/downloads","authors":"/api/v1/crates/json/0.11.15/authors"},"crate_size":46802,"published_by":{"id":3670,"login":"maciejhirsz","name":"Maciej Hirsz","avatar":"https://avatars1.githubusercontent.com/u/1096222?v=4","url":"https://github.com/maciejhirsz"}},{"id":154478,"crate":"json","num":"0.11.14","dl_path":"/api/v1/crates/json/0.11.14/download","readme_path":"/api/v1/crates/json/0.11.14/readme","updated_at":"2019-06-04T15:03:03.471326+00:00","created_at":"2019-06-04T15:03:03.471326+00:00","downloads":96915,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.14/dependencies","version_downloads":"/api/v1/crates/json/0.11.14/downloads","authors":"/api/v1/crates/json/0.11.14/authors"},"crate_size":46645,"published_by":{"id":3670,"login":"maciejhirsz","name":"Maciej Hirsz","avatar":"https://avatars1.githubusercontent.com/u/1096222?v=4","url":"https://github.com/maciejhirsz"}},{"id":77794,"crate":"json","num":"0.11.13","dl_path":"/api/v1/crates/json/0.11.13/download","readme_path":"/api/v1/crates/json/0.11.13/readme","updated_at":"2018-01-17T15:43:01.944150+00:00","created_at":"2018-01-17T15:43:01.944150+00:00","downloads":745172,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.13/dependencies","version_downloads":"/api/v1/crates/json/0.11.13/downloads","authors":"/api/v1/crates/json/0.11.13/authors"},"crate_size":null,"published_by":null},{"id":70917,"crate":"json","num":"0.11.12","dl_path":"/api/v1/crates/json/0.11.12/download","readme_path":"/api/v1/crates/json/0.11.12/readme","updated_at":"2017-11-30T02:35:38.159293+00:00","created_at":"2017-11-10T17:30:27.550568+00:00","downloads":81750,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.12/dependencies","version_downloads":"/api/v1/crates/json/0.11.12/downloads","authors":"/api/v1/crates/json/0.11.12/authors"},"crate_size":null,"published_by":null},{"id":70910,"crate":"json","num":"0.11.11","dl_path":"/api/v1/crates/json/0.11.11/download","readme_path":"/api/v1/crates/json/0.11.11/readme","updated_at":"2017-11-30T03:02:53.581351+00:00","created_at":"2017-11-10T15:45:16.634206+00:00","downloads":290,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.11/dependencies","version_downloads":"/api/v1/crates/json/0.11.11/downloads","authors":"/api/v1/crates/json/0.11.11/authors"},"crate_size":null,"published_by":null},{"id":67655,"crate":"json","num":"0.11.10","dl_path":"/api/v1/crates/json/0.11.10/download","readme_path":"/api/v1/crates/json/0.11.10/readme","updated_at":"2017-11-30T02:25:31.288779+00:00","created_at":"2017-10-07T13:11:25.059988+00:00","downloads":5882,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.10/dependencies","version_downloads":"/api/v1/crates/json/0.11.10/downloads","authors":"/api/v1/crates/json/0.11.10/authors"},"crate_size":null,"published_by":null},{"id":60919,"crate":"json","num":"0.11.9","dl_path":"/api/v1/crates/json/0.11.9/download","readme_path":"/api/v1/crates/json/0.11.9/readme","updated_at":"2017-11-30T03:16:24.666232+00:00","created_at":"2017-07-28T10:07:50.238735+00:00","downloads":7518,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.9/dependencies","version_downloads":"/api/v1/crates/json/0.11.9/downloads","authors":"/api/v1/crates/json/0.11.9/authors"},"crate_size":null,"published_by":null},{"id":58613,"crate":"json","num":"0.11.8","dl_path":"/api/v1/crates/json/0.11.8/download","readme_path":"/api/v1/crates/json/0.11.8/readme","updated_at":"2017-11-30T03:15:04.094109+00:00","created_at":"2017-07-03T14:26:41.039569+00:00","downloads":2772,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.8/dependencies","version_downloads":"/api/v1/crates/json/0.11.8/downloads","authors":"/api/v1/crates/json/0.11.8/authors"},"crate_size":null,"published_by":null},{"id":58053,"crate":"json","num":"0.11.7","dl_path":"/api/v1/crates/json/0.11.7/download","readme_path":"/api/v1/crates/json/0.11.7/readme","updated_at":"2017-11-30T03:03:53.462428+00:00","created_at":"2017-06-26T11:32:33.025827+00:00","downloads":909,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.7/dependencies","version_downloads":"/api/v1/crates/json/0.11.7/downloads","authors":"/api/v1/crates/json/0.11.7/authors"},"crate_size":null,"published_by":null},{"id":47860,"crate":"json","num":"0.11.6","dl_path":"/api/v1/crates/json/0.11.6/download","readme_path":"/api/v1/crates/json/0.11.6/readme","updated_at":"2017-11-30T03:31:13.008624+00:00","created_at":"2017-03-15T12:24:17.607152+00:00","downloads":8643,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.6/dependencies","version_downloads":"/api/v1/crates/json/0.11.6/downloads","authors":"/api/v1/crates/json/0.11.6/authors"},"crate_size":null,"published_by":null},{"id":42931,"crate":"json","num":"0.11.5","dl_path":"/api/v1/crates/json/0.11.5/download","readme_path":"/api/v1/crates/json/0.11.5/readme","updated_at":"2017-11-30T02:38:02.497011+00:00","created_at":"2017-01-21T17:56:53.791250+00:00","downloads":26715,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.5/dependencies","version_downloads":"/api/v1/crates/json/0.11.5/downloads","authors":"/api/v1/crates/json/0.11.5/authors"},"crate_size":null,"published_by":null},{"id":41398,"crate":"json","num":"0.11.4","dl_path":"/api/v1/crates/json/0.11.4/download","readme_path":"/api/v1/crates/json/0.11.4/readme","updated_at":"2017-11-30T02:44:50.657167+00:00","created_at":"2017-01-02T17:38:11.472798+00:00","downloads":1409,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.4/dependencies","version_downloads":"/api/v1/crates/json/0.11.4/downloads","authors":"/api/v1/crates/json/0.11.4/authors"},"crate_size":null,"published_by":null},{"id":38809,"crate":"json","num":"0.11.3","dl_path":"/api/v1/crates/json/0.11.3/download","readme_path":"/api/v1/crates/json/0.11.3/readme","updated_at":"2017-11-30T02:48:57.683501+00:00","created_at":"2016-11-30T19:14:47.210057+00:00","downloads":2105,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.3/dependencies","version_downloads":"/api/v1/crates/json/0.11.3/downloads","authors":"/api/v1/crates/json/0.11.3/authors"},"crate_size":null,"published_by":null},{"id":38454,"crate":"json","num":"0.11.2","dl_path":"/api/v1/crates/json/0.11.2/download","readme_path":"/api/v1/crates/json/0.11.2/readme","updated_at":"2017-11-30T03:56:31.670050+00:00","created_at":"2016-11-25T12:29:26.640843+00:00","downloads":988,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.2/dependencies","version_downloads":"/api/v1/crates/json/0.11.2/downloads","authors":"/api/v1/crates/json/0.11.2/authors"},"crate_size":null,"published_by":null},{"id":38336,"crate":"json","num":"0.11.1","dl_path":"/api/v1/crates/json/0.11.1/download","readme_path":"/api/v1/crates/json/0.11.1/readme","updated_at":"2017-11-30T03:16:38.051321+00:00","created_at":"2016-11-23T11:46:32.411485+00:00","downloads":475,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.1/dependencies","version_downloads":"/api/v1/crates/json/0.11.1/downloads","authors":"/api/v1/crates/json/0.11.1/authors"},"crate_size":null,"published_by":null},{"id":36807,"crate":"json","num":"0.11.0","dl_path":"/api/v1/crates/json/0.11.0/download","readme_path":"/api/v1/crates/json/0.11.0/readme","updated_at":"2017-11-30T03:22:35.426910+00:00","created_at":"2016-10-28T06:56:11.451192+00:00","downloads":2567,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.11.0/dependencies","version_downloads":"/api/v1/crates/json/0.11.0/downloads","authors":"/api/v1/crates/json/0.11.0/authors"},"crate_size":null,"published_by":null},{"id":36734,"crate":"json","num":"0.10.3","dl_path":"/api/v1/crates/json/0.10.3/download","readme_path":"/api/v1/crates/json/0.10.3/readme","updated_at":"2017-11-30T03:06:15.348+00:00","created_at":"2016-10-26T18:46:04.219066+00:00","downloads":3963,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.10.3/dependencies","version_downloads":"/api/v1/crates/json/0.10.3/downloads","authors":"/api/v1/crates/json/0.10.3/authors"},"crate_size":null,"published_by":null},{"id":31597,"crate":"json","num":"0.10.2","dl_path":"/api/v1/crates/json/0.10.2/download","readme_path":"/api/v1/crates/json/0.10.2/readme","updated_at":"2017-11-30T03:42:13.158854+00:00","created_at":"2016-08-11T08:03:54.464905+00:00","downloads":6365,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.10.2/dependencies","version_downloads":"/api/v1/crates/json/0.10.2/downloads","authors":"/api/v1/crates/json/0.10.2/authors"},"crate_size":null,"published_by":null},{"id":31124,"crate":"json","num":"0.10.1","dl_path":"/api/v1/crates/json/0.10.1/download","readme_path":"/api/v1/crates/json/0.10.1/readme","updated_at":"2017-11-30T04:16:41.439619+00:00","created_at":"2016-08-01T19:07:32.359324+00:00","downloads":1701,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.10.1/dependencies","version_downloads":"/api/v1/crates/json/0.10.1/downloads","authors":"/api/v1/crates/json/0.10.1/authors"},"crate_size":null,"published_by":null},{"id":30599,"crate":"json","num":"0.10.0","dl_path":"/api/v1/crates/json/0.10.0/download","readme_path":"/api/v1/crates/json/0.10.0/readme","updated_at":"2017-11-30T03:17:36.095363+00:00","created_at":"2016-07-22T22:30:14.178850+00:00","downloads":1439,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.10.0/dependencies","version_downloads":"/api/v1/crates/json/0.10.0/downloads","authors":"/api/v1/crates/json/0.10.0/authors"},"crate_size":null,"published_by":null},{"id":30354,"crate":"json","num":"0.9.1","dl_path":"/api/v1/crates/json/0.9.1/download","readme_path":"/api/v1/crates/json/0.9.1/readme","updated_at":"2017-11-30T03:17:42.013327+00:00","created_at":"2016-07-18T11:23:49.505060+00:00","downloads":977,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.9.1/dependencies","version_downloads":"/api/v1/crates/json/0.9.1/downloads","authors":"/api/v1/crates/json/0.9.1/authors"},"crate_size":null,"published_by":null},{"id":30119,"crate":"json","num":"0.9.0","dl_path":"/api/v1/crates/json/0.9.0/download","readme_path":"/api/v1/crates/json/0.9.0/readme","updated_at":"2017-11-30T03:17:24.487156+00:00","created_at":"2016-07-16T11:45:59.627563+00:00","downloads":361,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.9.0/dependencies","version_downloads":"/api/v1/crates/json/0.9.0/downloads","authors":"/api/v1/crates/json/0.9.0/authors"},"crate_size":null,"published_by":null},{"id":29810,"crate":"json","num":"0.8.8","dl_path":"/api/v1/crates/json/0.8.8/download","readme_path":"/api/v1/crates/json/0.8.8/readme","updated_at":"2017-11-30T04:16:40.298835+00:00","created_at":"2016-07-10T10:07:31.645122+00:00","downloads":2929,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.8/dependencies","version_downloads":"/api/v1/crates/json/0.8.8/downloads","authors":"/api/v1/crates/json/0.8.8/authors"},"crate_size":null,"published_by":null},{"id":29702,"crate":"json","num":"0.8.7","dl_path":"/api/v1/crates/json/0.8.7/download","readme_path":"/api/v1/crates/json/0.8.7/readme","updated_at":"2017-11-30T02:35:55.575081+00:00","created_at":"2016-07-07T21:18:00.148594+00:00","downloads":560,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.7/dependencies","version_downloads":"/api/v1/crates/json/0.8.7/downloads","authors":"/api/v1/crates/json/0.8.7/authors"},"crate_size":null,"published_by":null},{"id":29655,"crate":"json","num":"0.8.6","dl_path":"/api/v1/crates/json/0.8.6/download","readme_path":"/api/v1/crates/json/0.8.6/readme","updated_at":"2017-11-30T04:16:41.431741+00:00","created_at":"2016-07-06T19:49:56.426153+00:00","downloads":449,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.6/dependencies","version_downloads":"/api/v1/crates/json/0.8.6/downloads","authors":"/api/v1/crates/json/0.8.6/authors"},"crate_size":null,"published_by":null},{"id":29558,"crate":"json","num":"0.8.5","dl_path":"/api/v1/crates/json/0.8.5/download","readme_path":"/api/v1/crates/json/0.8.5/readme","updated_at":"2017-11-30T02:35:59.271801+00:00","created_at":"2016-07-04T07:48:38.370908+00:00","downloads":821,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.5/dependencies","version_downloads":"/api/v1/crates/json/0.8.5/downloads","authors":"/api/v1/crates/json/0.8.5/authors"},"crate_size":null,"published_by":null},{"id":29389,"crate":"json","num":"0.8.4","dl_path":"/api/v1/crates/json/0.8.4/download","readme_path":"/api/v1/crates/json/0.8.4/readme","updated_at":"2017-11-30T03:14:29.884246+00:00","created_at":"2016-07-01T00:51:35.596508+00:00","downloads":397,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.4/dependencies","version_downloads":"/api/v1/crates/json/0.8.4/downloads","authors":"/api/v1/crates/json/0.8.4/authors"},"crate_size":null,"published_by":null},{"id":29388,"crate":"json","num":"0.8.2","dl_path":"/api/v1/crates/json/0.8.2/download","readme_path":"/api/v1/crates/json/0.8.2/readme","updated_at":"2017-11-30T03:33:33.657838+00:00","created_at":"2016-07-01T00:39:39.255557+00:00","downloads":310,"features":{},"yanked":true,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.2/dependencies","version_downloads":"/api/v1/crates/json/0.8.2/downloads","authors":"/api/v1/crates/json/0.8.2/authors"},"crate_size":null,"published_by":null},{"id":29377,"crate":"json","num":"0.8.1","dl_path":"/api/v1/crates/json/0.8.1/download","readme_path":"/api/v1/crates/json/0.8.1/readme","updated_at":"2017-11-30T03:20:35.592966+00:00","created_at":"2016-06-30T21:50:12.524126+00:00","downloads":309,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.1/dependencies","version_downloads":"/api/v1/crates/json/0.8.1/downloads","authors":"/api/v1/crates/json/0.8.1/authors"},"crate_size":null,"published_by":null},{"id":29291,"crate":"json","num":"0.8.0","dl_path":"/api/v1/crates/json/0.8.0/download","readme_path":"/api/v1/crates/json/0.8.0/readme","updated_at":"2017-11-30T02:35:59.269762+00:00","created_at":"2016-06-28T22:58:51.596959+00:00","downloads":424,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.8.0/dependencies","version_downloads":"/api/v1/crates/json/0.8.0/downloads","authors":"/api/v1/crates/json/0.8.0/authors"},"crate_size":null,"published_by":null},{"id":29207,"crate":"json","num":"0.7.4","dl_path":"/api/v1/crates/json/0.7.4/download","readme_path":"/api/v1/crates/json/0.7.4/readme","updated_at":"2017-11-30T02:50:49.524621+00:00","created_at":"2016-06-26T15:03:52.608111+00:00","downloads":1931,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.7.4/dependencies","version_downloads":"/api/v1/crates/json/0.7.4/downloads","authors":"/api/v1/crates/json/0.7.4/authors"},"crate_size":null,"published_by":null},{"id":29203,"crate":"json","num":"0.7.3","dl_path":"/api/v1/crates/json/0.7.3/download","readme_path":"/api/v1/crates/json/0.7.3/readme","updated_at":"2017-11-30T04:16:41.428984+00:00","created_at":"2016-06-26T11:57:35.192546+00:00","downloads":383,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.7.3/dependencies","version_downloads":"/api/v1/crates/json/0.7.3/downloads","authors":"/api/v1/crates/json/0.7.3/authors"},"crate_size":null,"published_by":null},{"id":29127,"crate":"json","num":"0.7.2","dl_path":"/api/v1/crates/json/0.7.2/download","readme_path":"/api/v1/crates/json/0.7.2/readme","updated_at":"2017-11-30T02:35:55.572575+00:00","created_at":"2016-06-24T15:47:50.906320+00:00","downloads":437,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.7.2/dependencies","version_downloads":"/api/v1/crates/json/0.7.2/downloads","authors":"/api/v1/crates/json/0.7.2/authors"},"crate_size":null,"published_by":null},{"id":29037,"crate":"json","num":"0.7.1","dl_path":"/api/v1/crates/json/0.7.1/download","readme_path":"/api/v1/crates/json/0.7.1/readme","updated_at":"2017-11-30T02:37:55.403299+00:00","created_at":"2016-06-22T22:15:52.217606+00:00","downloads":420,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.7.1/dependencies","version_downloads":"/api/v1/crates/json/0.7.1/downloads","authors":"/api/v1/crates/json/0.7.1/authors"},"crate_size":null,"published_by":null},{"id":29005,"crate":"json","num":"0.7.0","dl_path":"/api/v1/crates/json/0.7.0/download","readme_path":"/api/v1/crates/json/0.7.0/readme","updated_at":"2017-11-30T03:45:11.011176+00:00","created_at":"2016-06-22T09:14:20.081961+00:00","downloads":305,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.7.0/dependencies","version_downloads":"/api/v1/crates/json/0.7.0/downloads","authors":"/api/v1/crates/json/0.7.0/authors"},"crate_size":null,"published_by":null},{"id":28948,"crate":"json","num":"0.6.1","dl_path":"/api/v1/crates/json/0.6.1/download","readme_path":"/api/v1/crates/json/0.6.1/readme","updated_at":"2017-11-30T04:16:41.426894+00:00","created_at":"2016-06-21T16:32:09.081738+00:00","downloads":324,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.6.1/dependencies","version_downloads":"/api/v1/crates/json/0.6.1/downloads","authors":"/api/v1/crates/json/0.6.1/authors"},"crate_size":null,"published_by":null},{"id":28917,"crate":"json","num":"0.6.0","dl_path":"/api/v1/crates/json/0.6.0/download","readme_path":"/api/v1/crates/json/0.6.0/readme","updated_at":"2017-11-30T02:41:39.957064+00:00","created_at":"2016-06-20T23:01:33.821328+00:00","downloads":390,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.6.0/dependencies","version_downloads":"/api/v1/crates/json/0.6.0/downloads","authors":"/api/v1/crates/json/0.6.0/authors"},"crate_size":null,"published_by":null},{"id":28886,"crate":"json","num":"0.5.1","dl_path":"/api/v1/crates/json/0.5.1/download","readme_path":"/api/v1/crates/json/0.5.1/readme","updated_at":"2017-11-30T04:16:41.424694+00:00","created_at":"2016-06-20T07:29:12.934968+00:00","downloads":352,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.5.1/dependencies","version_downloads":"/api/v1/crates/json/0.5.1/downloads","authors":"/api/v1/crates/json/0.5.1/authors"},"crate_size":null,"published_by":null},{"id":28858,"crate":"json","num":"0.5.0","dl_path":"/api/v1/crates/json/0.5.0/download","readme_path":"/api/v1/crates/json/0.5.0/readme","updated_at":"2017-11-30T02:58:48.987345+00:00","created_at":"2016-06-19T19:21:36.422740+00:00","downloads":344,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.5.0/dependencies","version_downloads":"/api/v1/crates/json/0.5.0/downloads","authors":"/api/v1/crates/json/0.5.0/authors"},"crate_size":null,"published_by":null},{"id":28821,"crate":"json","num":"0.4.0","dl_path":"/api/v1/crates/json/0.4.0/download","readme_path":"/api/v1/crates/json/0.4.0/readme","updated_at":"2017-11-30T03:34:04.495708+00:00","created_at":"2016-06-18T20:06:39.365549+00:00","downloads":442,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.4.0/dependencies","version_downloads":"/api/v1/crates/json/0.4.0/downloads","authors":"/api/v1/crates/json/0.4.0/authors"},"crate_size":null,"published_by":null},{"id":28798,"crate":"json","num":"0.3.4","dl_path":"/api/v1/crates/json/0.3.4/download","readme_path":"/api/v1/crates/json/0.3.4/readme","updated_at":"2017-11-30T03:33:55.467492+00:00","created_at":"2016-06-17T19:58:50.645056+00:00","downloads":365,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.3.4/dependencies","version_downloads":"/api/v1/crates/json/0.3.4/downloads","authors":"/api/v1/crates/json/0.3.4/authors"},"crate_size":null,"published_by":null},{"id":28739,"crate":"json","num":"0.3.3","dl_path":"/api/v1/crates/json/0.3.3/download","readme_path":"/api/v1/crates/json/0.3.3/readme","updated_at":"2017-11-30T03:54:03.145358+00:00","created_at":"2016-06-16T20:58:52.819806+00:00","downloads":325,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.3.3/dependencies","version_downloads":"/api/v1/crates/json/0.3.3/downloads","authors":"/api/v1/crates/json/0.3.3/authors"},"crate_size":null,"published_by":null},{"id":28689,"crate":"json","num":"0.3.2","dl_path":"/api/v1/crates/json/0.3.2/download","readme_path":"/api/v1/crates/json/0.3.2/readme","updated_at":"2017-11-30T04:16:41.422122+00:00","created_at":"2016-06-15T18:50:40.827945+00:00","downloads":317,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.3.2/dependencies","version_downloads":"/api/v1/crates/json/0.3.2/downloads","authors":"/api/v1/crates/json/0.3.2/authors"},"crate_size":null,"published_by":null},{"id":28627,"crate":"json","num":"0.3.1","dl_path":"/api/v1/crates/json/0.3.1/download","readme_path":"/api/v1/crates/json/0.3.1/readme","updated_at":"2017-11-30T02:28:33.460562+00:00","created_at":"2016-06-14T19:38:58.103015+00:00","downloads":313,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.3.1/dependencies","version_downloads":"/api/v1/crates/json/0.3.1/downloads","authors":"/api/v1/crates/json/0.3.1/authors"},"crate_size":null,"published_by":null},{"id":28624,"crate":"json","num":"0.3.0","dl_path":"/api/v1/crates/json/0.3.0/download","readme_path":"/api/v1/crates/json/0.3.0/readme","updated_at":"2017-11-30T03:33:00.972605+00:00","created_at":"2016-06-14T18:36:30.189726+00:00","downloads":312,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.3.0/dependencies","version_downloads":"/api/v1/crates/json/0.3.0/downloads","authors":"/api/v1/crates/json/0.3.0/authors"},"crate_size":null,"published_by":null},{"id":28607,"crate":"json","num":"0.2.1","dl_path":"/api/v1/crates/json/0.2.1/download","readme_path":"/api/v1/crates/json/0.2.1/readme","updated_at":"2017-11-30T04:16:41.415081+00:00","created_at":"2016-06-14T11:18:55.626479+00:00","downloads":337,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.2.1/dependencies","version_downloads":"/api/v1/crates/json/0.2.1/downloads","authors":"/api/v1/crates/json/0.2.1/authors"},"crate_size":null,"published_by":null},{"id":28513,"crate":"json","num":"0.2.0","dl_path":"/api/v1/crates/json/0.2.0/download","readme_path":"/api/v1/crates/json/0.2.0/readme","updated_at":"2017-11-30T03:32:57.987819+00:00","created_at":"2016-06-12T14:50:22.388425+00:00","downloads":316,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.2.0/dependencies","version_downloads":"/api/v1/crates/json/0.2.0/downloads","authors":"/api/v1/crates/json/0.2.0/authors"},"crate_size":null,"published_by":null},{"id":28508,"crate":"json","num":"0.1.1","dl_path":"/api/v1/crates/json/0.1.1/download","readme_path":"/api/v1/crates/json/0.1.1/readme","updated_at":"2017-11-30T03:32:50.830670+00:00","created_at":"2016-06-12T12:07:22.844970+00:00","downloads":332,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.1.1/dependencies","version_downloads":"/api/v1/crates/json/0.1.1/downloads","authors":"/api/v1/crates/json/0.1.1/authors"},"crate_size":null,"published_by":null},{"id":28506,"crate":"json","num":"0.1.0","dl_path":"/api/v1/crates/json/0.1.0/download","readme_path":"/api/v1/crates/json/0.1.0/readme","updated_at":"2017-11-30T03:16:54.016723+00:00","created_at":"2016-06-12T10:45:12.398700+00:00","downloads":303,"features":{},"yanked":false,"license":"MIT/Apache-2.0","links":{"dependencies":"/api/v1/crates/json/0.1.0/dependencies","version_downloads":"/api/v1/crates/json/0.1.0/downloads","authors":"/api/v1/crates/json/0.1.0/authors"},"crate_size":null,"published_by":null}],"keywords":[],"categories":[]}""" +} diff --git a/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStageTests.scala b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStageTests.scala new file mode 100644 index 00000000..8c3d83cf --- /dev/null +++ b/codefeedr-plugins/codefeedr-cargo/src/test/scala/org/codefeedr/plugins/cargo/stages/CargoReleasesStageTests.scala @@ -0,0 +1,54 @@ +package org.codefeedr.plugins.cargo.stages + +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.codefeedr.stages.OutputStage +import org.scalatest.FunSuite +import org.codefeedr.plugins.cargo.operators.CargoSourceConfig +import org.codefeedr.plugins.cargo.protocol.Protocol.CrateRelease + +class CargoReleasesStageTests extends FunSuite { + + test("CargoReleasesIntegrationTest") { + val source = new CargoReleasesStage(sourceConfig = CargoSourceConfig(1000, 4, 4)) + val sink = new LimitingSinkStage(4) + + val pipeline: Unit = new PipelineBuilder() + .append(source) + .append(sink) + .build() + .startMock() + } +} + +// Simple Sink Pipeline Object that limits the output to a certain number +// and is able to get a list of all the items that were received in the sink +class LimitingSinkStage(elements: Int = -1) + extends OutputStage[CrateRelease] + with Serializable { + var sink: LimitingSink = _ + + override def main(source: DataStream[CrateRelease]): Unit = { + sink = new LimitingSink(elements) + source.addSink(sink).setParallelism(1) + } +} + +class LimitingSink(elements: Int) extends SinkFunction[CrateRelease] { + var count = 0 + var items: List[CrateRelease] = List() + + override def invoke(value: CrateRelease, + context: SinkFunction.Context[_]): Unit = { + count += 1 + items = value :: items + + println(count) + + if (elements != -1 && count >= elements) { + throw new RuntimeException() + } + } + +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSource.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSource.scala new file mode 100644 index 00000000..efc12715 --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSource.scala @@ -0,0 +1,150 @@ +package org.codefeedr.plugins.clearlydefined.operators + +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.codefeedr.plugins.clearlydefined.protocol.Protocol +import org.codefeedr.plugins.clearlydefined.protocol.Protocol.ClearlyDefinedRelease +import org.codefeedr.stages.utilities.{HttpRequester, RequestException} +import org.codefeedr.pipeline.{PluginReleasesSource, PluginSourceConfig} +import org.json4s._ +import org.json4s.jackson.JsonMethods._ +import org.json4s.jackson.Serialization.read +import scalaj.http.Http + +/** + * The configuration class for the ClearlyDefinedReleasesSource class + * @param pollingInterval Amount of milliseconds to wait for next poll. Put to 30 seconds due to typically slow feed + * @param maxNumberOfRuns if positive, runs definitely up till x. If negative, runs indefinitely. + */ +case class ClearlyDefinedSourceConfig(pollingInterval: Int = 30000, + maxNumberOfRuns: Int = -1, + timeout: Int = 32) + extends PluginSourceConfig + +/** + * Important to note in retrieving data from the stream of projects in ClearlyDefined is the following: + * - The stream URL is https://api.clearlydefined.io/definitions?matchCasing=false&sort=releaseDate&sortDesc=true + * - 100 most recently changed packages can be found there, which is way too much to process with each poll + * - Therefore only the first {packageAmount} number of packages are processed + * @param config the ClearlyDefined source configuration, has pollingInterval and maxNumberOfRuns fields + */ +class ClearlyDefinedReleasesSource( + config: ClearlyDefinedSourceConfig = ClearlyDefinedSourceConfig() +) extends PluginReleasesSource[ClearlyDefinedRelease](config) { + + /** url for the stream of new CD projects */ + val url = + "https://api.clearlydefined.io/definitions?matchCasing=false&sort=releaseDate&sortDesc=true" + + /** The first x number of packages to process with each poll */ + val packageAmount = 10 + + /** + * Main fetcher of new items in the ClearlyDefined package source + * @param ctx context + */ + override def run( + ctx: SourceFunction.SourceContext[ClearlyDefinedRelease] + ): Unit = { + val lock = ctx.getCheckpointLock + + /** While is running or #runs left. */ + while (isRunning && runsLeft != 0) { + lock.synchronized { // Synchronize to the checkpoint lock. + try { + // Polls the RSS feed + val rssAsString: String = getRSSAsString.get + // Parses the received rss items + val items: Seq[ClearlyDefinedRelease] = parseRSSString(rssAsString) + // Collect right items and update last item + val validSortedItems: Seq[ClearlyDefinedRelease] = + sortAndDropDuplicates(items) + // Decrease runs left + super.decreaseRunsLeft() + // Add a timestamp to the item + validSortedItems.foreach( + x => + ctx.collectWithTimestamp( + x, + Protocol.getDate(x._meta.updated).getTime + )) + // Call run in parent + super.runPlugin(ctx, validSortedItems) + } catch { + case _: Throwable => + } + } + } + } + + /** + * Drops items that already have been collected and sorts them based on times + * TODO: x._meta.updated is not chronological ~5% of the time, which means 1 in 20 packages are SKIPPED + * + * @param items Potential items to be collected + * @return Valid sorted items + */ + def sortAndDropDuplicates( + items: Seq[ClearlyDefinedRelease] + ): Seq[ClearlyDefinedRelease] = { + items + .filter((x: ClearlyDefinedRelease) => { + if (lastItem.isDefined) + Protocol + .getDate(lastItem.get._meta.updated) + .before(Protocol.getDate(x._meta.updated)) + else + true + }) + .sortWith((x: ClearlyDefinedRelease, y: ClearlyDefinedRelease) => + Protocol + .getDate(x._meta.updated) + .before(Protocol.getDate(y._meta.updated))) + } + + /** + * Requests the RSS feed and returns its body as a string. + * Will keep trying with increasing intervals if it doesn't succeed + * + * @return Body of requested RSS feed + */ + @throws[RequestException] + def getRSSAsString: Option[String] = { + try { + Some(new HttpRequester().retrieveResponse(Http(url)).body) + } catch { + case _: Throwable => None + } + } + + /** + * Parses a string that contains JSON with RSS items into a list of ClearlyDefinedRelease's + * @param rssString JSON string which contains RSS items + * @return + */ + def parseRSSString(rssString: String): Seq[ClearlyDefinedRelease] = { + try { + // Parse the big release string as a Json object + val json: JValue = parse(rssString) + + // Retrieve the first {packageAmount} packages + val packages: List[JValue] = + (json \ "data").children.take(this.packageAmount) + + // Render back to string to prepare for the 'read' call (requires string input) + val packagesString: List[String] = packages.map(x => compact(render(x))) + + // Convert from string to ClearlyDefinedRelease + implicit val formats = DefaultFormats + for (packageString <- packagesString) yield { + read[ClearlyDefinedRelease](packageString) + } + } catch { + // If the string cannot be parsed return an empty list + case _: Throwable => + printf( + "Failed parsing the RSSString in the ClearlyDefinedReleasesSource.scala file" + ) + Nil + } + } +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/protocol/Protocol.scala new file mode 100644 index 00000000..a2c16cef --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/protocol/Protocol.scala @@ -0,0 +1,554 @@ +package org.codefeedr.plugins.clearlydefined.protocol + +import java.sql.Timestamp +import java.text.SimpleDateFormat +import java.util.{Calendar, Date} + +object Protocol { + + /** Date format used in ClearlyDefined */ + val dateFormats = + List("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'", "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") + + case class ClearlyDefinedRelease( + described: CDDescribed, + licensed: CDLicensed, + coordinates: CDCoordinates, + _meta: CD_meta, + scores: CDScores + ) + + class ClearlyDefinedReleasePojo extends Serializable { + var described: CDDescribedPojo = _ + var licensed: CDLicensedPojo = _ + var coordinates: CDCoordinatesPojo = _ + var _meta: CD_metaPojo = _ + var scores: CDScoresPojo = _ + } + + class ClearlyDefinedReleasePojoExt extends ClearlyDefinedReleasePojo { + var name: String = _ + } + + object ClearlyDefinedReleasePojo { + def fromClearlyDefinedRelease( + clearlyDefinedRelease: ClearlyDefinedRelease + ): ClearlyDefinedReleasePojo = { + val pojo = new ClearlyDefinedReleasePojo + pojo.described = + CDDescribedPojo.fromCDDescribed(clearlyDefinedRelease.described) + pojo.licensed = + CDLicensedPojo.fromCDLicensed(clearlyDefinedRelease.licensed) + pojo.coordinates = + CDCoordinatesPojo.fromCDCoordinates(clearlyDefinedRelease.coordinates) + pojo._meta = CD_metaPojo.fromCD_meta(clearlyDefinedRelease._meta) + pojo.scores = CDScoresPojo.fromCDScores(clearlyDefinedRelease.scores) + pojo + } + } + + case class CDDescribed( + releaseDate: String, + urls: CDDescribedUrls, + projectWebsite: Option[String], + issueTracker: Option[String], + hashes: CDDescribedHashes, + files: Int, + tools: List[String], + toolScore: CDDescribedToolScore, + sourceLocation: Option[CDDescribedSourceLocation], + score: CDDescribedScore + ) + + class CDDescribedPojo extends Serializable { + var releaseDate: String = _ + var urls: CDDescribedUrlsPojo = _ + var projectWebsite: String = _ + var issueTracker: String = _ + var hashes: CDDescribedHashesPojo = _ + var files: Int = _ + //var tools: List[String] = _ + var tools: List[ToolPojo] = _ + var toolScore: CDDescribedToolScorePojo = _ + var sourceLocation: CDDescribedSourceLocationPojo = _ + var score: CDDescribedScorePojo = _ + } + + class CDDescribedPojoExt extends CDDescribedPojo { + var name: String = _ + } + + object CDDescribedPojo { + def fromCDDescribed(cdDescribed: CDDescribed): CDDescribedPojo = { + val pojo = new CDDescribedPojo + pojo.releaseDate = cdDescribed.releaseDate + pojo.urls = CDDescribedUrlsPojo.fromCDDescribedUrls(cdDescribed.urls) + pojo.projectWebsite = cdDescribed.projectWebsite.orNull + pojo.issueTracker = cdDescribed.issueTracker.orNull + pojo.hashes = + CDDescribedHashesPojo.fromCDDescribedHashes(cdDescribed.hashes) + pojo.files = cdDescribed.files + //pojo.tools = cdDescribed.tools + pojo.tools = + cdDescribed.tools.map(toolname => ToolPojo.fromToolAsString(toolname)) + pojo.toolScore = + CDDescribedToolScorePojo.fromCDDescribedToolScore(cdDescribed.toolScore) + + // Set the source location + if (cdDescribed.sourceLocation.isDefined) { + pojo.sourceLocation = + CDDescribedSourceLocationPojo.fromCDDescribedSourceLocation( + cdDescribed.sourceLocation.get + ) + } + + pojo.score = CDDescribedScorePojo.fromCDDescribedScore(cdDescribed.score) + pojo + } + } + + case class CDDescribedUrls( + registry: String, + version: String, + download: String + ) + + class CDDescribedUrlsPojo extends Serializable { + var registry: String = _ + var version: String = _ + var download: String = _ + } + + class CDDescribedUrlsPojoExt extends CDDescribedUrlsPojo { + var name: String = _ + } + + object CDDescribedUrlsPojo { + def fromCDDescribedUrls( + cdDescribedUrls: CDDescribedUrls + ): CDDescribedUrlsPojo = { + val pojo = new CDDescribedUrlsPojo + pojo.registry = cdDescribedUrls.registry + pojo.version = cdDescribedUrls.version + pojo.download = cdDescribedUrls.download + pojo + } + } + + case class CDDescribedHashes( + gitSha: Option[String], + sha1: Option[String], + sha256: Option[String] + ) + + class CDDescribedHashesPojo extends Serializable { + var gitSha: String = _ + var sha1: String = _ + var sha256: String = _ + } + + class CDDescribedHashesPojoExt extends CDDescribedHashesPojo { + var name: String = _ + } + + object CDDescribedHashesPojo { + def fromCDDescribedHashes( + cdDescribedHashes: CDDescribedHashes + ): CDDescribedHashesPojo = { + val pojo = new CDDescribedHashesPojo + pojo.gitSha = cdDescribedHashes.gitSha.orNull + pojo.sha1 = cdDescribedHashes.sha1.orNull + pojo.sha256 = cdDescribedHashes.sha256.orNull + pojo + } + } + + class ToolPojo extends Serializable { + var toolname: String = _ + } + + class ToolPojoExt extends ToolPojo { + var id: String = _ + } + + object ToolPojo { + def fromToolAsString(toolname: String): ToolPojo = { + val pojo = new ToolPojo() + pojo.toolname = toolname + pojo + } + } + + case class CDDescribedToolScore(total: Int, date: Int, source: Int) + + class CDDescribedToolScorePojo extends Serializable { + var total: Int = _ + var date: Int = _ + var source: Int = _ + } + + class CDDescribedToolScorePojoExt extends CDDescribedToolScorePojo { + var name: String = _ + } + + object CDDescribedToolScorePojo { + def fromCDDescribedToolScore( + cdDescribedToolScore: CDDescribedToolScore + ): CDDescribedToolScorePojo = { + val pojo = new CDDescribedToolScorePojo + pojo.total = cdDescribedToolScore.total + pojo.date = cdDescribedToolScore.date + pojo.source = cdDescribedToolScore.source + pojo + } + } + + case class CDDescribedSourceLocation( + locationType: String, + provider: String, + namespace: String, + name: String, + revision: String, + url: String + ) + + class CDDescribedSourceLocationPojo extends Serializable { + var locationType: String = _ + var provider: String = _ + var namespace: String = _ + var name: String = _ + var revision: String = _ + var url: String = _ + } + + class CDDescribedSourceLocationPojoExt extends CDDescribedSourceLocationPojo { + var packageName: String = _ + } + + object CDDescribedSourceLocationPojo { + def fromCDDescribedSourceLocation( + cdDescribedSourceLocation: CDDescribedSourceLocation + ): CDDescribedSourceLocationPojo = { + val pojo = new CDDescribedSourceLocationPojo + pojo.locationType = cdDescribedSourceLocation.locationType + pojo.provider = cdDescribedSourceLocation.provider + pojo.namespace = cdDescribedSourceLocation.namespace + pojo.name = cdDescribedSourceLocation.name + pojo.revision = cdDescribedSourceLocation.revision + pojo.url = cdDescribedSourceLocation.url + pojo + } + } + + case class CDDescribedScore(total: Int, date: Int, source: Int) + + class CDDescribedScorePojo extends Serializable { + var total: Int = _ + var date: Int = _ + var source: Int = _ + } + + class CDDescribedScorePojoExt extends CDDescribedScorePojo { + var name: String = _ + } + + object CDDescribedScorePojo { + def fromCDDescribedScore( + cdDescribedScore: CDDescribedScore + ): CDDescribedScorePojo = { + val pojo = new CDDescribedScorePojo + pojo.total = cdDescribedScore.total + pojo.date = cdDescribedScore.date + pojo.source = cdDescribedScore.source + pojo + } + } + + case class CDLicensed( + declared: Option[String], + toolScore: CDLicensedToolScore, + facets: CDLicensedFacets, + score: CDLicensedScore + ) + + class CDLicensedPojo extends Serializable { + var declared: String = _ + var toolScore: CDLicensedToolScorePojo = _ + var facets: CDLicensedFacetsPojo = _ + var score: CDLicensedScorePojo = _ + } + + class CDLicensedPojoExt extends CDLicensedPojo { + var name: String = _ + } + + object CDLicensedPojo { + def fromCDLicensed(cdLicensed: CDLicensed): CDLicensedPojo = { + val pojo = new CDLicensedPojo + if (cdLicensed.declared.isDefined) { + pojo.declared = cdLicensed.declared.get + } + pojo.toolScore = + CDLicensedToolScorePojo.fromCDLicensedToolScore(cdLicensed.toolScore) + pojo.facets = CDLicensedFacetsPojo.fromCDLicensedFacets(cdLicensed.facets) + pojo.score = CDLicensedScorePojo.fromCDLicensedScore(cdLicensed.score) + pojo + } + } + + case class CDLicensedToolScore( + total: Int, + declared: Int, + discovered: Int, + consistency: Int, + spdx: Int, + texts: Int + ) + + class CDLicensedToolScorePojo extends Serializable { + var total: Int = _ + var declared: Int = _ + var discovered: Int = _ + var consistency: Int = _ + var spdx: Int = _ + var texts: Int = _ + } + + class CDLicensedToolScorePojoExt extends CDLicensedToolScorePojo { + var name: String = _ + } + + object CDLicensedToolScorePojo { + def fromCDLicensedToolScore( + cdLicensedToolScore: CDLicensedToolScore + ): CDLicensedToolScorePojo = { + val pojo = new CDLicensedToolScorePojo + pojo.total = cdLicensedToolScore.total + pojo.declared = cdLicensedToolScore.declared + pojo.discovered = cdLicensedToolScore.discovered + pojo.consistency = cdLicensedToolScore.consistency + pojo.spdx = cdLicensedToolScore.spdx + pojo.texts = cdLicensedToolScore.texts + pojo + } + } + + case class CDLicensedFacets(core: CDLFCore) + + class CDLicensedFacetsPojo extends Serializable { + var core: CDLFCorePojo = _ + } + + class CDLicensedFacetsPojoExt extends CDLicensedFacetsPojo { + var name: String = _ + } + + object CDLicensedFacetsPojo { + def fromCDLicensedFacets( + cdLicensedFacets: CDLicensedFacets + ): CDLicensedFacetsPojo = { + val pojo = new CDLicensedFacetsPojo + pojo.core = CDLFCorePojo.fromCDLFCore(cdLicensedFacets.core) + pojo + } + } + + case class CDLFCore( + attribution: CDLFCoreAttribution, + discovered: CDLFCoreDiscovered, + files: Int + ) + + class CDLFCorePojo extends Serializable { + var attribution: CDLFCoreAttributionPojo = _ + var discovered: CDLFCoreDiscoveredPojo = _ + var files: Int = _ + } + + class CDLFCorePojoExt extends CDLFCorePojo { + var name: String = _ + } + + object CDLFCorePojo { + def fromCDLFCore(cdlfCore: CDLFCore): CDLFCorePojo = { + val pojo = new CDLFCorePojo + pojo.attribution = + CDLFCoreAttributionPojo.fromCDLFCoreAttribution(cdlfCore.attribution) + pojo.discovered = + CDLFCoreDiscoveredPojo.fromCDLFCoreDiscovered(cdlfCore.discovered) + pojo.files = cdlfCore.files + pojo + } + } + + case class CDLFCoreAttribution(unknown: Int, parties: Option[List[String]]) + + class CDLFCoreAttributionPojo extends Serializable { + var unknown: Int = _ + var parties: List[String] = _ + } + + class CDLFCoreAttributionPojoExt extends Serializable { + var id: String = _ + var unknown: Int = _ + var party: String = _ + } + + object CDLFCoreAttributionPojo { + def fromCDLFCoreAttribution( + cdlfCoreAttribution: CDLFCoreAttribution + ): CDLFCoreAttributionPojo = { + val pojo = new CDLFCoreAttributionPojo + pojo.unknown = cdlfCoreAttribution.unknown + if (cdlfCoreAttribution.parties.isDefined) { + pojo.parties = cdlfCoreAttribution.parties.get + } + pojo + } + } + + case class CDLFCoreDiscovered(unknown: Int, expressions: List[String]) + + class CDLFCoreDiscoveredPojo extends Serializable { + var unknown: Int = _ + var expressions: List[String] = _ + } + + class CDLFCoreDiscoveredPojoExt extends Serializable { + var id: String = _ + var unknown: Int = _ + var expression: String = _ + } + + object CDLFCoreDiscoveredPojo { + def fromCDLFCoreDiscovered( + cdlfCoreDiscovered: CDLFCoreDiscovered + ): CDLFCoreDiscoveredPojo = { + val pojo = new CDLFCoreDiscoveredPojo + pojo.unknown = cdlfCoreDiscovered.unknown + pojo.expressions = cdlfCoreDiscovered.expressions + pojo + } + } + + case class CDLicensedScore( + total: Int, + declared: Int, + discovered: Int, + consistency: Int, + spdx: Int, + texts: Int + ) + + class CDLicensedScorePojo extends Serializable { + var total: Int = _ + var declared: Int = _ + var discovered: Int = _ + var consistency: Int = _ + var spdx: Int = _ + var texts: Int = _ + } + + class CDLicensedScorePojoExt extends CDLicensedScorePojo { + var name: String = _ + } + + object CDLicensedScorePojo { + def fromCDLicensedScore( + cdLicensedScore: CDLicensedScore + ): CDLicensedScorePojo = { + val pojo = new CDLicensedScorePojo + pojo.total = cdLicensedScore.total + pojo.declared = cdLicensedScore.declared + pojo.discovered = cdLicensedScore.discovered + pojo.consistency = cdLicensedScore.consistency + pojo.spdx = cdLicensedScore.spdx + pojo.texts = cdLicensedScore.texts + pojo + } + } + + case class CDCoordinates( + `type`: String, + provider: String, + name: String, + namespace: Option[String], + revision: String + ) + + class CDCoordinatesPojo extends Serializable { + var `type`: String = _ + var provider: String = _ + var name: String = _ + var namespace: String = _ + var revision: String = _ + } + + object CDCoordinatesPojo { + def fromCDCoordinates(cdCoordinates: CDCoordinates): CDCoordinatesPojo = { + val pojo = new CDCoordinatesPojo + pojo.`type` = cdCoordinates.`type` + pojo.provider = cdCoordinates.provider + pojo.name = cdCoordinates.name + pojo.namespace = cdCoordinates.namespace.orNull + pojo.revision = cdCoordinates.revision + pojo + } + } + + case class CD_meta(schemaVersion: String, updated: String) + + class CD_metaPojo extends Serializable { + var schemaVersion: String = _ + var updated: Timestamp = _ + } + + class CD_metaPojoExt extends CD_metaPojo { + var name: String = _ + } + + object CD_metaPojo { + def fromCD_meta(cd_meta: CD_meta): CD_metaPojo = { + val pojo = new CD_metaPojo + pojo.schemaVersion = cd_meta.schemaVersion + + val dateField: Date = getDate(cd_meta.updated) + + val cal: Calendar = Calendar.getInstance + cal.setTime(dateField) + val time = new Timestamp(cal.getTimeInMillis) + + pojo.updated = time + pojo + } + } + + def getDate(field: String): Date = { + var dateField: Date = null + for (dateFormat <- dateFormats) { + dateField = new SimpleDateFormat(dateFormat).parse(field) + } + if (dateField == null) println("Can't parse date") + dateField + } + + case class CDScores(effective: Int, tool: Int) + + class CDScoresPojo extends Serializable { + var effective: Int = _ + var tool: Int = _ + } + + class CDScoresPojoExt extends CDScoresPojo { + var name: String = _ + } + + object CDScoresPojo { + def fromCDScores(cdScores: CDScores): CDScoresPojo = { + val pojo = new CDScoresPojo + pojo.effective = cdScores.effective + pojo.tool = cdScores.tool + pojo + } + } +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala new file mode 100644 index 00000000..04197d97 --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala @@ -0,0 +1,41 @@ +package org.codefeedr.plugins.clearlydefined.stages + +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.Context +import org.codefeedr.stages.InputStage +import org.codefeedr.plugins.clearlydefined.operators.{ + ClearlyDefinedReleasesSource, + ClearlyDefinedSourceConfig +} +import org.codefeedr.plugins.clearlydefined.protocol.Protocol.ClearlyDefinedRelease +import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor + +import scala.language.higherKinds + +/** fetches real-time releases from ClearlyDefined */ +class ClearlyDefinedReleasesStage(stageId: String = + "clearlydefined_releases_min", + sourceConfig: ClearlyDefinedSourceConfig = + ClearlyDefinedSourceConfig(30000, -1)) + extends InputStage[ClearlyDefinedRelease](Some(stageId)) { + + /** Fetches [[ClearlyDefinedRelease]] from real-time ClearlyDefined feed. + * + * @param context The context to add the source to. + * @return The stream of type [[ClearlyDefinedRelease]]. + */ + override def main(context: Context): DataStream[ClearlyDefinedRelease] = { + implicit val typeInfo: TypeInformation[ClearlyDefinedRelease] = + TypeInformation.of(classOf[ClearlyDefinedRelease]) + context.env + .addSource(new ClearlyDefinedReleasesSource(sourceConfig))(typeInfo) + } + + override def getSchema: Schema = { + implicit val NpmSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[ClearlyDefinedRelease] + } +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSourceTests.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSourceTests.scala new file mode 100644 index 00000000..ccc3caab --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/operators/ClearlyDefinedReleasesSourceTests.scala @@ -0,0 +1,90 @@ +package org.codefeedr.plugins.clearlydefined.operators + +import org.codefeedr.plugins.clearlydefined.protocol.Protocol.ClearlyDefinedRelease +import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization.read +import org.scalatest.FunSuite +import org.codefeedr.plugins.clearlydefined.resources.ClearlyDefinedPackageSnapshot + +class ClearlyDefinedReleasesSourceTests extends FunSuite{ + + val cdrs = new ClearlyDefinedReleasesSource() + + test("getRSSAsString returns Option[String] and does not take too long") { + // Arrange + val startTime = System.currentTimeMillis() + + // Act + val result = cdrs.getRSSAsString + + // Assert + assert(result.isDefined) + // Assert that the string is not a small returned error string + assert(result.get.length > 100) + // Assert not longer than 3 seconds for a single web request + assert(System.currentTimeMillis() - startTime < 3000) + } + + test("jsonRead success with snapshot") { + // Arrange + implicit val formats: DefaultFormats.type = DefaultFormats + + // Act + val result = read[ClearlyDefinedRelease](ClearlyDefinedPackageSnapshot.snapshot) + + // Assert + assert(result.coordinates.name == "sweetalert2") + } + + // TODO discover reason for test failures +// test("parseRSSString success with web request") { +// // Arrange +// implicit val formats = DefaultFormats +// val stringInput = cdrs.getRSSAsString.get +// +// // Act +// val result = cdrs.parseRSSString(stringInput) +// +// // Assert +// assert(result(0).coordinates.`type`.isInstanceOf[String]) +// } +// +// test("dateFormatParser works for cd._meta.updated field") { +// // Arrange +// implicit val formats = DefaultFormats +// val stringInput = cdrs.getRSSAsString.get +// val tenCDReleases = cdrs.parseRSSString(stringInput) +// +// // Act +// val dateTime = tenCDReleases(0)._meta.updated +// val parsedDateTime = Protocol.getDate(dateTime) +// +// // Assert +// assert(parsedDateTime.isInstanceOf[Date]) +// } + + test("parseRSSString fails with wrong input") { + // Arrange + implicit val formats: DefaultFormats.type = DefaultFormats + + // Act + val result = cdrs.parseRSSString("bogus string") + + // Assert + assert(result.isEmpty) + } + + test("waitPollingInterval waits for x+ seoncds") { + // Assert + val withSeconds = 1000 + val newCdrs = new ClearlyDefinedReleasesSource(ClearlyDefinedSourceConfig(withSeconds, -1, 4)) + val startTime = System.currentTimeMillis() + + // Act + newCdrs.waitPollingInterval(5) + val newTime = System.currentTimeMillis() + + // Assert + assert(newTime - startTime >= 5000) + } +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/protocol/ProtocolTests.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/protocol/ProtocolTests.scala new file mode 100644 index 00000000..93d3583a --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/protocol/ProtocolTests.scala @@ -0,0 +1,257 @@ +package org.codefeedr.plugins.clearlydefined.protocol + +import org.scalatest.FunSuite +import org.codefeedr.plugins.clearlydefined.protocol.Protocol._ + +class ProtocolTests extends FunSuite { + /** all full example classes */ + val cdDescribedUrls: CDDescribedUrls = CDDescribedUrls("registry", "version", "download") + val cdDescribedHashes: CDDescribedHashes = CDDescribedHashes(Some("gitSha"), Some("sha1"), Some("sha256")) + val cdDescribedToolScore: CDDescribedToolScore = CDDescribedToolScore(1, 2, 3) + val cdDescribedSourceLocation: CDDescribedSourceLocation = CDDescribedSourceLocation("locationType", "provider", "namespace", "name", "revision", + "url") + val cdDescribedScore: CDDescribedScore = CDDescribedScore(4, 5, 6) + val cdLicensedToolScore: CDLicensedToolScore = CDLicensedToolScore(11, 12, 13, 14, 15, 16) + val cdlfCoreAttribution: CDLFCoreAttribution = CDLFCoreAttribution(1, Some(List("parties1", "parties2"))) + val cdlfCoreDiscovered: CDLFCoreDiscovered = CDLFCoreDiscovered(1, List("expressions1", "expressions2")) + val cdlfCore: CDLFCore = CDLFCore(cdlfCoreAttribution, cdlfCoreDiscovered, 1) + val cdLicensedFacets: CDLicensedFacets = CDLicensedFacets(cdlfCore) + val cdLicensedScore: CDLicensedScore = CDLicensedScore(117, 118, 119, 120, 121, 122) + val cdLicensed: CDLicensed = CDLicensed(Some("declared"), cdLicensedToolScore, cdLicensedFacets, cdLicensedScore) + val cdCoordinates: CDCoordinates = CDCoordinates("type", "provider", "name", Some("namespace"), "revision") + val cd_meta: CD_meta = CD_meta("schemaVersion", "2020-02-04T10:38:28.427Z") + val cdScores: CDScores = CDScores(7, 8) + val cdDescribed: CDDescribed = CDDescribed("releaseDate", cdDescribedUrls, Some("projectWebsite"), Some("issueTracker"), + cdDescribedHashes, 1, List("tools1", "tools2"), cdDescribedToolScore, Some(cdDescribedSourceLocation), + cdDescribedScore) + + val cdRelease: ClearlyDefinedRelease = ClearlyDefinedRelease(cdDescribed, cdLicensed, cdCoordinates, cd_meta, cdScores) + + /** all classes with potential None fields */ + val cdDescribedHashesEmpty: CDDescribedHashes = CDDescribedHashes(None, None, None) + val cdDescribedEmpty: CDDescribed = CDDescribed("releaseDate", cdDescribedUrls, None, None, cdDescribedHashesEmpty, 1, + List("tools1", "tools2"), cdDescribedToolScore, None, cdDescribedScore) + val cdCoordinatesEmpty: CDCoordinates = CDCoordinates("type", "provider", "name", None, "revision") + + val cdReleaseEmpty: ClearlyDefinedRelease = ClearlyDefinedRelease(cdDescribedEmpty, cdLicensed, cdCoordinatesEmpty, cd_meta, cdScores) + + test("ClearlyDefinedReleasePojo convert success") { + val pojo = ClearlyDefinedReleasePojo.fromClearlyDefinedRelease(cdRelease) + + // Assert various normal members and None members + assert(pojo.described.releaseDate.equals("releaseDate")) + assert(pojo.described.issueTracker.equals("issueTracker")) + assert(pojo.licensed.score.spdx == 121) + assert(pojo.coordinates.namespace.equals("namespace")) + assert(pojo._meta.updated.toString.equals("2020-02-04 10:38:28.427")) + assert(pojo.scores.effective == 7) + } + + test("ClearlyDefinedReleasePojo convert with None fields success") { + val pojo = ClearlyDefinedReleasePojo.fromClearlyDefinedRelease(cdReleaseEmpty) + + // Assert none fields + assert(pojo.described.issueTracker == null) + assert(pojo.described.projectWebsite == null) + assert(pojo.described.sourceLocation == null) + assert(pojo.described.hashes.gitSha == null) + assert(pojo.described.hashes.sha1 == null) + assert(pojo.described.hashes.sha256 == null) + assert(pojo.coordinates.namespace == null) + } + + test("CDDescribed convert success") { + val pojo = CDDescribedPojo.fromCDDescribed(cdDescribed) + + // Assert fields + assert(pojo.releaseDate.equals("releaseDate")) + assert(pojo.projectWebsite.equals("projectWebsite")) + assert(pojo.issueTracker.equals("issueTracker")) + assert(pojo.files == 1) + assert(pojo.tools.size==2) + assert(pojo.tools.head.toolname=="tools1") + assert(pojo.tools.last.toolname=="tools2") + + // Assert complex fields + assert(pojo.urls.isInstanceOf[CDDescribedUrlsPojo]) + assert(pojo.hashes.isInstanceOf[CDDescribedHashesPojo]) + assert(pojo.toolScore.isInstanceOf[CDDescribedToolScorePojo]) + assert(pojo.sourceLocation.isInstanceOf[CDDescribedSourceLocationPojo]) + assert(pojo.score.isInstanceOf[CDDescribedScorePojo]) + } + + test("CDDescribed convert with none fields success") { + val pojo = CDDescribedPojo.fromCDDescribed(cdDescribedEmpty) + + // Assert fields + None's + assert(pojo.releaseDate.equals("releaseDate")) + assert(pojo.projectWebsite == null) + assert(pojo.issueTracker == null) + assert(pojo.files == 1) + assert(pojo.tools.head.toolname=="tools1") + assert(pojo.tools.last.toolname=="tools2") + assert(pojo.sourceLocation == null) + } + + test("CDDescribedUrls convert success") { + val pojo = CDDescribedUrlsPojo.fromCDDescribedUrls(cdDescribedUrls) + + // Assert fields + assert(pojo.registry.equals("registry")) + assert(pojo.version.equals("version")) + assert(pojo.download.equals("download")) + } + + test("CDDescribedHashes convert success") { + val pojo = CDDescribedHashesPojo.fromCDDescribedHashes(cdDescribedHashes) + + // Assert fields + assert(pojo.gitSha.equals("gitSha")) + assert(pojo.sha1.equals("sha1")) + assert(pojo.sha256.equals("sha256")) + } + + test("CDDescribedHashes convert with None fields success") { + val pojo = CDDescribedHashesPojo.fromCDDescribedHashes(cdDescribedHashesEmpty) + + // Assert fields + assert(pojo.gitSha == null) + assert(pojo.sha1 == null) + assert(pojo.sha256 == null) + } + + test("CDDescribedToolScore convert success") { + val pojo = CDDescribedToolScorePojo.fromCDDescribedToolScore(cdDescribedToolScore) + + // Assert fields + assert(pojo.total == 1) + assert(pojo.date == 2) + assert(pojo.source == 3) + } + + test("CDDescribedSourceLocation convert success") { + val pojo = CDDescribedSourceLocationPojo.fromCDDescribedSourceLocation(cdDescribedSourceLocation) + + // Assert fields + assert(pojo.locationType.equals("locationType")) + assert(pojo.provider.equals("provider")) + assert(pojo.namespace.equals("namespace")) + assert(pojo.name.equals("name")) + assert(pojo.revision.equals("revision")) + assert(pojo.url.equals("url")) + } + + test("CDDescribedScore convert success") { + val pojo = CDDescribedScorePojo.fromCDDescribedScore(cdDescribedScore) + + // Assert fields + assert(pojo.total == 4) + assert(pojo.date == 5) + assert(pojo.source == 6) + } + + test("CDLicensed convert success") { + val pojo = CDLicensedPojo.fromCDLicensed(cdLicensed) + + // Assert fields + assert(pojo.declared.equals("declared")) + assert(pojo.toolScore.isInstanceOf[CDLicensedToolScorePojo]) + assert(pojo.facets.isInstanceOf[CDLicensedFacetsPojo]) + assert(pojo.score.isInstanceOf[CDLicensedScorePojo]) + } + + test("CDLicensedToolScore convert success") { + val pojo = CDLicensedToolScorePojo.fromCDLicensedToolScore(cdLicensedToolScore) + + // Assert fields + assert(pojo.total == 11) + assert(pojo.declared == 12) + assert(pojo.discovered == 13) + assert(pojo.consistency == 14) + assert(pojo.spdx == 15) + assert(pojo.texts == 16) + } + + test("CDLicensedFacets convert success") { + val pojo = CDLicensedFacetsPojo.fromCDLicensedFacets(cdLicensedFacets) + + // Assert fields + assert(pojo.core.isInstanceOf[CDLFCorePojo]) + } + + test("CDLFCore convert success") { + val pojo = CDLFCorePojo.fromCDLFCore(cdlfCore) + + // Assert fields + assert(pojo.attribution.isInstanceOf[CDLFCoreAttributionPojo]) + assert(pojo.discovered.isInstanceOf[CDLFCoreDiscoveredPojo]) + assert(pojo.files == 1) + } + + test("CDLFCoreAttribution convert success") { + val pojo = CDLFCoreAttributionPojo.fromCDLFCoreAttribution(cdlfCoreAttribution) + + // Assert fields + assert(pojo.unknown == 1) + assert(pojo.parties.head.equals("parties1")) + } + + test("CDLFCoreDiscovered convert success") { + val pojo = CDLFCoreDiscoveredPojo.fromCDLFCoreDiscovered(cdlfCoreDiscovered) + + // Assert fields + assert(pojo.unknown == 1) + assert(pojo.expressions.head.equals("expressions1")) + } + + test("CDLicensedScore convert success") { + val pojo = CDLicensedScorePojo.fromCDLicensedScore(cdLicensedScore) + + // Assert fields + assert(pojo.total == 117) + assert(pojo.declared == 118) + assert(pojo.discovered == 119) + assert(pojo.consistency == 120) + assert(pojo.spdx == 121) + assert(pojo.texts == 122) + } + + test("CDCoordinates convert success") { + val pojo = CDCoordinatesPojo.fromCDCoordinates(cdCoordinates) + + // Assert fields + assert(pojo.`type`.equals("type")) + assert(pojo.provider.equals("provider")) + assert(pojo.name.equals("name")) + assert(pojo.namespace.equals("namespace")) + assert(pojo.revision.equals("revision")) + } + + test("CDCoordinates convert with None fields success") { + val pojo = CDCoordinatesPojo.fromCDCoordinates(cdCoordinatesEmpty) + + // Assert fields + assert(pojo.`type`.equals("type")) + assert(pojo.provider.equals("provider")) + assert(pojo.name.equals("name")) + assert(pojo.namespace == null) + assert(pojo.revision.equals("revision")) + } + + test("CD_meta convert success") { + val pojo = CD_metaPojo.fromCD_meta(cd_meta) + + // Assert fields + assert(pojo.schemaVersion.equals("schemaVersion")) + assert(pojo.updated.toString.equals("2020-02-04 10:38:28.427")) + } + + test("CDScores convert success") { + val pojo = CDScoresPojo.fromCDScores(cdScores) + + // Assert fields + assert(pojo.effective == 7) + assert(pojo.tool == 8) + } + +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/resources/ClearlyDefinedPackageSnapshot.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/resources/ClearlyDefinedPackageSnapshot.scala new file mode 100644 index 00000000..335ebae3 --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/resources/ClearlyDefinedPackageSnapshot.scala @@ -0,0 +1,93 @@ +package org.codefeedr.plugins.clearlydefined.resources + +object ClearlyDefinedPackageSnapshot { + + val snapshot = """{ + "described": { + "releaseDate": "2019-12-03", + "urls": { + "registry": "https://github.com/sweetalert2/sweetalert2", + "version": "https://github.com/sweetalert2/sweetalert2/tree/618540bad0f548d47ff1b0b3d118e81ef41b627b", + "download": "https://github.com/sweetalert2/sweetalert2/archive/618540bad0f548d47ff1b0b3d118e81ef41b627b.zip" + }, + "hashes": { + "gitSha": "618540bad0f548d47ff1b0b3d118e81ef41b627b" + }, + "files": 154, + "tools": [ + "clearlydefined/1.3.0", + "licensee/9.13.0", + "scancode/3.2.2" + ], + "toolScore": { + "total": 100, + "date": 30, + "source": 70 + }, + "sourceLocation": { + "type": "git", + "provider": "github", + "namespace": "sweetalert2", + "name": "sweetalert2", + "revision": "618540bad0f548d47ff1b0b3d118e81ef41b627b", + "url": "https://github.com/sweetalert2/sweetalert2/tree/618540bad0f548d47ff1b0b3d118e81ef41b627b" + }, + "score": { + "total": 100, + "date": 30, + "source": 70 + } + }, + "licensed": { + "declared": "MIT", + "toolScore": { + "total": 75, + "declared": 30, + "discovered": 0, + "consistency": 15, + "spdx": 15, + "texts": 15 + }, + "facets": { + "core": { + "attribution": { + "unknown": 153, + "parties": [ + "Copyright (c) 2014 Tristan Edwards & Limon Monte" + ] + }, + "discovered": { + "unknown": 152, + "expressions": [ + "MIT" + ] + }, + "files": 154 + } + }, + "score": { + "total": 75, + "declared": 30, + "discovered": 0, + "consistency": 15, + "spdx": 15, + "texts": 15 + } + }, + "coordinates": { + "type": "git", + "provider": "github", + "namespace": "sweetalert2", + "name": "sweetalert2", + "revision": "618540bad0f548d47ff1b0b3d118e81ef41b627b" + }, + "_meta": { + "schemaVersion": "1.6.1", + "updated": "2019-12-03T13:58:56.579Z" + }, + "scores": { + "effective": 87, + "tool": 87 + } +}""" +} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStageTests.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStageTests.scala new file mode 100644 index 00000000..eb0b519a --- /dev/null +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/test/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStageTests.scala @@ -0,0 +1,55 @@ +package org.codefeedr.plugins.clearlydefined.stages + +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.codefeedr.stages.OutputStage +import org.scalatest.FunSuite +import org.codefeedr.plugins.clearlydefined.operators.ClearlyDefinedSourceConfig +import org.codefeedr.plugins.clearlydefined.protocol.Protocol.ClearlyDefinedRelease + +class ClearlyDefinedReleasesStageTests extends FunSuite { + + test("ClearlyDefinedReleasesIntegrationTest") { + val source = new ClearlyDefinedReleasesStage(sourceConfig = ClearlyDefinedSourceConfig(1000, 10, 4)) + val sink = new LimitingSinkStage(10) + + val pipeline: Unit = new PipelineBuilder() + .append(source) + .append(sink) + .build() + .startMock() + } +} + +// Simple Sink Pipeline Object that limits the output to a certain number +// and is able to get a list of all the items that were received in the sink +class LimitingSinkStage(elements: Int = -1) + extends OutputStage[ClearlyDefinedRelease] + with Serializable { + var sink: LimitingSink = _ + + override def main(source: DataStream[ClearlyDefinedRelease]): Unit = { + sink = new LimitingSink(elements) + source.addSink(sink).setParallelism(1) + } +} + +class LimitingSink(elements: Int) extends SinkFunction[ClearlyDefinedRelease] { + var count = 0 + var items: List[ClearlyDefinedRelease] = List() + + override def invoke(value: ClearlyDefinedRelease, + context: SinkFunction.Context[_]): Unit = { + count += 1 + items = value :: items + + println(count) + + if (elements != -1 && count >= elements) { + throw new RuntimeException() + } + } + +} + diff --git a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTAbstractEventStage.scala b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTAbstractEventStage.scala index 0259160a..150a0c37 100644 --- a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTAbstractEventStage.scala +++ b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTAbstractEventStage.scala @@ -83,7 +83,8 @@ protected class GHTAbstractEventStage[ new FlinkKafkaProducer[Record]( sideOutput.sideOutputTopic, Serializer.getSerde[Record](Serializer.JSON), - props)) + props, + FlinkKafkaProducer.Semantic.NONE)) } trans diff --git a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTCommitStage.scala b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTCommitStage.scala index 10378531..8728f8be 100644 --- a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTCommitStage.scala +++ b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTCommitStage.scala @@ -67,7 +67,8 @@ class GHTCommitStage(stageName: String = "ght_commit", new FlinkKafkaProducer[Record]( sideOutput.sideOutputTopic, Serializer.getSerde[Record](Serializer.JSON), - props)) + props, + FlinkKafkaProducer.Semantic.NONE)) } trans diff --git a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTInputStage.scala b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTInputStage.scala index 56a262d3..e62c3434 100644 --- a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTInputStage.scala +++ b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/stages/GHTInputStage.scala @@ -52,7 +52,12 @@ class GHTInputStage(username: String, override def main(context: Context): DataStream[Record] = { context.env .addSource( - new GHTorrentRabbitMQSource(username, host, port, routingKeysFile, password = password, virtualHost = virtualHost)) + new GHTorrentRabbitMQSource(username, + host, + port, + routingKeysFile, + password = password, + virtualHost = virtualHost)) .map { x => val splitEl = x.split("#", 2) val routingKey = splitEl(0) diff --git a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/util/GHTorrentRabbitMQSource.scala b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/util/GHTorrentRabbitMQSource.scala index 01277006..7b3904d1 100644 --- a/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/util/GHTorrentRabbitMQSource.scala +++ b/codefeedr-plugins/codefeedr-ghtorrent/src/main/scala/org/codefeedr/plugins/ghtorrent/util/GHTorrentRabbitMQSource.scala @@ -203,7 +203,7 @@ class GHTorrentRabbitMQSource(username: String = "streamer", if (!autoAck) { //If autoAck is disabled, we provide the delivery tag to a list of sessionIds. val deliveryTag = envelope.getDeliveryTag - /** if (usesCorrelationId) { + /** if (usesCorrelationId) { val correlationId = properties.getCorrelationId Preconditions.checkNotNull( diff --git a/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonExitStage.scala b/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonExitStage.scala new file mode 100644 index 00000000..bada0b56 --- /dev/null +++ b/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonExitStage.scala @@ -0,0 +1,28 @@ +package org.codefeedr.plugins.json + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.stages.OutputStage +import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization.write + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + +/** + * A stage which prints an object to json representation of the objects in the incoming datastream + * + * @param classTag$T The classTag of the input stream + * @param typeTag$T The typeTag of the input stream + * @tparam T The type of input stream + */ +class JsonExitStage[T <: Serializable with AnyRef: ClassTag: TypeTag] + extends OutputStage[T] { + override def main(source: DataStream[T]): Unit = { + implicit val typeInfo: TypeInformation[Unit] = + TypeInformation.of(classOf[Unit]) + implicit lazy val formats: DefaultFormats.type = DefaultFormats + source.map((x: T) => println(write(x))) + } + +} diff --git a/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonTransformStage.scala b/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonTransformStage.scala new file mode 100644 index 00000000..5a93714c --- /dev/null +++ b/codefeedr-plugins/codefeedr-json/src/main/scala/org/codefeedr/plugins/json/JsonTransformStage.scala @@ -0,0 +1,35 @@ +package org.codefeedr.plugins.json + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.stages.TransformStage +import org.json4s.DefaultFormats +import org.json4s.jackson.Serialization.write + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + +/** + * Since CodeFeedr stages require an input with AnyRef, this wrapper class is used to wrap a String + * + * @param s The string inside the wrapper class + */ +case class StringWrapper(s: String) + +/** + * This stage transforms a stream to a stream of StringWrapper with the json representation + * of the incoming stream + * + * @param classTag$T The classTag of the input stream + * @param typeTag$T The typeTag of the input stream + * @tparam T Type of the input stream + */ +class JsonTransformStage[T <: Serializable with AnyRef: ClassTag: TypeTag] + extends TransformStage[T, StringWrapper] { + override def transform(source: DataStream[T]): DataStream[StringWrapper] = { + implicit val typeInfo: TypeInformation[StringWrapper] = + TypeInformation.of(classOf[StringWrapper]) + implicit lazy val formats: DefaultFormats.type = DefaultFormats + source.map((x: T) => StringWrapper(write(x)))(typeInfo) + } +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSource.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSource.scala new file mode 100644 index 00000000..7a4ca4e9 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSource.scala @@ -0,0 +1,117 @@ +package org.codefeedr.plugins.maven.operators + +import java.text.SimpleDateFormat + +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.codefeedr.stages.utilities.{HttpRequester, RequestException} +import org.codefeedr.pipeline.{PluginReleasesSource, PluginSourceConfig} +import org.codefeedr.plugins.maven.protocol.Protocol.{Guid, MavenRelease} +import scalaj.http.Http + +import scala.xml.XML + +case class MavenSourceConfig(pollingInterval: Int = 60000, // 1 min interval + maxNumberOfRuns: Int = -1, + timeout: Int = 32) + extends PluginSourceConfig + +class MavenReleasesSource(config: MavenSourceConfig = MavenSourceConfig()) + extends PluginReleasesSource[MavenRelease](config) { + // Date formats + URL + val pubDateFormat = "EEE, dd MMM yyyy HH:mm:ss ZZ" + val url = "https://mvnrepository.com/feeds/rss2.0.xml" + + /** Runs the source. + * + * @param ctx the source the context. + */ + override def run(ctx: SourceFunction.SourceContext[MavenRelease]): Unit = { + val lock = ctx.getCheckpointLock + + /** While is running or #runs left. */ + while (isRunning && runsLeft != 0) { + lock.synchronized { // Synchronize to the checkpoint lock. + try { + // Polls the RSS feed + val rssAsString = getRSSAsString + // Parses the received rss items + val items: Seq[MavenRelease] = parseRSSString(rssAsString) + // Collect right items and update last item + val validSortedItems = sortAndDropDuplicates(items) + // Decrease runs left + super.decreaseRunsLeft() + // Timestamp the items + validSortedItems.foreach(x => + ctx.collectWithTimestamp(x, x.pubDate.getTime)) + // Call parent run + super.runPlugin(ctx, validSortedItems) + } catch { + case _: Throwable => + } + } + } + } + + /** + * Requests the RSS feed and returns its body as a string. + * Will keep trying with increasing intervals if it doesn't succeed + * + * @return Body of requested RSS feed + */ + @throws[RequestException] + def getRSSAsString: String = { + new HttpRequester().retrieveResponse(Http(url)).body + } + + /** + * Parses a string that contains xml with RSS items + * + * @param rssString XML string with RSS items + * @return Sequence of RSS items + */ + def parseRSSString(rssString: String): Seq[MavenRelease] = { + try { + val xml = XML.loadString(rssString) + val nodes = xml \\ "item" + for (t <- nodes) yield xmlToMavenRelease(t) + } catch { + // If the string cannot be parsed return an empty list + case _: Throwable => Nil + } + } + + /** + * Parses a xml node to a RSS item + * + * @param node XML node + * @return RSS item + */ + def xmlToMavenRelease(node: scala.xml.Node): MavenRelease = { + val title = (node \ "title").text + val link = (node \ "link").text + val description = (node \ "description").text + val formatterPub = new SimpleDateFormat(pubDateFormat) + val pubDate = formatterPub.parse((node \ "pubDate").text) + + val tag = (node \ "guid").text + MavenRelease(title, link, description, pubDate, Guid(tag)) + } + + /** + * Drops items that already have been collected and sorts them based on times + * + * @param items Potential items to be collected + * @return Valid sorted items + */ + def sortAndDropDuplicates(items: Seq[MavenRelease]): Seq[MavenRelease] = { + items + .filter((x: MavenRelease) => { + if (lastItem.isDefined) { + lastItem.get.pubDate.before(x.pubDate) && lastItem.get.link != x.link + } else + true + }) + .sortWith((x: MavenRelease, y: MavenRelease) => + x.pubDate.before(y.pubDate)) + } +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/RetrieveProjectAsync.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/RetrieveProjectAsync.scala new file mode 100644 index 00000000..c9c7ced9 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/operators/RetrieveProjectAsync.scala @@ -0,0 +1,76 @@ +package org.codefeedr.plugins.maven.operators + +import org.apache.flink.streaming.api.functions.async.{ + ResultFuture, + RichAsyncFunction +} +import org.codefeedr.plugins.maven.protocol.Protocol.{ + MavenProject, + MavenRelease, + MavenReleaseExt +} +import org.codefeedr.plugins.maven.util.MavenService + +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} +import collection.JavaConverters._ + +/** Retrieves a project related to a release asynchronously. */ +class RetrieveProjectAsync + extends RichAsyncFunction[MavenRelease, MavenReleaseExt] { + + /** Retrieve the execution context lazily. */ + implicit lazy val executor: ExecutionContext = ExecutionContext.global + + /** Async retrieves the project belonging to the release. + * + * @param input the release. + * @param resultFuture the future to add the project to. + */ + override def asyncInvoke( + input: MavenRelease, + resultFuture: ResultFuture[MavenReleaseExt]): Unit = { + + val projectName = transformProjectName(input) + + /** Retrieve the project in a Future. */ + val requestProject: Future[Option[MavenProject]] = Future( + MavenService.getProject(projectName)) + + /** Collects the result. */ + requestProject.onComplete { + case Success(result: Option[MavenProject]) => + if (result.isDefined) { + resultFuture.complete( + List( + MavenReleaseExt(input.title, + input.link, + input.description, + input.pubDate, + input.guid, + result.get)).asJava) + } else resultFuture.complete(List().asJava) + case Failure(e) => + resultFuture.complete(List().asJava) + e.printStackTrace() + } + } + + /** + * Transform the title of a project to be retrieved by the MavenService + * @param input The project of which the name needs to be transformed + * @return the transformed project name + */ + private def transformProjectName(input: MavenRelease): String = { + val splitTitle = input.title.split(" ") + val org = splitTitle(0).replace(".", "/").replace(":", "/") + val name = splitTitle(1).replace(" ", "/") + val version = splitTitle(2) + org + name + "/" + version + "/" + name + "-" + version + ".pom" + } + + /** If we retrieve a time-out, then we just complete the future with an empty list. */ + override def timeout(input: MavenRelease, + resultFuture: ResultFuture[MavenReleaseExt]): Unit = + resultFuture.complete(List().asJava) +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala new file mode 100644 index 00000000..91f416c0 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala @@ -0,0 +1,378 @@ +package org.codefeedr.plugins.maven.protocol + +import java.io.Serializable +import java.util.Date + +object Protocol { + + /** + * A POJO style representation of the MavenRelease case class + */ + class MavenReleasePojo extends Serializable { + var title: String = _ + var link: String = _ + var description: String = _ + var pubDate: Long = _ + var guid_tag: String = _ + } + + object MavenReleasePojo { + def fromMavenRelease(release: MavenRelease): MavenReleasePojo = { + val pojo = new MavenReleasePojo + pojo.title = release.title + pojo.link = release.link + pojo.description = release.description + pojo.pubDate = release.pubDate.getTime + pojo.guid_tag = release.guid.tag + pojo + } + } + + class MavenReleaseExtPojo extends Serializable { + var title: String = _ + var link: String = _ + var description: String = _ + var pubDate: Long = _ + var guid_tag: String = _ + var project: MavenProjectPojo = _ + } + + object MavenReleaseExtPojo { + def fromMavenReleaseExt( + releaseExt: MavenReleaseExt + ): MavenReleaseExtPojo = { + val pojo = new MavenReleaseExtPojo + pojo.title = releaseExt.title + pojo.link = releaseExt.link + pojo.description = releaseExt.description + pojo.pubDate = releaseExt.pubDate.getTime + pojo.guid_tag = releaseExt.guid.tag + pojo.project = MavenProjectPojo.fromMavenProject(releaseExt.project) + pojo + } + } + + class ParentPojo extends Serializable { + var groupId: String = _ + var artifactId: String = _ + var version: String = _ + var relativePath: String = _ + } + + class ParentPojoExt extends ParentPojo { + var childId: String = _ + } + + object ParentPojo { + def fromParent(parent: Parent): ParentPojo = { + val pojo = new ParentPojo + pojo.groupId = parent.groupId + pojo.artifactId = parent.artifactId + pojo.version = parent.version + pojo.relativePath = parent.relativePath match { + case None => "" + case Some(x) => x + } + pojo + } + } + + class MavenProjectPojo extends Serializable { + var modelVersion: String = _ + var groupId: String = _ + var artifactId: String = _ + var version: String = _ + var parent: ParentPojo = _ + var dependencies: List[DependencyPojo] = _ + var licenses: List[LicensePojo] = _ + var repositories: List[RepositoryPojo] = _ + var organization: OrganizationPojo = _ + var packaging: String = _ + var issueManagement: IssueManagementPojo = _ + var scm: SCMPojo = _ + } + + object MavenProjectPojo { + def fromMavenProject(mavenProject: MavenProject): MavenProjectPojo = { + val pojo = new MavenProjectPojo + pojo.modelVersion = mavenProject.modelVersion + pojo.groupId = mavenProject.groupId + pojo.artifactId = mavenProject.artifactId + pojo.version = mavenProject.version + + // Set the parent + if (mavenProject.parent.isDefined) { + pojo.parent = ParentPojo.fromParent(mavenProject.parent.get) + } + + // Map the dependencies + if (mavenProject.dependencies.isDefined) { + pojo.dependencies = mavenProject.dependencies.get.map(x => { + DependencyPojo.fromDependency(x) + }) + } + + // Map the licenses + if (mavenProject.licenses.isDefined) { + pojo.licenses = mavenProject.licenses.get.map(x => { + LicensePojo.fromLicense(x) + }) + } + + // Map the repositories + if (mavenProject.repositories.isDefined) { + pojo.repositories = mavenProject.repositories.get.map(x => { + RepositoryPojo.fromRepository(x) + }) + } + + // Set the organization + if (mavenProject.organization.isDefined) { + pojo.organization = + OrganizationPojo.fromOrganization(mavenProject.organization.get) + } + + // Set the packaging + if (mavenProject.packaging.isDefined) { + pojo.packaging = mavenProject.packaging.get + } + + // Set the issueManagement + if (mavenProject.issueManagement.isDefined) { + pojo.issueManagement = IssueManagementPojo.fromIssueManagement( + mavenProject.issueManagement.get + ) + } + + // Set the SCM + if (mavenProject.scm.isDefined) { + pojo.scm = SCMPojo.fromSCM(mavenProject.scm.get) + } + + pojo + } + } + + class MavenProjectPojoExt extends MavenProjectPojo { + var title: String = _ + } + + class DependencyPojo extends Serializable { + var groupId: String = _ + var artifactId: String = _ + var version: String = _ + var `type`: String = _ + var scope: String = _ + var optional: Boolean = _ + } + + class DependencyPojoExt extends DependencyPojo { + var projectId: String = _ + } + + object DependencyPojo { + def fromDependency(dependency: Dependency): DependencyPojo = { + val pojo = new DependencyPojo + pojo.groupId = dependency.groupId + pojo.artifactId = dependency.artifactId + pojo.version = dependency.version match { + case None => "" + case Some(x) => x + } + pojo.`type` = dependency.`type` match { + case None => "" + case Some(x) => x + } + pojo.scope = dependency.scope match { + case None => "" + case Some(x) => x + } + if (dependency.optional.isDefined) { + pojo.optional = dependency.optional.get + } + pojo + } + } + + class LicensePojo extends Serializable { + var name: String = _ + var url: String = _ + var distribution: String = _ + var comments: String = _ + } + + class LicensePojoExt extends LicensePojo { + var projectId: String = _ + } + + object LicensePojo { + def fromLicense(license: License): LicensePojo = { + val pojo = new LicensePojo + pojo.name = license.name + pojo.url = license.url + pojo.distribution = license.distribution + pojo.comments = license.comments match { + case None => "" + case Some(x) => x + } + pojo + } + } + + class RepositoryPojo extends Serializable { + var id: String = _ + var name: String = _ + var url: String = _ + } + + class RepositoryPojoExt extends RepositoryPojo { + var projectId: String = _ + } + + object RepositoryPojo { + def fromRepository(repository: Repository): RepositoryPojo = { + val pojo = new RepositoryPojo + pojo.id = repository.id + pojo.name = repository.name + pojo.url = repository.url + pojo + } + } + + class OrganizationPojo extends Serializable { + var name: String = _ + var url: String = _ + } + + class OrganizationPojoExt extends OrganizationPojo { + var root_id: String = _ + } + + object OrganizationPojo { + def fromOrganization(organization: Organization): OrganizationPojo = { + val pojo = new OrganizationPojo + pojo.name = organization.name + pojo.url = organization.url + pojo + } + } + + class IssueManagementPojo extends Serializable { + var system: String = _ + var url: String = _ + } + + class IssueManagementPojoExt extends IssueManagementPojo { + var root_id: String = _ + } + + object IssueManagementPojo { + def fromIssueManagement( + issueManagement: IssueManagement + ): IssueManagementPojo = { + val pojo = new IssueManagementPojo + pojo.system = issueManagement.system + pojo.url = issueManagement.url + pojo + } + } + + class SCMPojo extends Serializable { + var connection: String = _ + var developerConnection: String = _ + var tag: String = _ + var url: String = _ + } + + class SCMPojoExt extends SCMPojo { + var root_id: String = _ + } + + object SCMPojo { + def fromSCM(scm: SCM): SCMPojo = { + val pojo = new SCMPojo + pojo.connection = scm.connection + pojo.developerConnection = scm.developerConnection match { + case None => "" + case Some(x) => x + } + pojo.tag = scm.tag match { + case None => "" + case Some(x) => x + } + pojo.url = scm.url + pojo + } + } + + case class MavenRelease( + title: String, + link: String, + description: String, + pubDate: Date, + guid: Guid + ) + + case class Guid(tag: String) + + case class MavenReleaseExt( + title: String, + link: String, + description: String, + pubDate: Date, + guid: Guid, + project: MavenProject + ) + + case class MavenProject( + modelVersion: String, + groupId: String, + artifactId: String, + version: String, + parent: Option[Parent], + dependencies: Option[List[Dependency]], + licenses: Option[List[License]], + repositories: Option[List[Repository]], + organization: Option[Organization], + packaging: Option[String], + issueManagement: Option[IssueManagement], + scm: Option[SCM] + ) + + case class SCM( + connection: String, + developerConnection: Option[String], + tag: Option[String], + url: String + ) + + case class Organization(name: String, url: String) + + case class IssueManagement(system: String, url: String) + + case class Parent( + groupId: String, + artifactId: String, + version: String, + relativePath: Option[String] + ) + + case class License( + name: String, + url: String, + distribution: String, + comments: Option[String] + ) + + case class Repository(id: String, name: String, url: String) + + case class Dependency( + groupId: String, + artifactId: String, + version: Option[String], + `type`: Option[String], + scope: Option[String], + optional: Option[Boolean] + ) + +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala new file mode 100644 index 00000000..aabc127c --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala @@ -0,0 +1,40 @@ +package org.codefeedr.plugins.maven.stages + +import java.util.concurrent.TimeUnit + +import org.apache.flink.streaming.api.datastream.{ + AsyncDataStream => JavaAsyncDataStream +} +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.stages.TransformStage +import org.codefeedr.plugins.maven.operators.RetrieveProjectAsync +import org.codefeedr.plugins.maven.protocol.Protocol.{ + MavenRelease, + MavenReleaseExt +} + +/** Transform a [[MavenRelease]] to [[MavenReleaseExt]]. + * + * @param stageId the name of this stage. + */ +class MavenReleasesExtStage(stageId: String = "maven_releases") + extends TransformStage[MavenRelease, MavenReleaseExt](Some(stageId)) { + + /** Transform a [[MavenRelease]] to [[MavenReleaseExt]]. + * + * @param source The input source with type [[MavenRelease]]. + * @return The transformed stream with type [[MavenReleaseExt]]. + */ + override def transform( + source: DataStream[MavenRelease]): DataStream[MavenReleaseExt] = { + + /** Retrieve project from release asynchronously. */ + val async = JavaAsyncDataStream.orderedWait(source.javaStream, + new RetrieveProjectAsync, + 5, + TimeUnit.SECONDS, + 100) + + new org.apache.flink.streaming.api.scala.DataStream(async) + } +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala new file mode 100644 index 00000000..67a17da9 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala @@ -0,0 +1,40 @@ +package org.codefeedr.plugins.maven.stages + +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.Context +import org.codefeedr.stages.InputStage +import org.codefeedr.plugins.maven.protocol.Protocol.MavenRelease +import org.codefeedr.plugins.maven.operators.{ + MavenReleasesSource, + MavenSourceConfig +} +import org.codefeedr.stages.utilities.DefaultTypeMapper._ + +import scala.language.higherKinds + +/** Fetches real-time releases from Maven. */ +class MavenReleasesStage(stageId: String = "maven_releases_min", + sourceConfig: MavenSourceConfig = + MavenSourceConfig(1000, -1)) + extends InputStage[MavenRelease](Some(stageId)) { + + /** Fetches [[MavenRelease]] from real-time Maven feed. + * + * @param context The context to add the source to. + * @return The stream of type [[MavenRelease]]. + */ + override def main(context: Context): DataStream[MavenRelease] = { + implicit val typeInfo: TypeInformation[MavenRelease] = + TypeInformation.of(classOf[MavenRelease]) + context.env + .addSource(new MavenReleasesSource(sourceConfig))(typeInfo) + } + + override def getSchema: Schema = { + implicit val MavenSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[MavenRelease] + } +} diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/util/MavenService.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/util/MavenService.scala new file mode 100644 index 00000000..39410dbf --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/util/MavenService.scala @@ -0,0 +1,344 @@ +package org.codefeedr.plugins.maven.util + +import org.apache.logging.log4j.scala.Logging +import org.codefeedr.stages.utilities.HttpRequester +import org.codefeedr.plugins.maven.protocol.Protocol._ +import scalaj.http.Http + +import scala.util.control.Breaks._ +import scala.xml.{Node, XML} + +/** Services to retrieve a project from the Maven APi. */ +object MavenService extends Logging with Serializable { + + /** Retrieve the API url. */ + private val url = "https://repo1.maven.org/maven2/" + + /** Retrieves a Maven project. + * + * @param projectName the name of the project. + * @return an optional MavenProject. + */ + def getProject(projectName: String): Option[MavenProject] = { + + /** Retrieve the project. */ + val rawProject = getProjectRaw(projectName) + if (rawProject.isEmpty) { + logger.error(s"Couldn't retrieve Maven project with name $projectName.") + + return None + } + + var xml: scala.xml.Node = null + try { + xml = XML.loadString(rawProject.get) + } catch { + case _: Exception => + logger.error(s"Couldn't convert string to xml with name $projectName.") + return None + } + + /** Extract into an optional if it can't be parsed. */ + val project = xmlToMavenProject(xml) + + if (project.isEmpty) { + logger.error( + s"Couldn't retrieve Maven project with name $projectName and xml $xml.") + + return None + } + + /** Forward the project */ + project + } + + /** + * Turns xml of a pom.xml to a MavenProject + * + * @param node the xml containing the pom.xml + * @return an option of a MavenProject Some(project) if parsed successfully, else otherwise + */ + def xmlToMavenProject(node: scala.xml.Node): Option[MavenProject] = { + // Set initial values + // Modelversion is always the same, so read the value + val modelVersion = (node \ "modelVersion").text + + // Sometimes these are inherited from parent + var groupId = "" + var artifactId = "" + var version = "" + + // Get the parent of a project + val parent = parseParent(getInnerNode(node, "parent")) + + // Sometimes groupId, artifactId and version are inside parent, sometimes not + groupId = (node \ "groupId").text + artifactId = (node \ "artifactId").text + version = (node \ "version").text + + // If (one of) the fields is/are empty, inherit it/them from the parent + if (groupId == "") { + groupId = parent.get.groupId + } + if (artifactId == "") { + artifactId = parent.get.artifactId + } + if (version == "") { + version = parent.get.version + } + + // Get dependencies + val dependencies = + parseDependencies(getInnerNode(node, "dependencies"), version) + // Get licenses + val licenses = parseLicenses(getInnerNode(node, "licenses")) + // Get repositories (often empty) + val repositories = parseRepositories(getInnerNode(node, "repositories")) + // Get organization + val organization = parseOrganization(getInnerNode(node, "organization")) + + // Parse the packaging information + val packaging: Option[String] = checkEmpty((node \ "packaging").text) + + // Get IssueManagement + val issueManagement = parseIssueManagement( + getInnerNode(node, "issueManagement")) + // Get SCM + val scm = parseSCM(getInnerNode(node, "scm")) + + Some( + MavenProject(modelVersion, + groupId, + artifactId, + version, + parent, + dependencies, + licenses, + repositories, + organization, + packaging, + issueManagement, + scm)) + } + + /** + * This function is used as often a check is needed to determine whether a option field was filled or not + * + * @param in the string which needs to be checked if it is empty + * @return Option of $in if filled, None if $in == "" + */ + def checkEmpty(in: String): Option[String] = { + if (in == "") None else Some(in) + } + + /** + * Helper function for xmlToMavenProject as often we need to get an inner node with a certain label + * + * @param node the node from which to get the inner node + * @param label the label for the inner node + * @return a sequence of inner nodes corresponding to $label + */ + def getInnerNode(node: Node, label: String): Seq[Node] = { + node.child.filter(item => item.label == label) + } + + def parseSCM(nodes: Seq[Node]): Option[SCM] = { + if (nodes.isEmpty) { + None + } else { + val scm = nodes.head + val connection = (scm \ "connection").text + val devConnection = checkEmpty((scm \ "developerConnection").text) + val tag = checkEmpty((scm \ "tag").text) + val url = (scm \ "url").text + Some(SCM(connection, devConnection, tag, url)) + } + } + + /** + * Parses the issueManagement from a xml node + * + * @param nodes the node which (possibly) contains the organization information + * @return Option of IssueManagement, Some(IM) if $nodes was nonEmpty, None otherwise + */ + def parseIssueManagement(nodes: Seq[Node]): Option[IssueManagement] = { + if (nodes.isEmpty) { + None + } else { + val issueManagement = nodes.head + val system = (issueManagement \ "system").text + val url = (issueManagement \ "url").text + Some(IssueManagement(system, url)) + } + } + + /** + * Parses the organizatino from a xml node + * + * @param nodes the node which (possibly) contains the organization information + * @return Option of Organization, Some(org) if $nodes was nonEmpty, None otherwise + */ + def parseOrganization(nodes: Seq[Node]): Option[Organization] = { + if (nodes.isEmpty) { + None + } else { + val org = nodes.head + val name = (org \ "name").text + val url = (org \ "url").text + Some(Organization(name, url)) + } + } + + /** + * Parses the parent from a xml node + * + * @param parentNode the node which (possibly) contains the parent information + * @return Option of Parent, Some(parent) if $parentNode was nonEmpty, None otherwise + */ + def parseParent(parentNode: Seq[Node]): Option[Parent] = { + // If the parentNode is empty, return None + if (parentNode.isEmpty) { + None + } + // Else parse and return the information of the Parent xml + else { + val parent = parentNode.head + val groupId = (parent \ "groupId").text + val artifactId = (parent \ "artifactId").text + val version = (parent \ "version").text + val relativePath: Option[String] = checkEmpty( + (parent \ "relativePath").text) + Some(Parent(groupId, artifactId, version, relativePath)) + } + } + + /** + * Parses the repositories from a xml node + * + * @param nodes the nodes which (possibly) contain the repository information + * @return Option of List[Repository], Some(list) if $nodes was filled, None otherwise + */ + def parseRepositories(nodes: Seq[Node]): Option[List[Repository]] = { + //If there are repositories parse them + if (nodes.nonEmpty) { + var repoList = List[Repository]() + for (n <- nodes.head.child) { + breakable { + //The first node in the xml structure is always empty so we skip this one as it contains no info + if (n.toString().startsWith("\n")) { + break() + } + val currentId = (n \ "id").text + val currentName = (n \ "name").text + val currentUrl = (n \ "url").text + repoList = Repository(currentId, currentName, currentUrl) :: repoList + } + } + return Some(repoList.reverse) + } + //If there are no repositories, return None + None + } + + /** + * Parses the licenses from a xml node + * + * @param nodes the nodes which (possibly) contain the licenses information + * @return Option of List[License], Some(list) if $nodes was filled, None otherwise + */ + def parseLicenses(nodes: Seq[Node]): Option[List[License]] = { + //If there are licenses parse them + if (nodes.nonEmpty) { + var licensesList = List[License]() + for (n <- nodes.head.child) { + breakable { + //The first node in the xml structure is always empty so we skip this one as it contains no info + if (n.toString().startsWith("\n")) { + break() + } + val currentName = (n \ "name").text + val currentUrl = (n \ "url").text + val currentDistribution = (n \ "distribution").text + val currentComments: Option[String] = + checkEmpty((n \ "comments").text) + + licensesList = License(currentName, + currentUrl, + currentDistribution, + currentComments) :: licensesList + } + } + return Some(licensesList.reverse) + } + //If there are no licenses, return None + None + } + + /** + * Returns a list of Dependencies if the xml node contains them + * + * @param node the list of nodes from which the depencies will be extracted + * @return An option of a list of dependencies, Some(list) if $node contained them, None otherwise + */ + def parseDependencies(node: Seq[Node], + projectVersion: String): Option[List[Dependency]] = { + //If there are dependencies parse them + if (node.nonEmpty) { + var dependencyList = List[Dependency]() + for (n <- node.head.child) { + breakable { + //The first node in the xml structure is always empty so we skip this one as it contains no info + if (n.toString().startsWith("\n")) { + break() + } + val currentGroupId = (n \ "groupId").text + val currentArtifactId = (n \ "artifactId").text + var currentVersion: Option[String] = checkEmpty((n \ "version").text) + if (currentVersion.getOrElse(None) == "${project.version}") { + currentVersion = Some(projectVersion) + } + val dType: Option[String] = checkEmpty((n \ "type").text) + val scope: Option[String] = checkEmpty((n \ "scope").text) + + val optionalString: String = (n \ "optional").text + var optional: Option[Boolean] = None + if (optionalString == "true" || optionalString == "false") { + optional = Some(optionalString.toBoolean) + } + + dependencyList = Dependency(currentGroupId, + currentArtifactId, + currentVersion, + dType, + scope, + optional) :: dependencyList + } + } + // As we prepended everything, reverse the list to obtain the original order + return Some(dependencyList.reverse) + } + // If there are no dependencies, return None + None + } + + /** Returns a project as a raw string. + * + * @param endpoint the end_point to do the request. + * @return an optional String. + */ + def getProjectRaw(endpoint: String): Option[String] = { + val response = try { + val request = Http(url + endpoint).headers(getHeaders) + new HttpRequester().retrieveResponse(request) + } catch { + case _: Throwable => return None + } + + Some(response.body) + } + + /** Add a user-agent with contact details. */ + def getHeaders: List[(String, String)] = + ("User-Agent", "CodeFeedr-Maven/1.0 Contact: zonneveld.noordwijk@gmail.com") :: Nil + +} diff --git a/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSourceTest.scala b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSourceTest.scala new file mode 100644 index 00000000..c7e416d1 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/operators/MavenReleasesSourceTest.scala @@ -0,0 +1,30 @@ +package org.codefeedr.plugins.maven.operators + +import org.scalatest.FunSuite + +class MavenReleasesSourceTest extends FunSuite{ + + val mavenReleasesSource = new MavenReleasesSource(MavenSourceConfig(500, -1, 4)) + + test("parseRSSString bad"){ + val res = mavenReleasesSource.parseRSSString("wrong string") + + assert(res == Nil) + } + + test("waitPollingInterval"){ + val startTime = System.currentTimeMillis() + mavenReleasesSource.waitPollingInterval() + val endTime = System.currentTimeMillis() + + assert(endTime - startTime >= 500) + } + + test("defaultValuesTest"){ + val config = MavenSourceConfig() + + assert(config.maxNumberOfRuns == -1) + assert(config.pollingInterval == 60000) + } + +} diff --git a/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/protocol/ProtocolTest.scala b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/protocol/ProtocolTest.scala new file mode 100644 index 00000000..0ee8fc92 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/protocol/ProtocolTest.scala @@ -0,0 +1,160 @@ +package org.codefeedr.plugins.maven.protocol + +import java.util.Date + +import org.scalatest.FunSuite +import org.codefeedr.plugins.maven.protocol.Protocol._ + +class ProtocolTest extends FunSuite{ + + val parent: Option[Parent] = Some(Parent("groupPId", "artifactPId", "pVersion", Some("path"))) + val organization: Option[Organization] = Some(Organization("name", "url")) + val issueManagement: Option[IssueManagement] = Some(IssueManagement("system", "url")) + val scm: Option[SCM] = Some(SCM("connection", Some("devConnection"), Some("tag"), "url")) + val deps: Option[List[Dependency]] = Some(List(Dependency("groupId2", "artifactId2", Some("version"), Some("type"), Some("scope"), Some(true)))) + val depsEmptyVersion: Option[List[Dependency]] = Some(List(Dependency("groupId2", "artifactId2", None, None, None, None))) + val license: Option[List[License]] = Some(List(License("name", "url", "distribution", Some("comments")))) + val licenseEmpty: Option[List[License]] = Some(List(License("name", "url", "distribution", None))) + val repository: Option[List[Repository]] = Some(List(Repository("id", "name","url"))) + + val projectFull: MavenProject = MavenProject("modelVersion", "groupId", "artifactId", "version", parent, deps, license, repository, organization, Some("packaging"), issueManagement, scm) + val projectEmpty: MavenProject = MavenProject("modelVersion", "groupId", "artifactId", "version", None, None, None, None, None, None, None, None) + + test("mavenReleasePojoTest"){ + val release = MavenRelease("title", "link", "description", new Date(0), Guid("tag")) + val pojo = MavenReleasePojo.fromMavenRelease(release) + + assert(pojo.title == "title") + assert(pojo.link == "link") + assert(pojo.description == "description") + assert(pojo.pubDate == 0) + assert(pojo.guid_tag == "tag") + } + + test("mavenReleaseExtPojoTest"){ + val releaseExt = MavenReleaseExt("title", "link", "description", new Date(0), Guid("tag"), projectFull) + val pojo = MavenReleaseExtPojo.fromMavenReleaseExt(releaseExt) + + assert(pojo.title == "title") + assert(pojo.link == "link") + assert(pojo.description == "description") + assert(pojo.pubDate == 0) + assert(pojo.guid_tag == "tag") + assert(pojo.project != null) + } + + test("parentPojoTest"){ + val pojo = ParentPojo.fromParent(parent.get) + + assert(pojo.groupId == "groupPId") + assert(pojo.artifactId == "artifactPId") + assert(pojo.version == "pVersion") + assert(pojo.relativePath.contains("path")) + } + + test("dependencyPojoFullTest"){ + val pojo = DependencyPojo.fromDependency(deps.get.head) + + assert(pojo.groupId == "groupId2") + assert(pojo.artifactId == "artifactId2") + assert(pojo.version.contains("version")) + assert(pojo.`type`.contains("type")) + assert(pojo.scope.contains("scope")) + assert(pojo.optional) + } + + test("dependencyPojoEmptyVersionTest"){ + val pojo = DependencyPojo.fromDependency(depsEmptyVersion.get.head) + + assert(pojo.groupId == "groupId2") + assert(pojo.artifactId == "artifactId2") + assert(pojo.version.isEmpty) + assert(pojo.`type`.isEmpty) + assert(pojo.scope.isEmpty) + assert(!pojo.optional) + } + + test("licensePojoFullTest"){ + val pojo = LicensePojo.fromLicense(license.get.head) + + assert(pojo.name == "name") + assert(pojo.url == "url") + assert(pojo.distribution == "distribution") + assert(pojo.comments.contains("comments")) + } + + test("licensePojoEmptyTest"){ + val pojo = LicensePojo.fromLicense(licenseEmpty.get.head) + + assert(pojo.name == "name") + assert(pojo.url == "url") + assert(pojo.distribution == "distribution") + assert(pojo.comments.isEmpty) + } + + test("repositoryPojoTest"){ + val pojo = RepositoryPojo.fromRepository(repository.get.head) + + assert(pojo.id == "id") + assert(pojo.name == "name") + assert(pojo.url == "url") + } + + test("organizationPojoTest"){ + val pojo = OrganizationPojo.fromOrganization(organization.get) + + assert(pojo.name == "name") + assert(pojo.url == "url") + } + + test("issueManagementPojoTest"){ + val pojo = IssueManagementPojo.fromIssueManagement(issueManagement.get) + + assert(pojo.system == "system") + assert(pojo.url == "url") + } + + test("scmPojoTest"){ + val pojo = SCMPojo.fromSCM(scm.get) + + assert(pojo.connection == "connection") + assert(pojo.developerConnection.contains("devConnection")) + assert(pojo.tag.contains("tag")) + assert(pojo.url == "url") + } + + test("mavenProjectPojoFullTest"){ + val pojo = MavenProjectPojo.fromMavenProject(projectFull) + + assert(pojo.modelVersion == "modelVersion") + assert(pojo.groupId == "groupId") + assert(pojo.artifactId == "artifactId") + assert(pojo.version == "version") + assert(pojo.packaging.contains("packaging")) + assert(pojo.parent != null) + assert(pojo.dependencies != null) + assert(pojo.licenses != null) + assert(pojo.repositories != null) + assert(pojo.organization != null) + assert(pojo.issueManagement != null) + assert(pojo.scm != null) + } + + test("mavenProjectPojoEmptyTest"){ + val pojo = MavenProjectPojo.fromMavenProject(projectEmpty) + + assert(pojo.modelVersion == "modelVersion") + assert(pojo.groupId == "groupId") + assert(pojo.artifactId == "artifactId") + assert(pojo.version == "version") + assert(pojo.dependencies == null) + assert(pojo.licenses == null) + assert(pojo.repositories == null) + assert(pojo.parent == null) + assert(pojo.organization == null) + assert(pojo.packaging == null) + assert(pojo.issueManagement == null) + assert(pojo.scm == null) + } + +} diff --git a/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStageTest.scala b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStageTest.scala new file mode 100644 index 00000000..17b6831a --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStageTest.scala @@ -0,0 +1,63 @@ +package org.codefeedr.plugins.maven.stages + +import java.util +import java.util.Date + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.scalatest.FunSuite +import org.codefeedr.plugins.maven.protocol.Protocol.{Guid, MavenRelease, MavenReleaseExt} + +class MavenReleasesExtStageTest extends FunSuite { + + //TODO this test now pulls an existing project, mocking is better though since we now rely on external sources + //TODO the order of these tests now matters, find out why + + test("MavenReleasesExtStage integration empty project test"){ + val release = MavenRelease("org.neo4j.ERROR: it-test-support 3.5.13", "url", "desc", new Date(), Guid("tag")) + implicit val typeInfo: TypeInformation[MavenRelease] = TypeInformation.of(classOf[MavenRelease]) + + + new PipelineBuilder() + .appendSource(x => x.fromCollection(List(release))(typeInfo)) + .append(new MavenReleasesExtStage()) + .append { x: DataStream[MavenReleaseExt] => + x.addSink(new CollectReleases) + } + .build() + .startMock() + + assert(CollectReleases.result.size() == 0) + } + + test("MavenReleasesExtStage integration test"){ + val release = MavenRelease("org.neo4j.community: it-test-support 3.5.13", "url", "desc", new Date(), Guid("tag")) + implicit val typeInfo: TypeInformation[MavenRelease] = TypeInformation.of(classOf[MavenRelease]) + + + new PipelineBuilder() + .appendSource(x => x.fromCollection(List(release))(typeInfo)) + .append(new MavenReleasesExtStage()) + .append { x: DataStream[MavenReleaseExt] => + x.addSink(new CollectReleases) + } + .build() + .startMock() + + assert(CollectReleases.result.size() == 1) + } + +} + +object CollectReleases { + val result = new util.ArrayList[MavenReleaseExt]() +} + +class CollectReleases extends SinkFunction[MavenReleaseExt] { + override def invoke(value: MavenReleaseExt, + context: SinkFunction.Context[_]): Unit = { + CollectReleases.result.add(value) + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStageTest.scala b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStageTest.scala new file mode 100644 index 00000000..0918a9b5 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStageTest.scala @@ -0,0 +1,53 @@ +package org.codefeedr.plugins.maven.stages + +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.codefeedr.stages.OutputStage +import org.scalatest.FunSuite +import org.codefeedr.plugins.maven.operators.MavenSourceConfig +import org.codefeedr.plugins.maven.protocol.Protocol.MavenRelease + +class MavenReleasesStageTest extends FunSuite { + + test("MavenReleasesIntegrationTest") { + val source = new MavenReleasesStage(sourceConfig = MavenSourceConfig(1000, 12, 4)) + val sink = new LimitingSinkStage(4) + + val pipeline: Unit = new PipelineBuilder() + .append(source) + .append(sink) + .build() + .startMock() + } +} + +// Simple Sink Pipeline Object that limits the output to a certain number +// and is able to get a list of all the items that were received in the sink +class LimitingSinkStage(elements: Int = -1) + extends OutputStage[MavenRelease] + with Serializable { + var sink: LimitingSink = _ + + override def main(source: DataStream[MavenRelease]): Unit = { + sink = new LimitingSink(elements) + source.addSink(sink).setParallelism(1) + } +} + +class LimitingSink(elements: Int) extends SinkFunction[MavenRelease] { + var count = 0 + var items: List[MavenRelease] = List() + + override def invoke(value: MavenRelease, + context: SinkFunction.Context[_]): Unit = { + count += 1 + items = value :: items + + println(count) + + if (elements != -1 && count >= elements) { + throw new RuntimeException() + } + } +} diff --git a/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/util/MavenServiceTest.scala b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/util/MavenServiceTest.scala new file mode 100644 index 00000000..45988bb0 --- /dev/null +++ b/codefeedr-plugins/codefeedr-maven/src/test/scala/org/codefeedr/plugins/maven/util/MavenServiceTest.scala @@ -0,0 +1,239 @@ +package org.codefeedr.plugins.maven.util + +import org.scalatest.FunSuite + +import scala.xml.Node + +class MavenServiceTest extends FunSuite{ + + //Dit kan evt in een before, moet nog ff uitzoeken hoe + val dependencyNodesMinimal: Node = xml.XML.loadString("org.droolsdrools-core4.0.7") + val dependencyNodesFull: Node = xml.XML.loadString("org.testtest-core4.2.0jartesttrue") + val dependencyNodesMultiple: Node = xml.XML.loadString("org.eurekaclinicaljavautilorg.droolsdrools-core") + val dependencyNodesProjectVersion: Node = xml.XML.loadString("org.droolsdrools-core${project.version}") + + val licenseNodesNoComment: Node = xml.XML.loadString("The Apache Software License, Version 2.0http://www.apache.org/licenses/LICENSE-2.0.txtrepo") + val licenseNodesWithComment: Node = xml.XML.loadString("MIThttps://opensource.org/licenses/MITrepoExample comment") + + val repoNodesGood: Node = xml.XML.loadString("codehausSnapshotsCodehaus Snapshotshttp://snapshots.maven.codehaus.org/maven2") + val scmNode: Node = xml.XML.loadString("127.0.0.1/127.0.0.1/svn/HEADhttp://127.0.0.1") + val issueNode: Node = xml.XML.loadString("Bugzillahttp://127.0.0.1/bugzilla/") + val organizationNode: Node = xml.XML.loadString("Codehaus Mojohttp://mojo.codehaus.org") + val parentNode: Node = xml.XML.loadString("org.codehaus.mojomy-parent2.0../my-parent") + val emptyNode: Seq[Node] = dependencyNodesMinimal.child.filter(item => item.label == "non-existing") + + val projectNoDepNoLicense: Node = xml.XML.loadString("4.0.0exampleGroupIdexampleArtifactId6.9.420") + val projectNoDepNoLicenseWithParent: Node = xml.XML.loadString("4.0.0exampleGroupIdexampleArtifactId6.9.420") + val projectDepsAndLic: Node = xml.XML.loadString("4.0.0exampleGroupIdexampleArtifactId6.9.420" + + "org.droolsdrools-core4.0.7" + + "MIThttps://opensource.org/licenses/MITrepoExample comment" + + "") + + //----------------- parseDependencies -----------------// + test("parseDepsNoVersionTest"){ + //Parse node + val dependencyNodesNoVersion = xml.XML.loadString("io.cucumbercucumber-core") + val res = MavenService.parseDependencies(dependencyNodesNoVersion, "").get + + //Assert the result + assert(res.size == 1) + assert(res.head.groupId == "io.cucumber") + assert(res.head.artifactId == "cucumber-core") + assert(res.head.version.isEmpty) + } + + test("parseDepsWithVersionTest"){ + val res = MavenService.parseDependencies(dependencyNodesMinimal, "").get + + assert(res.size == 1) + assert(res.head.groupId == "org.drools") + assert(res.head.artifactId == "drools-core") + assert(res.head.version.get == "4.0.7") + } + + test("parseDepsOptionalFieldsEmpty"){ + val res = MavenService.parseDependencies(dependencyNodesMinimal, "").get + + assert(res.size == 1) + assert(res.head.`type`.isEmpty) + assert(res.head.scope.isEmpty) + assert(res.head.optional.isEmpty) + } + + test("parseDepsOptionalFieldsFull"){ + val res = MavenService.parseDependencies(dependencyNodesFull, "").get + + assert(res.size == 1) + assert(res.head.`type`.get == "jar") + assert(res.head.scope.get == "test") + assert(res.head.optional.get == true) + } + + test("parseDepsMultipleTest"){ + val res = MavenService.parseDependencies(dependencyNodesMultiple, "").get + + assert(res.size == 2) + assert(res.head.groupId == "org.eurekaclinical") + assert(res(1).groupId == "org.drools") + } + + + test("parseDepsNoChildTest"){ + val res = MavenService.parseDependencies(emptyNode, "") + + assert(res.isEmpty) + } + + test("parseDepProjectVersionTest"){ + val projectVersion = "1.2.3" + val res = MavenService.parseDependencies(dependencyNodesProjectVersion, projectVersion) + + assert(res.get.head.version.get == projectVersion) + } + + + //----------------- parseLicenses -----------------// + test("parseLicensesTest"){ + val res = MavenService.parseLicenses(licenseNodesNoComment).get + + assert(res.size == 1) + assert(res.head.url == "http://www.apache.org/licenses/LICENSE-2.0.txt") + assert(res.head.name == "The Apache Software License, Version 2.0") + assert(res.head.distribution == "repo") + assert(res.head.comments.isEmpty) + } + + test("parseLicenceswithCommentsTest"){ + val res = MavenService.parseLicenses(licenseNodesWithComment).get + + assert(res.size == 1) + assert(res.head.url == "https://opensource.org/licenses/MIT") + assert(res.head.name == "MIT") + assert(res.head.distribution == "repo") + assert(res.head.comments.get == "Example comment") + } + + test("parseLicenseEmpty"){ + val res = MavenService.parseLicenses(emptyNode) + + assert(res.isEmpty) + } + + //------------------ parseRepository ------------------// + test("parseRepositoryGoodTest"){ + val res = MavenService.parseRepositories(repoNodesGood).get + + assert(res.size == 1) + assert(res.head.id == "codehausSnapshots") + assert(res.head.name == "Codehaus Snapshots") + assert(res.head.url == "http://snapshots.maven.codehaus.org/maven2") + } + + test("parseRepositoryBadTest"){ + val res = MavenService.parseRepositories(emptyNode) + + assert(res.isEmpty) + } + + //-----------------------parseSCM-----------------------// + test("parseSCMTest"){ + val res = MavenService.parseSCM(scmNode).get + + assert(res.connection == "127.0.0.1/") + assert(res.developerConnection.get.contains("127.0.0.1/svn/")) + assert(res.tag.get == "HEAD") + assert(res.url == "http://127.0.0.1") + } + + test("parseSCMEmptyTest"){ + val res = MavenService.parseSCM(emptyNode) + + assert(res.isEmpty) + } + + //-------------------parseIssueManagement----------------// + test("parseIssueManagementTest"){ + val res = MavenService.parseIssueManagement(issueNode).get + + assert(res.system == "Bugzilla") + assert(res.url == "http://127.0.0.1/bugzilla/") + } + + test("parseIssueManagementEmptyTest"){ + val res = MavenService.parseIssueManagement(emptyNode) + + assert(res.isEmpty) + } + + //-----------------parseOrganization---------------------// + test("parseOrganizationTest"){ + val res = MavenService.parseOrganization(organizationNode).get + + assert(res.name == "Codehaus Mojo") + assert(res.url == "http://mojo.codehaus.org") + } + + test("parseOrganizationEmptyTest"){ + val res = MavenService.parseOrganization(emptyNode) + + assert(res.isEmpty) + } + + //----------------parseParent--------------------------// + test("parseParentTest"){ + val res = MavenService.parseParent(parentNode).get + + assert(res.groupId == "org.codehaus.mojo") + assert(res.artifactId == "my-parent") + assert(res.version == "2.0") + assert(res.relativePath.get == "../my-parent") + } + + test("parseParentEmptyTest"){ + val res = MavenService.parseParent(emptyNode) + + assert(res.isEmpty) + } + + //----------------- xmlToMavenProject -----------------// + test("xmlToMavenProjectNoDepNoLicNoParentTest"){ + val res = MavenService.xmlToMavenProject(projectNoDepNoLicense).get + + assert(res.groupId == "exampleGroupId") + assert(res.artifactId == "exampleArtifactId") + assert(res.modelVersion == "4.0.0") + assert(res.version == "6.9.420") + assert(res.parent.isEmpty) + } + + test("xmlToMavenProjectNoDepNoLicWithParentTest"){ + val res = MavenService.xmlToMavenProject(projectNoDepNoLicenseWithParent).get + + //assert the fields are inherited correctly + assert(res.modelVersion == "4.0.0") + assert(res.groupId == "exampleGroupId") + assert(res.artifactId == "exampleArtifactId") + assert(res.version == "6.9.420") + //assert the parent obj is correct + assert(res.parent.get.groupId == "exampleGroupId") + assert(res.parent.get.artifactId == "exampleArtifactId") + assert(res.parent.get.version == "6.9.420") + assert(res.parent.get.relativePath.isEmpty) + } + + test("xmlToMavenProjectDepsAndLicTest"){ + val res = MavenService.xmlToMavenProject(projectDepsAndLic).get + + assert(res.groupId == "exampleGroupId") + assert(res.dependencies.get.head.groupId == "org.drools") + assert(res.licenses.get.head.name == "MIT") + } + + //----------------- getProject -----------------// + test("getProjectNoProject"){ + val res= MavenService.getProject("thisDoesntExist") + + assert(res.isEmpty) + } + +} diff --git a/codefeedr-plugins/codefeedr-mongodb/src/test/scala/org/codefeedr/plugins/mongodb/stages/MongoInputOutputTest.scala b/codefeedr-plugins/codefeedr-mongodb/src/test/scala/org/codefeedr/plugins/mongodb/stages/MongoInputOutputTest.scala index 2269effc..c3a0f29d 100644 --- a/codefeedr-plugins/codefeedr-mongodb/src/test/scala/org/codefeedr/plugins/mongodb/stages/MongoInputOutputTest.scala +++ b/codefeedr-plugins/codefeedr-mongodb/src/test/scala/org/codefeedr/plugins/mongodb/stages/MongoInputOutputTest.scala @@ -30,9 +30,9 @@ import org.codefeedr.pipeline.PipelineBuilder import org.codefeedr.plugins.mongodb.MongoQuery import org.codefeedr.stages.utilities.{SeqInput, StringInput, StringType} import org.mongodb.scala.MongoClient -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterAll, FunSuite} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.Await import scala.concurrent.duration.Duration @@ -96,7 +96,7 @@ Etiam nisl sem, egestas sit amet pretium quis, tristique ut diam. Ut dapibus sod val list = StringCollectSink.asList val items = longString.split("[ \n]").toList - assert(items.containsAll(StringCollectSink.result)) + assert(items.asJava.containsAll(StringCollectSink.result)) assert(list.size == items.size) assert(StringCollectSink.numEventTimes == 0) } @@ -193,7 +193,7 @@ Etiam nisl sem, egestas sit amet pretium quis, tristique ut diam. Ut dapibus sod pipeline.startMock() assert(TestEventCollectSink.result.size == 2) - assert(TestEventCollectSink.result.containsAll(Seq("klaas", "nagellak"))) + assert(TestEventCollectSink.result.containsAll(Seq("klaas", "nagellak").asJava)) } } @@ -209,7 +209,7 @@ object StringCollectSink { numEventTimes = 0 } - def asList: List[String] = result.toList + def asList: List[String] = result.asScala.toList } class StringCollectSink extends SinkFunction[StringType] { @@ -234,7 +234,7 @@ object TestEventCollectSink { numEventTimes = 0 } - def asList: List[String] = result.toList + def asList: List[String] = result.asScala.toList } class TestEventCollectSink extends SinkFunction[TestEvent] { diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSource.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSource.scala new file mode 100644 index 00000000..78a32d8b --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSource.scala @@ -0,0 +1,143 @@ +package org.codefeedr.plugins.npm.operators + +import java.util.{Calendar, Date} + +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.codefeedr.pipeline.{PluginReleasesSource, PluginSourceConfig} +import org.codefeedr.plugins.npm.protocol.Protocol.NpmRelease +import org.codefeedr.stages.utilities.HttpRequester +import scalaj.http.Http + +/** + * Configuration parameters for connecting to the NPM Packages data source + * + * @param pollingInterval the milliseconds between consecutive polls + * @param maxNumberOfRuns the maximum number of polls executed + * + * @author Roald van der Heijden + * Date: 2019 - 12 - 03 + */ +case class NpmSourceConfig(pollingInterval: Int = 10000, + maxNumberOfRuns: Int = -1, + timeout: Int = 32) + extends PluginSourceConfig + +/** + * Class to represent a source in CodeFeedr to query NPM package releases + * + * This files gets information on NPM packages from "https://npm-update-stream.libraries.io/" + * Then queries "http://registry.npmjs.com/%packagenameWithPossibleScope% + * to acquire the information for each specific package + * + * @param config the configuration paramaters object for this specific data source + * + * @author Roald van der Heijden + * Date: 2019 - 12 - 03 + */ +class NpmReleasesSource(config: NpmSourceConfig = NpmSourceConfig()) + extends PluginReleasesSource[NpmRelease](config) { + + /** + * URL to get update stream from + */ + val url_updatestream = "https://npm-update-stream.libraries.io/" + + //var timeout = 2 // timeout set specifically to speed up testing + + /** + * The latest poll + */ + private var lastPoll: Option[Seq[NpmRelease]] = None + + /** + * Runs the source. + * @param ctx the source the context. + */ + override def run(ctx: SourceFunction.SourceContext[NpmRelease]): Unit = { + val lock = ctx.getCheckpointLock + + /* While is running or #runs left. */ + while (isRunning && runsLeft != 0) { + lock.synchronized { // Synchronize to the checkpoint lock. + try { + // Polls the update stream + val resultString = retrieveUpdateStringFrom(url_updatestream) + val now = Calendar.getInstance().getTime + // convert string with updated packages into list of updated packages + val items: Seq[NpmRelease] = + createListOfUpdatedNpmIdsFrom(resultString.get, now) + // Sort and drop the duplicate releases + val newItems: Seq[NpmRelease] = sortAndDropDuplicates(items, now) + // Decrease runs left + super.decreaseRunsLeft() + // Timestamp each item + newItems.foreach(x => + ctx.collectWithTimestamp(x, x.retrieveDate.getTime)) + // Update lastPoll + if (items.nonEmpty) { + lastPoll = Some(items) + } + // call parent run + super.runPlugin(ctx, newItems) + } catch { + case _: Throwable => + } + } + } + } + + /** + * Drops items that already have been collected and sorts them based on names + * + * @param items Potential items to be collected + * @return Valid sorted items + */ + def sortAndDropDuplicates(items: Seq[NpmRelease], + now: Date): Seq[NpmRelease] = { + // Collect right items and update last item + if (lastPoll.isDefined) { + items + .map(x => x.name) + .diff(lastPoll.get.map(x => x.name)) + .map(x => NpmRelease(x, now)) + } else { + items + } + } + + /** + * Requests the update stream and returns its body as a string. + * Will keep trying with increasing intervals if it doesn't succeed + * + * @return Body of requested update stream + */ + def retrieveUpdateStringFrom(urlEndpoint: String): Option[String] = { + val response = try { + val request = Http(urlEndpoint) + return Some( + new HttpRequester(config.timeout).retrieveResponse(request).body) + } catch { + case _: Throwable => None + } + response + } + + /** + * Turns a given string (following request response from Librarios.io's update stream) into npm ids + * @param input the string containing the names of the releases + * @return a sequence of NPMRelease case classes + */ + def createListOfUpdatedNpmIdsFrom(input: String, + time: Date): List[NpmRelease] = { + val packageList = input + .replace("\"", "") + .replace("[", "") + .replace("]", "") + .split(",") + .toList + packageList match { + case List("") => Nil + case _ => packageList.map(arg => NpmRelease(arg, time)) + } + } +} diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/RetrieveProjectAsync.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/RetrieveProjectAsync.scala new file mode 100644 index 00000000..9d3a2fce --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/operators/RetrieveProjectAsync.scala @@ -0,0 +1,73 @@ +package org.codefeedr.plugins.npm.operators + +import org.apache.flink.streaming.api.functions.async.{ + ResultFuture, + RichAsyncFunction +} +import org.codefeedr.plugins.npm.protocol.Protocol.{NpmRelease, NpmReleaseExt} +import org.codefeedr.plugins.npm.util.NpmService +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} +import collection.JavaConverters._ + +/** + * Retrieves a project related to a release asynchronously. + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +class RetrieveProjectAsync + extends RichAsyncFunction[NpmRelease, NpmReleaseExt] { + + /** + * Retrieve the execution context lazily. + */ + implicit lazy val executor: ExecutionContext = ExecutionContext.global + + /** + * Async retrieves the project belonging to the release. + * + * @param input the release. + * @param resultFuture the future to add the project to. + */ + override def asyncInvoke(input: NpmRelease, + resultFuture: ResultFuture[NpmReleaseExt]): Unit = { + + /** + * transform the title of a project to be retrieved by the NpmService + */ + val link = input.name + + /** + * Retrieve the project in a Future. + */ + val requestProject: Future[Option[NpmReleaseExt]] = Future( + NpmService.getProject(link)) + + /** + * Collects the result. + */ + requestProject.onComplete { + case Success(result: Option[NpmReleaseExt]) => + if (result.isDefined) { //If we get None, we return nothing. + resultFuture.complete( + List(NpmReleaseExt(input.name, + input.retrieveDate, + result.get.project)).asJava) + } else { + resultFuture.complete(List().asJava) + } + case Failure(e) => + resultFuture.complete(List().asJava) + e.printStackTrace() + } + + } + + /** + * If we retrieve a time-out, then we just complete the future with an empty list. + */ + override def timeout(input: NpmRelease, + resultFuture: ResultFuture[NpmReleaseExt]): Unit = + resultFuture.complete(List().asJava) +} diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala new file mode 100644 index 00000000..2b91e100 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala @@ -0,0 +1,260 @@ +package org.codefeedr.plugins.npm.protocol + +import java.util.Date + +/** + * Contains all the case classes and POJO equivalent classes to represent a NPM package release + * + * @author Roald van der Heijden + * Date: 2019-12-19 (YYYY-MM-DD) + */ +object Protocol { + + case class NpmRelease(name: String, retrieveDate: Date) // using ingestion time + + case class NpmReleaseExt(name: String, + retrieveDate: Date, + project: NpmProject) + + case class NpmProject(_id: String, + _rev: Option[String], + name: String, + author: Option[PersonObject], + contributors: Option[List[PersonObject]], + description: Option[String], + homepage: Option[String], + keywords: Option[List[String]], + license: Option[String], + dependencies: Option[List[Dependency]], + maintainers: List[PersonObject], + readme: String, + readmeFilename: String, + bugs: Option[Bug], + bugString: Option[String], + repository: Option[Repository], + time: TimeObject) + + case class Dependency(packageName: String, version: String) + + case class PersonObject(name: String, + email: Option[String], + url: Option[String]) + + case class Repository(`type`: String, url: String, directory: Option[String]) + + case class Bug(url: Option[String], email: Option[String]) + + case class TimeObject(created: String, modified: Option[String]) + + // underneath is a POJO representation of all case classes mentioned above + + class NpmReleasePojo extends Serializable { + var name: String = _ + var retrieveDate: Long = _ + } + + object NpmReleasePojo { + def fromNpmRelease(release: NpmRelease): NpmReleasePojo = { + val pojo = new NpmReleasePojo() + pojo.name = release.name + pojo.retrieveDate = release.retrieveDate.getTime + pojo + } + } + + class NpmReleaseExtPojo extends Serializable { + var name: String = _ + var retrieveDate: Long = _ + var project: NpmProjectPojo = _ + } + + object NpmReleaseExtPojo { + def fromNpmReleaseExt(release: NpmReleaseExt): NpmReleaseExtPojo = { + val pojo = new NpmReleaseExtPojo + pojo.name = release.name + pojo.retrieveDate = release.retrieveDate.getTime + pojo.project = NpmProjectPojo.fromNpmProject(release.project) + pojo + } + } + + class NpmProjectPojo extends Serializable { + var _id: String = _ + var _rev: String = _ + var name: String = _ + var author: PersonObjectPojo = _ + var contributors: List[PersonObjectPojo] = _ + var description: String = _ + var homepage: String = _ + var keywords: List[NpmKeyWordPojo] = _ + var license: String = _ + var dependencies: List[DependencyPojo] = _ + var maintainers: List[PersonObjectPojo] = _ + var readme: String = _ + var readmeFilename: String = _ + var bugs: BugPojo = _ + var bugString: String = _ + var repository: RepositoryPojo = _ + var time: TimePojo = _ + } + + object NpmProjectPojo { + def fromNpmProject(project: NpmProject): NpmProjectPojo = { + val pojo = new NpmProjectPojo + + pojo._id = project._id + pojo._rev = project._rev.orNull + pojo.name = project.name + if (project.author.isDefined) { + pojo.author = PersonObjectPojo.fromPersonObject(project.author.get) + } + if (project.contributors.isDefined) { + pojo.contributors = project.contributors.get + .map(person => PersonObjectPojo.fromPersonObject(person)) + } + pojo.description = project.description.orNull + pojo.homepage = project.homepage.orNull + if (project.keywords.isDefined) { + pojo.keywords = project.keywords.get + .map(keyword => NpmKeyWordPojo.fromKeywordAsString(keyword)) + } + pojo.license = project.license.orNull + if (project.dependencies.isDefined) { + pojo.dependencies = + project.dependencies.get.map(x => DependencyPojo.fromDependency(x)) + } + pojo.maintainers = project.maintainers.map( + person => PersonObjectPojo.fromPersonObject(person) + ) + pojo.readme = project.readme + pojo.readmeFilename = project.readmeFilename + if (project.bugs.isDefined) { + pojo.bugs = BugPojo.fromBug(project.bugs.get) + } + pojo.bugString = project.bugString.orNull + if (project.repository.isDefined) { + pojo.repository = RepositoryPojo.fromRepository(project.repository.get) + } + pojo.time = TimePojo.fromTime(project.time) + + pojo + } + } + class NpmKeyWordPojo extends Serializable { + var keyword: String = _ + } + + // added for the ability to register KeywordPojo as a streaming SQL table + class NpmKeyWordPojoExt extends NpmKeyWordPojo { + var id: String = _ + } + + object NpmKeyWordPojo { + def fromKeywordAsString(keyword: String): NpmKeyWordPojo = { + val pojo = new NpmKeyWordPojo() + pojo.keyword = keyword + pojo + } + } + + class DependencyPojo extends Serializable { + var packageName: String = _ + var version: String = _ + } + + // added for the ability to register DependencyPojo as a streaming SQL table + class DependencyPojoExt extends DependencyPojo { + var id: String = _ + } + + object DependencyPojo { + def fromDependency(dep: Dependency): DependencyPojo = { + val pojo = new DependencyPojo() + pojo.packageName = dep.packageName + pojo.version = dep.version + pojo + } + } + + class PersonObjectPojo extends Serializable { + var name: String = _ + var email: String = _ + var url: String = _ + } + + // added for the ability to register PersonObjectPojo as a streaming SQL table + class PersonObjectPojoExt extends PersonObjectPojo { + var id: String = _ + } + + object PersonObjectPojo { + def fromPersonObject(person: PersonObject): PersonObjectPojo = { + val pojo = new PersonObjectPojo() + pojo.name = person.name + pojo.email = person.email.orNull + pojo.url = person.url.orNull + pojo + } + } + + class RepositoryPojo extends Serializable { + var `type`: String = _ + var url: String = _ + var directory: String = _ + } + + // added for the ability to register RepositoryPojo as a streaming SQL table + class RepositoryPojoExt extends RepositoryPojo { + var id: String = _ + } + + object RepositoryPojo { + def fromRepository(r: Repository): RepositoryPojo = { + val pojo = new RepositoryPojo() + pojo.`type` = r.`type` + pojo.url = r.url + pojo.directory = r.directory.orNull + pojo + } + } + + class BugPojo extends Serializable { + var url: String = _ + var email: String = _ + } + + // added for the ability to register BugPojo as a streaming SQL table + class BugPojoExt extends BugPojo { + var id: String = _ + } + + object BugPojo { + def fromBug(b: Bug): BugPojo = { + val pojo = new BugPojo() + pojo.url = b.url.orNull + pojo.email = b.email.orNull + pojo + } + } + + class TimePojo extends Serializable { + var created: String = _ + var modified: String = _ + } + + // added for the ability to register TimePojo as a streaming SQL table + class TimePojoExt extends TimePojo { + var id: String = _ + } + + object TimePojo { + def fromTime(obj: TimeObject): TimePojo = { + val pojo = new TimePojo() + pojo.created = obj.created + pojo.modified = obj.modified.orNull + pojo + } + } + + override def toString: String = "Protocol companion object" +} diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala new file mode 100644 index 00000000..a78aa651 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala @@ -0,0 +1,60 @@ +package org.codefeedr.plugins.npm.stages + +import java.util.concurrent.TimeUnit + +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.streaming.api.datastream.{ + AsyncDataStream => JavaAsyncDataStream +} +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.plugins.npm.operators.RetrieveProjectAsync +import org.codefeedr.plugins.npm.protocol.Protocol.{NpmRelease, NpmReleaseExt} +import org.codefeedr.stages.TransformStage +import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor + +import scala.language.higherKinds + +/** Transform a [[NpmRelease]] to [[NpmReleaseExt]]. + * + * @param stageId the name of this stage. + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +class NpmReleasesExtStage(stageId: String = "npm_releases") + extends TransformStage[NpmRelease, NpmReleaseExt](Some(stageId)) { + + /** + * Transform a [[NpmRelease]] to [[NpmReleaseExt]]. + * + * @param source The input source with type [[NpmRelease]]. + * @return The transformed stream with type [[NpmReleaseExt]]. + */ + override def transform( + source: DataStream[NpmRelease]): DataStream[NpmReleaseExt] = { + + // Retrieve project from release asynchronously. + val async = JavaAsyncDataStream.orderedWait(source.javaStream, + new RetrieveProjectAsync, + 5, + TimeUnit.SECONDS, + 100) + /* + for testing purposes this is commmented out, but until we merge into master at the end of the project + please leave these lines commented!! + */ + + // implicit val typeInfo = TypeInformation.of(classOf[NpmReleaseExt]) +// new org.apache.flink.streaming.api.scala.DataStream(async) +// .map(x => NpmReleaseExt(x.name, x.retrieveDate, x.project)) +// .print() + + new org.apache.flink.streaming.api.scala.DataStream(async) + } + + override def getSchema: Schema = { + implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[NpmReleaseExt] + } +} diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala new file mode 100644 index 00000000..bf4044dd --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala @@ -0,0 +1,42 @@ +package org.codefeedr.plugins.npm.stages + +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.Context +import org.codefeedr.stages.InputStage +import org.codefeedr.plugins.npm.operators.{NpmReleasesSource, NpmSourceConfig} +import org.codefeedr.plugins.npm.protocol.Protocol.NpmRelease +import org.codefeedr.stages.utilities.DefaultTypeMapper._ + +import scala.language.higherKinds + +/** + * Fetches real-time releases from Npm. + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +class NpmReleasesStage(stageId: String = "npm_releases_min", + sourceConfig: NpmSourceConfig = NpmSourceConfig()) + extends InputStage[NpmRelease](Some(stageId)) { + + /** + * Fetches [[NpmRelease]] from real-time Npm feed. + * + * @param context The context to add the source to. + * @return The stream of type [[NpmRelease]]. + */ + override def main(context: Context): DataStream[NpmRelease] = { + implicit val typeInfo: TypeInformation[NpmRelease] = + TypeInformation.of(classOf[NpmRelease]) + context.env + .addSource(new NpmReleasesSource(sourceConfig))(typeInfo) + } + + override def getSchema: Schema = { + implicit val NpmSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[NpmRelease] + } +} diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/util/NpmService.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/util/NpmService.scala new file mode 100644 index 00000000..db2e46f8 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/util/NpmService.scala @@ -0,0 +1,244 @@ +package org.codefeedr.plugins.npm.util + +import java.util.Date + +import org.apache.logging.log4j.scala.Logging +import org.codefeedr.stages.utilities.HttpRequester +import org.json4s.JsonAST.{JNothing, JObject, JString} +import org.json4s.ext.JavaTimeSerializers +import org.json4s.jackson.JsonMethods.parse +import org.json4s.jackson.Serialization.read +import org.json4s.{DefaultFormats, Formats, JValue} +import org.codefeedr.plugins.npm.protocol.Protocol +import org.codefeedr.plugins.npm.protocol.Protocol.{ + Dependency, + NpmProject, + NpmReleaseExt, + PersonObject, + TimeObject +} +import scalaj.http.Http + +/** + * Services to retrieve a project from the NPM API registry. + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +object NpmService extends Logging with Serializable { + + var timeout = 8 + + /** + * the API url to retrieve projects from. + */ + val url_packageInfo = "http://registry.npmjs.com/" + + /** + * Retrieves a Npm project, resulting in the Protocol case class with information filled in where possible. + * + * @param projectName the name of the project. + * @return an optional NpmProject. + */ + def getProject(projectName: String): Option[NpmReleaseExt] = { + for { + jsonString <- createJsonStringFor(url_packageInfo, projectName) + npmProject <- convertProjectFrom(jsonString) + } yield buildNpmReleaseExtUsing(projectName, jsonString, npmProject) + } + + /** + * Creates a JSON String for this project + * + * @param projectName the project for which to get the information and create the JSON String + * @return the JSON in Option[String] (so None, if something went wrong and also None when UNPUBLISHED!!) + */ + def createJsonStringFor( + updateStreamBaseURL: String, + projectName: String + ): Option[String] = { + val jsonString: Option[String] = + getProjectRaw(updateStreamBaseURL, projectName, timeout) + if (jsonString.isEmpty || jsonString.get == """{"error":"Not found"}""") { + logger.error(s"Couldn't retrieve npm project with name $projectName.") + return None + } + // check if UNPUBLISHED, if so => err msg/None, else just the jsonString + val json = parse(jsonString.get) + (json \ "time") \ "unpublished" match { + case JNothing => jsonString + case _ => + logger.error( + s"skipping UNPUBLISHED state of project with name $projectName." + ) + None + } + } + + /** + * Creates a NpmProject from given JSON String while taking care of some error handling as well + * @param json the JSON String to parse + * @return None if something went wrong, or Some[NpmProject] + */ + def convertProjectFrom(json: String): Option[NpmProject] = { + implicit val formats: Formats = + new DefaultFormats {} ++ JavaTimeSerializers.all + val myProject = + try { + Some(read[Protocol.NpmProject](json)) + } catch { + case _: Throwable => None + } + myProject + } + + /** + * Builds an extended Release from given name, JSON String and NpmProject. + * @param projectName the name of the project we want to build an extended release for + * @param jsonString the JSON string belonging to this projectName + * @param project the NPMproject extracted from given JSON String + * @return an NpmReleaseExt with all required details filled in + */ + def buildNpmReleaseExtUsing( + projectName: String, + jsonString: String, + project: NpmProject + ): NpmReleaseExt = { + val json = parse(jsonString) + // STEP 1: Now set the time right (find the created / modified field and update the time) + val myTime = extractTimeFrom(json) + // STEP 2 : Now lookup the dependencies + val myDependencies = extractDependenciesFrom(json) + // STEP 3: Then lookup the author + val myAuthor = extractAuthorFrom(json) + // STEP 4: Update the Case Class with the results of time & dependencies + NpmReleaseExt( + projectName, + new Date(), + project.copy( + time = myTime, + author = myAuthor, + dependencies = Some(myDependencies) + ) + ) + } + + /** + * Parses the time for a given project + * @param json the JValue to parse the time from + * @return a case class TimeObject with relevant details filled in + */ + def extractTimeFrom(json: JValue): TimeObject = { + // find the first creation time + val createdField = (json \ "time") \ "created" match { + case JString(s: String) => s + case _ => "unknown" + } + // find the latest modification time + val modifiedField = (json \ "time") \ "modified" match { + case JString(s: String) => Some(s) + case _ => None + } + TimeObject(createdField, modifiedField) + } + + /** + * Finds the latest version string of a NPM Project + * @param json the JSON object to look for the latest version in + * @return a String denoting the latest version on success or "-1" on failure + */ + def findLatestVersionNr(json: JValue): String = { + (json \ "dist-tags") \ "latest" match { + case JString(x) => x + case _ => "-1" + } + } + + /** + * Parses the dependencies for the latest version of a given project, if the field "latest" exists within the JSON + * @param json the JValue from which we glean the List[DependencyObject] + * @return the list with Dependencies or Nil if something went wrong + */ + def extractDependenciesFrom(json: JValue): List[Dependency] = { + // first look up the latest version number + val latestVersionNr = findLatestVersionNr(json) + // then get me that version object and look up the dependencies field + val dependenciesList = + ((json \ "versions") \ latestVersionNr) \ "dependencies" match { + case JObject(lijstje) => lijstje + case _ => Nil + } + // EXHAUSTIVE match using Option, then flatMap to get the correct type back! + dependenciesList.flatMap(tupleElem => + tupleElem match { + case (name, JString(version)) => Some(Dependency(name, version)) + case _ => None + }) + } + + /** + * Tries to parse an author for a given project. Will look in the root children for a complex author object, + * or in the field of the latest version if the author cannot be parsed from there. + * @param json the JValue from which we glean complex PersonObject + * @return Some[PersonObject] on success, None on failure + */ + def extractAuthorFrom(json: JValue): Option[PersonObject] = { + val authorField1 = json \ "author" + val result1 = convertAuthorFrom(authorField1) + if (result1.isDefined) result1 + else { + val latestVersionNr = findLatestVersionNr(json) + val authorField2 = ((json \ "versions") \ latestVersionNr) \ "author" + convertAuthorFrom(authorField2) + } + } + + /** + * converts given JSON into Option[PersonObject] + * @param jsonAuthorField the JSON to convert + * @return Some(...) on success, None on failure + */ + def convertAuthorFrom(jsonAuthorField: JValue): Option[PersonObject] = { + jsonAuthorField match { + case JObject(_) => + implicit val formats: Formats = + new DefaultFormats {} ++ JavaTimeSerializers.all + jsonAuthorField.extractOpt[PersonObject] + case _ => None + } + } + + /** + * Returns a project as a raw string. + * + * @param endpoint the end_point to do the request. + * + * @return an optional JSON String. + */ + def getProjectRaw( + base_url: String, + endpoint: String, + maxTimeOutInsec: Int + ): Option[String] = { + val response = + try { + val request = Http(base_url + endpoint).headers(withConfiguredHeaders) + new HttpRequester(maxTimeOutInsec).retrieveResponse(request) + } catch { + case _: Throwable => return None + } + Some(response.body) + } + + /** + * Add a user-agent with contact details. + */ + def withConfiguredHeaders: List[(String, String)] = { + ( + "User-Agent", + "CodeFeedr-Npm/1.0 Contact: zonneveld.noordwijk@gmail.com" + ) :: Nil + } + + override def toString = "NpmService Companion Object" +} diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/bslet.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/bslet.json new file mode 100644 index 00000000..ec71380c --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/bslet.json @@ -0,0 +1,288 @@ +{ + "_id": "bs-let", + "_rev": "3-bbdfd0170944489f4dc72db21d4b9f3c", + "name": "bs-let", + "dist-tags": { + "latest": "0.1.11" + }, + "versions": { + "0.1.8": { + "name": "bs-let", + "version": "0.1.8", + "description": "A PPX for async/await (and general monadic binding) in ReasonML.", + "license": "MIT", + "esy": { + "build": "refmterr dune build -p let_anything_ppx" + }, + "publishConfig": { + "access": "public" + }, + "scripts": { + "build": "esy" + }, + "dependencies": {}, + "resolutions": { + "@esy-ocaml/esy-installer": "0.0.1", + "@opam/menhir": "20171013", + "@opam/re": "1.8.0" + }, + "devDependencies": { + "@opam/dune": "*", + "@opam/ocaml-migrate-parsetree": "^1.0.11", + "@esy-ocaml/reason": "*", + "refmterr": "*", + "@esy-ocaml/merlin": "*", + "esy": "^0.5.8", + "ocaml": "~4.6.1000" + }, + "gitHead": "405b34184b8858742d53a836eba465aa2057d1aa", + "_id": "bs-let@0.1.8", + "_nodeVersion": "11.15.0", + "_npmVersion": "6.7.0", + "dist": { + "integrity": "sha512-mRZnkdNsjg49os9R9ljnkJOVSNX801f+Hmjjb2GY3JtG8NgO82PPXT5EFF765AYW3zxpZlMJ6Ub4/EL7R6TpUA==", + "shasum": "1c9e89546aa6b78ec2e4eb00c6cb9364f06412cf", + "tarball": "https://registry.npmjs.org/bs-let/-/bs-let-0.1.8.tgz", + "fileCount": 49, + "unpackedSize": 20796010, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJd1YNKCRA9TVsSAnZWagAADF0P/R70D0TyS+480y9AVtLZ\nmVGcIAoVRcl9tW6aUBa/GBg2utOsRo8PWzvxkpm086WZvYDYySSyZxs7Im5F\ndaPZbh2E5olbe6JcZP/lmE3KtJ+Tw1TtpQZ2iNEng8BF2KhmMK6F+UrnCdJH\n0wjfgCPgbpcBkJ/b3yN16VzS2VEXv3cym7u7Yi+p3Tfv0bJ+/AKieczFj0eU\ncDpDwMqviHK7jQyGUHtbCw74HHFiW6riWdaehKWo95+DvMOKqSv7ChFimOta\n86RLN58yiPc22Xuvoj907XO4NHQR4m45AmCuJk3bCJbaHp0YD9TkMD9Ip5YN\nyrHSs93C/b7UoDHrkv6PohNuyTMNmlRw6QfbUvstKKD/VP2bydEvADHrQ3Ca\nDTznaxrUeG+0NVA8XFBIo5Mj1USF15zw4TPyfS1a8e4f79nL02rN6bJfnQhT\nVgx5IqYsJxee7eCg12BLNhLxPSSVobdRbXCmWvkewqdGGgn4+Zw/C+QeWAEk\nl2YSX6Dp+ZAHGkmWEgGCKCtRCUAuKl/i93rCS0RnqawMwfYYSeu1+hPCg++q\nkIS9EbKV3Y4XsEYtl5MFe/K07rKhZpLgNep+tVHVMvltZcdr2XiQYUTyQlOY\nxJBkr2qTdaK+a7axYZFe+5KNm36aFSX7nYYzGwan2VfhewTNOOH2pQo1OuDJ\nVqS3\r\n=PpD6\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + } + ], + "_npmUser": { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/bs-let_0.1.8_1574273865936_0.5986737878860178" + }, + "_hasShrinkwrap": false + }, + "0.1.9": { + "name": "bs-let", + "version": "0.1.9", + "description": "A PPX for async/await (and general monadic binding) in ReasonML.", + "repository": { + "url": "git+https://github.com/reasonml-labs/bs-let.git" + }, + "license": "MIT", + "esy": { + "build": "refmterr dune build -p let_anything_ppx" + }, + "publishConfig": { + "access": "public" + }, + "scripts": { + "build": "esy" + }, + "dependencies": {}, + "resolutions": { + "@esy-ocaml/esy-installer": "0.0.1", + "@opam/menhir": "20171013", + "@opam/re": "1.8.0" + }, + "devDependencies": { + "@opam/dune": "*", + "@opam/ocaml-migrate-parsetree": "^1.0.11", + "@esy-ocaml/reason": "*", + "refmterr": "*", + "@esy-ocaml/merlin": "*", + "esy": "^0.5.8", + "ocaml": "~4.6.1000" + }, + "gitHead": "5058093bcb24c92d8ef4def8b657c6c650f67d4f", + "bugs": { + "url": "https://github.com/reasonml-labs/bs-let/issues" + }, + "homepage": "https://github.com/reasonml-labs/bs-let#readme", + "_id": "bs-let@0.1.9", + "_nodeVersion": "11.15.0", + "_npmVersion": "6.7.0", + "dist": { + "integrity": "sha512-bS7afS9OXtv1RiKkRo/rVL/jGR1R0izUUWhWGdoZ0pr7OlHfr/MS2vSADFmOVpHS9CD2weuzV/alJnxbyU7UVw==", + "shasum": "bf2418bfd2a1629bdb4c72d6a66ed5d6df7dc5d3", + "tarball": "https://registry.npmjs.org/bs-let/-/bs-let-0.1.9.tgz", + "fileCount": 49, + "unpackedSize": 20796137, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJd1YhtCRA9TVsSAnZWagAASqgP/A0SfRvrqQvoJZhL2nPw\nK3LjbqBwMktxGeRnurs+YCIc7MSFZWDWPu6xMuwehS6AUQUxhl/onW3VCEfm\n2NizPjKs7KlnDc2QE9Bd9jt/Ar75XeaMBMXCPXBiYW+rYkTbYm5EMNA4MnE0\nt7vq5WP/exOQs92ED7czoQB4EPZkD8bh2Xugmd1H+hQUHXdkrEzCDcA7bl/l\nsItdOw2uESmuQIrKN4h+ryjWuXExFLvubuglrHBGzqC0OkSy6A1jelVfmZsc\nJPrkkBjQdovvfwqw7hkCVcNLaHPWc4sHaHmy5x3hR4CyfqRDX7z11CRPnPAa\n6gv9ifUGCHSci/brkY+OtS41mjCsa/8WOG+yoBQcKvEMgkT8uzkcy6Ni/gxN\nBa9qDFlkocp1yzhwUlAIJTuqs9q/3aozea45TA7SSTBP3FITd0UQI07jQikD\nKh0AO2X3WsVVTUjrAqi/tZ0xy2T6IphuMkgurc5v/Qi//BasRCcDxjuj6QdN\nAa178grCGzQ2zlT9B4AdTz7LipiFz6zBtuzNpIxmPjuxEXTjn15lcSPUfuRj\ncAemlbVmGHA9HgCYlDwy06eIM9OAv/TArw4nxQaJLGN/ZqXv+BYg+qop4CBQ\nmi8+sm3aWvHyFqPH1uf7JYj6Wji3DLScTXDD8yQqr6NYY29db6Cl/pBxrTvq\nd3Bv\r\n=R53s\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + } + ], + "_npmUser": { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/bs-let_0.1.9_1574275180805_0.19962669483563134" + }, + "_hasShrinkwrap": false + }, + "0.1.10": { + "name": "bs-let", + "version": "0.1.10", + "description": "A PPX for async/await (and general monadic binding) in ReasonML.", + "repository": { + "url": "git+https://github.com/reasonml-labs/bs-let.git" + }, + "license": "MIT", + "esy": { + "build": "refmterr dune build -p let_anything_ppx" + }, + "publishConfig": { + "access": "public" + }, + "scripts": { + "build": "esy" + }, + "dependencies": {}, + "resolutions": { + "@esy-ocaml/esy-installer": "0.0.1", + "@opam/menhir": "20171013", + "@opam/re": "1.8.0" + }, + "devDependencies": { + "@opam/dune": "*", + "@opam/ocaml-migrate-parsetree": "^1.0.11", + "@esy-ocaml/reason": "*", + "refmterr": "*", + "@esy-ocaml/merlin": "*", + "esy": "^0.5.8", + "ocaml": "~4.6.1000" + }, + "gitHead": "d7225c554afd9816ca1e99a86849cccd295acb81", + "bugs": { + "url": "https://github.com/reasonml-labs/bs-let/issues" + }, + "homepage": "https://github.com/reasonml-labs/bs-let#readme", + "_id": "bs-let@0.1.10", + "_nodeVersion": "11.15.0", + "_npmVersion": "6.7.0", + "dist": { + "integrity": "sha512-4YOXsX4qgXlToH7ciht8ne9rJpXU90PJo5ozPv7HtQSxFx7XaQ9U6IT7O4RW/TyuUiDyi0AszvwHo/1v1ZKXFg==", + "shasum": "acfcd9640743cda8201b628201fffad296320914", + "tarball": "https://registry.npmjs.org/bs-let/-/bs-let-0.1.10.tgz", + "fileCount": 49, + "unpackedSize": 20798484, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJd3XEPCRA9TVsSAnZWagAA5PIP/iVAIHq8gkY2ji3FvhGB\niNO1LIK22Kx2d0v34TSrJW1lOKujdOG6XEpYJ/0dGllCPwy5UhVykV/TnJXw\nu4Q/YqlUjAAc0BRnyV9Bn4tPWjtoLnYtuTKLJ4vdPHZFVoYCoLdf8juYTtKW\nkDN/DoPBlJ1tvQxcVo2qBeMkvOF3b5+CbzKVjbPMn3BJ0Tk/RfXllEfiZz+Z\novy/33KtA/nrqIY1M9xCUZsx3/Ki5Msrrrt6IPFHGdm70CXAxU4i4XxdURk2\nw5zK7GL+/yiv42i6sd8P0TnIDTQEhaS+NGaRn8/R9Nd4TcnY3+etDBI83l+z\nNYxfr6+OOk/IQcoMeKWrhWbB+dZ0YZByjerKTYCJPI9EQTkXfCGEAjy+yErm\nEYvsWq5WNYk0Z2Nie9JPSinAlh3Hzm3caIix2aCwloqrrDza14pCJB+oDrjq\njByShB6kTr2uo5OXn9bN5kWI3D656NtcZuxx/wt816AsRwZLLCHt6gBV2ify\nS8PSpfHPNXkC5IMLlXyCgmjpDCDNacUZyrrCX+urjoFcBY4AXe30AYWwdJl8\n2Ba2ROU7Vwg1EZKlojsiY7rNlPz0vXErGGiGjsG6ppnyXu1rhCPBpGREO94a\nQDKU90j/LpsB17x2SZm0+jJ1Nq3HfVRIaKlcJwGUAnCAQTCtfiPf/yKtgKG6\nit6m\r\n=n2Vr\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + } + ], + "_npmUser": { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/bs-let_0.1.10_1574793486830_0.451992588756281" + }, + "_hasShrinkwrap": false + }, + "0.1.11": { + "name": "bs-let", + "version": "0.1.11", + "description": "A PPX for async/await (and general monadic binding) in ReasonML.", + "repository": { + "url": "git+https://github.com/reasonml-labs/bs-let.git" + }, + "license": "MIT", + "esy": { + "build": "refmterr dune build -p let_anything_ppx" + }, + "publishConfig": { + "access": "public" + }, + "scripts": { + "build": "esy" + }, + "dependencies": {}, + "resolutions": { + "@esy-ocaml/esy-installer": "0.0.1", + "@opam/menhir": "20171013", + "@opam/re": "1.8.0" + }, + "devDependencies": { + "@opam/dune": "*", + "@opam/ocaml-migrate-parsetree": "^1.0.11", + "@esy-ocaml/reason": "*", + "refmterr": "*", + "@esy-ocaml/merlin": "*", + "esy": "^0.5.8", + "ocaml": "~4.6.1000" + }, + "gitHead": "dad1e6a828bbf51a65abd90a380ba974e0fc9742", + "bugs": { + "url": "https://github.com/reasonml-labs/bs-let/issues" + }, + "homepage": "https://github.com/reasonml-labs/bs-let#readme", + "_id": "bs-let@0.1.11", + "_nodeVersion": "11.15.0", + "_npmVersion": "6.7.0", + "dist": { + "integrity": "sha512-Zp5wMGEmdTyabiLhNvt1rUQQ5zL2T5z6uj4v6WEIU+K+tseljqrpEz9Dlp5YbF1uePVsXeatmkkhb5xKtvEOPw==", + "shasum": "159d0194c890be29d47d54db6f015c881e3269db", + "tarball": "https://registry.npmjs.org/bs-let/-/bs-let-0.1.11.tgz", + "fileCount": 49, + "unpackedSize": 20798484, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJd5ZoqCRA9TVsSAnZWagAAxC8P/RnyB9T/1LgX7fA1fQFD\nvwfb9N0iC4ELRdk3CHjG1ArXj4DG0DzXbjtP87zCGn6n8bJPKaxUTrrZ/Y2i\nTcFMmBozr+YnLK95gphX+8l3Ko5kbVCAhU8VPWA7uVURdD6zkb+mGWa5HHaR\nVyMffQuV2leUY7JHqAX2t8/xlHsbxf2LVM7DCtZYjIOEBB2RVkxRDJn6iVUG\nI/udN3xoQqAOOu3OU2IgQppykOknTDNv/QnmewVwtfu7PIXd69hPpZ5nF6u0\nrs89MwDuw9uO6LE8tBtfDzKs5o94+QqwjdIYPj1gAxaS6MLFEBq5+x4AJ7do\n7oVN4aS55hyu/fGDOHj8zSB7Yp5HgJYGAQ7xSqplfb2s+xQiB4tAdfYSYhVZ\nIBFHrqKoioqIA0CigqgCKTZGz/ydbUf2z2VLI2wLx8cROVqZDkAbqhdKIE00\n3ShJMqQRNlDX+oC+0nbpTd1n8lp0dEbj1ESyea+1oZvto9p6Yf5DnHFrhhbi\nE12I4vNie33tppCnRC9DBhp2MaBAFejuFN2wi0w+6B6ZfCF+alCBzeOog4Vg\n0/DEnMQLSPvvUD/fthAq+yuY1gfkGODRC2bh7OCNV5Yze1EhZ1R8vOV7bZmt\nwUh6TsuhVy3ukpqBaqUGu8ww6oNnspmxeE0qSglrxUZfd2HPe1FHoC8uJt3a\n3m4z\r\n=bqAI\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + } + ], + "_npmUser": { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/bs-let_0.1.11_1575328297080_0.22267136964113976" + }, + "_hasShrinkwrap": false + } + }, + "time": { + "created": "2019-11-20T18:17:45.698Z", + "0.1.8": "2019-11-20T18:17:46.237Z", + "modified": "2019-12-02T23:11:40.152Z", + "0.1.9": "2019-11-20T18:39:41.202Z", + "0.1.10": "2019-11-26T18:38:07.141Z", + "0.1.11": "2019-12-02T23:11:37.563Z" + }, + "maintainers": [ + { + "name": "mrmurphy", + "email": "m@mrmurphy.dev" + } + ], + "description": "A PPX for async/await (and general monadic binding) in ReasonML.", + "license": "MIT", + "readme": "# bs-let\n\n![A woman knitting some code](https://github.com/reasonml-labs/bs-let/raw/master/artwork/eastwood-page-under-construction.png)\n\nThis is a PPX (language extension) designed to make _monadic operations_ (think async functions with \"await\" statements in Javascript if you don't know what a monadic operation is) easy to write and read in ReasonML.\n\n## Warning: Experimental Project\n\nThis is package is an experimental community project (a.k.a \"labs\"). This means that community members use it and contribute to it, but it is not officially supported or recommended by the ReasonML community. Using this project in a production environment means being willing to contribute yourself if needs be.\n\nAlso, the expected lifetime of this PPX is relatively short. OCaml 4.08 has introduced native syntax for doing the same thing that this PPX does (`let+`). At the time of this writing, Bucklescript is still a ways out from supporting OCaml 4.08, or backporting support for `let+`. This PPX has two purposes:\n\n- Provide a temporary solution until Bucklescript and Reason both support `let+`.\n- Gague community interest in monadic syntax sugar in general and gather feedback on its usage.\n\n## Compatibility\n\nThis package only works with bs-platform 6.x and above. If you're stuck on 5.x take a look at [Jared's original repo](https://github.com/jaredly/let-anything).\n\n## Installation\n\n- `npm install --save-dev bs-let`\n- Open up your `bsconfig.json` and add `bs-let/ppx` to your `ppx-flags`. It should look something like this:\n ```json\n {\n ...\n \"bsc-flags\": [...],\n \"ppx-flags\": [\"bs-let/ppx\", ...],\n \"refmt\": ...\n }\n ```\n\n## Usage\n\nSimple and sweet, this is a language extension that flattens callbacks. All you need is a module which defines a function called `let_` which takes something to map over, and a callback to do the mapping. For example:\n\n```reasonml\nmodule Option = {\n let let_ = Belt.Option.flatMap;\n}\n```\n\nThen, when you're working with something you want to map, add a `%` onto your `let`, and the rest of the lines in the block will be turned into a callback and passed to the mapping function _at compile time_.\n\nFor example:\n\n```reasonml\n// Assume the `Option` module from above is defined already.\n\ntype address = {\n street: option(string)\n};\n\ntype personalInfo = {\n address: option(address)\n};\n\ntype user = {\n info: option(personalInfo)\n};\n\n// Get the user's street name from a bunch of nested options. If anything is\n// None, return None.\nlet getStreet = (maybeUser: option(user)): option(string) => {\n let%Option user = maybeUser;\n // Notice that info isn't an option anymore once we use let%Pom!\n let%Option info = user.info;\n let%Option address = info.address;\n let%Option street = info.street;\n Some(street->Js.String.toUpperCase)\n};\n```\n\nThat code is flat, readable, and understandable. Here's an alternative without the syntax sugar:\n\n```reasonml\nlet getStreet = (maybeUser: option(user)): option(string) => {\n maybeUser->Belt.Option.flatMap(user =>\n user.info\n ->Belt.Option.flatMap(personalInfo =>\n personalInfo.address\n ->Belt.Option.flatMap(address =>\n address.street\n ->Belt.Option.flatMap(street =>\n Some(street->Js.String.toUpperCase)\n )\n )\n )\n );\n};\n```\n\nMuch nicer to have the sugar, no? This PPX really shines, though, when we use it to chain async operations, since that has to be done quite a lot in Javascript, especially server-side, and it typically happens multiple times in the middle of large and complex functions.\n\nHere's a more complex example of an async control flow using the [Repromise](https://aantron.github.io/repromise/docs/QuickStart) library to work with Javascript promises:\n\n```reasonml\n\n// Repromise doesn't ship with native support for this PPX, so we simply add our\n// own by re-defining the module, including all the stuff from the original\n// module, and adding our own function.\nmodule Repromise = {\n include Repromise;\n let let_ = Repromise.andThen;\n\n // This is totally optional. It can be nice sometimes to return a\n // non-promise value at the end of a function and have it automatically\n // wrapped.\n module Wrap = {\n let let_ = Repromise.map;\n }\n}\n\nlet logUserIn = (email: string, password: string) => {\n // Assume this is a function that returns a promise of a hash.\n let%Repromise hash = UserService.hashPassword(password)\n let%Repromise maybeUser = UserService.findUserForEmailAndHash(email, hash);\n let result = switch (maybeUser) {\n | Some(user) =>\n // It even works inside of a switch expression!\n // Here you can see we're using \".Wrap\" to automatically wrap our result\n // in a promise.\n let%Repromise.Wrap apiToken = TokenService.generateForUser(user.id);\n Ok( user.firstName, apiToken )\n | None =>\n // We resolve a promise here to match the branch above.\n Error(\"Sorry, no user found for that email & password combination\")->Repromise.resolved\n };\n\n // Since let_ is defined as \"andThen\" we've got to remember to return a promise\n // at the end of the function! Remember, all the lines after each let% just get\n // turned into a callback!\n Repromise.resolved(result)\n};\n```\n\nThere's a whole lot that can be done with this PPX. It's even possible to go a little crazy and start writing modules that combine monads, like `AsyncOption` that will specifically handle optional values inside of promises. But, in practice, those modules are seldom needed. Don't go too crazy, keeping it simple will get you a long, long way.\n\nThings to remember:\n\n- You don't have to name your module anything special. It could be named `Foo` and you can `let%Foo blah = ...`.\n- Simple is better than complex.\n- Obvious is usually better than hidden.\n\n## About Performance\n\nIt's worth noting that this PPX simply produces a _function callback structure_. Why is this important? There are potential performance gains in situations where avoiding a callback structure is possible.\n\nFor example, this handrwitten code, which is pretty much what the PPX produces:\n\n```reasonml\nlet getStreet = (maybeUser: option(user)): option(string) => {\n maybeUser->Belt.Option.flatMap(user =>\n user.info\n ->Belt.Option.flatMap(personalInfo =>\n personalInfo.address\n ->Belt.Option.flatMap(address =>\n address.street\n ->Belt.Option.flatMap(street =>\n Some(street->Js.String.toUpperCase)\n )\n )\n )\n );\n};\n```\n\nIs _functionally_ equivalent, but inferior in terms of performance, to the following hand-written code:\n\n```reasonml\nlet getStreetExplicit = (maybeUser: option(user)): option(string) => {\n switch (maybeUser) {\n | None => None\n | Some(user) =>\n switch (user.info) {\n | None => None\n | Some(personalInfo) =>\n switch (personalInfo.address) {\n | None => None\n | Some(address) =>\n switch (address.street) {\n | None => None\n | Some(street) => Some(street->Js.String.toUpperCase)\n }\n }\n }\n };\n};\n```\n\nBecause we're working with Options, we can `switch` on the values instead of `flatMap`-ing. The generated Javascript of the second approach looks like this:\n\n```javascript\nfunction getStreetExplicit(maybeUser) {\n if (maybeUser !== undefined) {\n var match = maybeUser[/* info */ 0];\n if (match !== undefined) {\n var match$1 = match[/* address */ 0];\n if (match$1 !== undefined) {\n var match$2 = match$1[/* street */ 0];\n if (match$2 !== undefined) {\n return match$2.toUpperCase();\n } else {\n return;\n }\n } else {\n return;\n }\n } else {\n return;\n }\n }\n}\n```\n\nOnly one total function invocation is produced by the compiler in this case instead of one invocation _for every bind_. This is significantly faster to execute and may be worth choosing if this function will be very highly trafficked.\n\nIn summary, this PPX is not designed to produce the most performant code in every case. It's just designed to make callbacks easier to use.\n\n## Notes\n\n**A Note about Native VS Bucklescript**\nThis is specifically designed to be helpful with writing Javascript code through ReasonML and Bucklescript. Native OCaml 4.08 implemented a native monadic sugar syntax. So if you're writing native code, I'd suggest skipping this PPX and waiting until [this PR](https://github.com/facebook/reason/pull/2487) lands in Reason, and then adopting the new syntax.\n\n**A Note about Windows**\nCurrently this project only precomiles binaries for linux and OS X according to the needs of existing maintainers. If you're a Windows user and would like to use this PPX, We'd love a pull-request that moves the project from Travis to Azure Pipelines and builds for all three platforms.\n\n## Credit\n\nThis PPX was created by @jaredly and upgraded to the latest OCaml by @anmonteiro. Murphy Randle has merged Antonio's changes to upgrade the package for Bucklescript 6.x and 7.x, re-packaged it to build with `esy` and precompiled binaries for osx and linux. Murphy has also written this readme to describe the most common use-case for this PPX. More features are available but undocumented in this readme. You can see them here: https://github.com/jaredly/let-anything.\n\n- Lovely readme artwork https://icons8.com/ouch/illustration/eastwood-page-under-construction\n", + "readmeFilename": "README.md", + "homepage": "https://github.com/reasonml-labs/bs-let#readme", + "repository": { + "url": "git+https://github.com/reasonml-labs/bs-let.git" + }, + "bugs": { + "url": "https://github.com/reasonml-labs/bs-let/issues" + } +} diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/firetheNun.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/firetheNun.json new file mode 100644 index 00000000..0bf1afd9 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/firetheNun.json @@ -0,0 +1,112 @@ +{ + "_id": "tiny", + "_rev": "15-5aede384a190b4164eaf840629010349", + "name": "tiny", + "description": "An in-process key-value store", + "dist-tags": { + "latest": "0.0.10" + }, + "versions": { + "0.0.10": { + "name": "tiny", + "description": "An in-process key-value store", + "author": { + "name": "Christopher Jeffrey" + }, + "version": "0.0.10", + "main": "./lib/tiny.js", + "bin": { + "tiny": "./bin/tiny" + }, + "man": [ + "./man/tiny.1" + ], + "preferGlobal": true, + "repository": { + "type": "git", + "url": "git://github.com/chjj/tiny.git" + }, + "homepage": "https://github.com/chjj/tiny", + "bugs": { + "url": "http://github.com/chjj/tiny/issues" + }, + "license": "MIT", + "keywords": [ + "database", + "nosql", + "in-process" + ], + "tags": [ + "database", + "nosql", + "in-process" + ], + "scripts": { + "test": "node test", + "bench": "node test --bench" + }, + "dependencies": { + "lru-cache": 1 + }, + "engines": { + "node": ">= 0.8.0" + }, + "_id": "tiny@0.0.10", + "_shasum": "92d540b507c67034b8c7a53797aeb30b6c1780a3", + "_from": ".", + "_npmVersion": "1.4.9", + "_npmUser": { + "name": "chjj", + "email": "chjjeffrey@gmail.com" + }, + "maintainers": [ + { + "name": "chjj", + "email": "chjjeffrey@gmail.com" + } + ], + "dist": { + "shasum": "92d540b507c67034b8c7a53797aeb30b6c1780a3", + "tarball": "https://registry.npmjs.org/tiny/-/tiny-0.0.10.tgz" + }, + "directories": {} + } + }, + "maintainers": [ + { + "name": "chjj", + "email": "chjjeffrey@gmail.com" + } + ], + "time": { + "modified": "2014-06-01T00:32:24.082Z", + "created": "2011-07-20T06:42:43.303Z", + "0.0.4": "2011-12-08T18:15:35.613Z", + "0.0.5": "2011-12-08T18:15:35.613Z", + "0.0.6": "2011-12-08T18:15:35.613Z", + "0.0.7": "2011-12-08T18:15:35.613Z", + "0.0.8": "2011-12-08T18:15:35.613Z", + "0.0.9": "2014-05-19T01:11:09.573Z", + "0.0.10": "2014-06-01T00:32:24.082Z" + }, + "author": { + "name": "Christopher Jeffrey" + }, + "repository": { + "type": "git", + "url": "git://github.com/chjj/tiny.git" + }, + "readme": "# tiny\n\n__tiny__ is an in-process document/object store for node.js.\n\nIt is largely inspired by [nStore](https://github.com/creationix/nstore),\nhowever, its goal was to implement real querying which goes easy on the memory.\n\nTiny is very simple, there are no schemas, just store your objects. It supports\nmongo-style querying, or alternatively a \"mapreduce-like\" interface similar to\nCouchDB's views.\n\n## Install\n\n``` bash\n$ npm install tiny\n```\n\n## How Tiny works...\n\nTiny takes advantage of the fact that, normally, when you query for records in\na database, you're only comparing small properties (<128b) in the query itself.\nFor example, when you query for articles on a weblog, you'll usually only be\ncomparing the timestamp of the article, the title, the author, the category,\nthe tags, etc. - pretty much everything except the content of the article\nitself.\n\nTiny stores each document/object's property individually in the DB file and\ncaches all the small properties into memory when the DB loads, leaving anything\nabove 128b behind. When a query is performed, Tiny only lets you compare the\nproperties stored in memory, which is what you were going to do anyway. Once\nthe query is complete, Tiny will perform lookups on the FD to grab the large\nproperties and put them in their respective objects before results are returned\nto you.\n\nThis my attempt at combining what I think the best aspects of nStore and\nnode-dirty are. node-dirty is incredibly fast and simple (everything is\nin-memory), and nStore is very memory effecient, (but this only lasts until you\nperform a query). node-tiny allows for queries that perform lookups on the db\nfile, and it selectively caches properties as well, so it's fast and easier on\nmemory.\n\n## Example Querying\n\n``` js\nvar Tiny = require('./tiny');\nTiny('articles.tiny', function(err, db) {\n var time = Date.now()\n , low = time - (60*60*1000)\n , high = time - (30*60*1000);\n\n // mongo-style query\n db.find({$or: [\n { timestamp: { $lte: low } },\n { timestamp: { $gte: high } }\n ]})\n .desc('timestamp')\n .limit(3)(function(err, results) {\n console.log('Results:', results);\n });\n\n // is equivalent to...\n db.fetch({\n desc: 'timestamp',\n limit: 3\n }, function(doc, key) {\n if (doc.timestamp <= low\n || doc.timestamp >= high) {\n console.log('Found:', key);\n return true;\n }\n }, function(err, results) {\n console.log('Results:', results);\n });\n});\n```\n\nThe mongo-style querying should be fairly self-explanatory. The second query is\nsupposed to be similar to a mapreduce interface, but it's the rough equivalent\nof a `.filter` function.\n\nNote: there is a `shallow` parameter for `.fetch`, `.find`, and `.get`, wherein\nit will __only__ lookup properties that are under 128b in size. This is to go\neasy on the memory. `.each` and `.all` are shallow by default, but they do have\na `deep` parameter, (which I don't recommend using).\n\n## Other Usage\n\n``` js\n// save a document\ndb.set('myDocument', {\n title: 'a document',\n content: 'hello world'\n}, function(err) {\n console.log('set!');\n});\n\n// .each will iterate through\n// every object in the database\n// it is shallow by default\ndb.each(function(doc) {\n console.log(doc.title);\n});\n\n// returns every object in the DB\n// in an array, this is shallow\n// by default\ndb.all(function(err, docs) {\n console.log(docs.length);\n});\n\n// remove a doc\ndb.remove('myDocument', function(err) {\n console.log('deleted');\n});\n\n// retrieve an object from the database\ndb.get('someOtherThing', function(err, data) {\n // data._key is a property which\n // holds the key of every object\n console.log('found:', data._key);\n});\n\n// updates the object\n// without overwriting its other properties\ndb.update('article_1', {\n title: 'new title'\n}, function(err) {\n console.log('done');\n});\n\n// close the file descriptor\ndb.close(function(err) {\n console.log('db closed');\n});\n\n// clean up the mess\ndb.compact(function(err) {\n console.log('done');\n});\n\n// dump the entire database to a JSON file\n// in the same directory as the DB file\n// (with an optional pretty-print parameter)\ndb.dump(true, function(err) {\n console.log('dump complete');\n});\n```\n\n## Making data more memory efficient\n\nBecause of the way Tiny works, there are ways to alter your data to make it\nmore memory efficient. For example, if you have several properties on your\nobjects that aren't necessary to for queries, its best to nest them in an\nobject.\n\n``` js\nuser: {\n name: 'joe',\n prop1: 'data',\n prop2: 'data',\n prop3: 'data'\n}\n\nuser: {\n name: 'joe',\n data: {\n prop1: 'data',\n prop2: 'data',\n prop3: 'data'\n }\n}\n```\n\nThat way, the data will not be cached if it exceeds 128b collectively.\nEventually there may be an `ignore` method or an `index` method, which will be\nexplicitly inclusive or exclusive to which properties are cached and which\nproperties are able to be referenced within a query.\n\n## Documentation\n\n### Database\n- [Construction](#construction)\n- [dump](#dump)\n- [close](#close)\n- [kill](#kill)\n\n### Querying\n- [set](#set)\n- [each](#each)\n- [all](#all)\n\n## Database\n\n\n\n### Tiny(name, callback)\nCreates and returns a database with the given name.\n\n__Arguments__\n\n- name - filename to store and load the Tiny database\n- callback(err, db) - Called after the database file is opened and loaded\n\n__Example__\n\n``` js\nvar db;\nTiny('./articles.tiny', function(err, db_) {\n if (err) throw err;\n db = db_;\n ...\n});\n```\n\n---------------------------------------\n\n\n\n### dump(pretty, func) or dump(func)\n\nDumps the a database to a JSON file with the name as name.json. Pretty\nspecifies whether to indent each line with two spaces or not. Alternatively,\ndump(func) can be called.\n\n__Arguments__\n\n- pretty - if true, the JSON file will be indented with two spaces\n- func(err) - called after the dump is complete.\n\n__Example__\n\n``` js\ndb.dump(true, function(err) {\n console.log('dump complete');\n});\n```\n\n---------------------------------------\n\n\n\n### close(func)\n\nCloses the Tiny database file handle. A new Tiny object must be made to reopen\nthe file.\n\n__Arguments__\n\n- func() - callback function after the database has been closed\n\n__Example__\n\n``` js\ndb.close(function(err) {\n console.log('db closed');\n});\n```\n\n---------------------------------------\n\n\n\n### kill(func)\n\nCloses the Tiny database file, deletes the file and all the data in the\ndatabase, and then creates a new database with the same name and file.\n\n__Arguments__\n\n- func() - callback function after the database has been reloaded\n\n__Example__\n\n``` js\ndb.kill(function(err) {\n console.log('db has been destroyed and a new db has been loaded');\n});\n```\n\n## Querying\n\n\n\n### set(docKey, doc, func)\n\nSaves a object `doc` to database under the key `docKey`. Ideally, docKey should\nbe 128b or smaller.\n\n__Arguments__\n\n- docKey - a key to search the database for\n- doc - an object to save to the database under the given key\n- func - callback function after the doc object has been saved to the database\n\n__Example__\n\n``` js\ndb.set('myDocument', {\n title: 'a document',\n content: 'hello world'\n}, function(err) {\n console.log('set!');\n});\n```\n\n---------------------------------------\n\n\n\n### each(func, deep) or each(func)\n\nIterates through every object in the database.\n\n__Arguments__\n\n- func(doc) - Callback function that is called with every iterated object `doc`\n from the database\n- done() - Callback to be executed after the iterations complete.\n- deep - `true` if every object should be returned, `false` or unset if only\n cacheable objects should be returned (ones smaller than 128b)\n\n__Example__\n\n``` js\ndb.each(function(doc) {\n console.log(doc.title);\n}, function() {\n console.log('done');\n});\n```\n\n### Contribution and License Agreement\n\nIf you contribute code to this project, you are implicitly allowing your code\nto be distributed under the MIT license. You are also implicitly verifying that\nall code is your original work. ``\n\n## License\n\nCopyright (c) 2011-2014, Christopher Jeffrey. (MIT License)\n\nSee LICENSE for more info.\n", + "homepage": "https://github.com/chjj/tiny", + "keywords": [ + "database", + "nosql", + "in-process" + ], + "bugs": { + "url": "http://github.com/chjj/tiny/issues" + }, + "license": "MIT", + "readmeFilename": "README.md", + "_attachments": {} +} diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/fitfont-haskeywords.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/fitfont-haskeywords.json new file mode 100644 index 00000000..7212a2df --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/fitfont-haskeywords.json @@ -0,0 +1,790 @@ +{ + "_id": "fitfont", + "_rev": "9-8893e3db2492f122c3bf2293ec25d2e0", + "name": "fitfont", + "dist-tags": { + "latest": "1.3.1" + }, + "versions": { + "0.0.1": { + "scripts": { + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js", + "fitfont-install": "./bin/install.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "0.0.1", + "main": "index.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "gitHead": "6ae164d40a8cbc227bb0d7908adb618afecad06b", + "_id": "fitfont@0.0.1", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-E+8dIVsx7MZV3ytehJsYgBb2BvZDk4m81XTRKu+5EYFCnzd9S+jHAEUrFT4CIiOcdkj2Tlu3EJ9qZbPeWhJ1Zw==", + "shasum": "96da0851e97fff396313436150b656ea37f16fce", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-0.0.1.tgz", + "fileCount": 10, + "unpackedSize": 31602, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJblR8ACRA9TVsSAnZWagAAOvcP/RtE9knxVoXoBjqSOkA2\nz/giX9gjOAgZufwocJGl1dFyS/sgKblEceB8TKVvV7JxvWPDuSloA1caBwJj\n427zglwVeYDGvW7htWm5w4ZSPq56ce4YUpPmkY2xNbzNv6XSJI/s71QQVz3J\nMUufIlE8lCZ5GVFfjbbNbeaO0nNB3gHJh7G2bVcWDwK9L59OT08uRy4rp1lE\nizAiXlAPEW52i9LKQvSyzqh+XBbsEi8HPKKbnsnBDBL29nmW4rbBl7DRNHmK\n2PbBjAlgcMwkCcTMHDG9LpUky6uv0pRNv2qBdcoHyDVs8U3E4jwlUoRvY6SQ\nL6GivQmQBMC8KGpLijVHIMQ3WARDBDlA22tJ1nzS3dY6igrXr9Wug9tcRiRn\nZ17L+rvIXbn/fGvYP2YDE3rJ2GXMLf7Uv+9bfdJvtp7bMw/kAht0Phmib9cU\nibEY+xX+Bzh70jUDpwT8gE0QKIECzQ73Aygsj2C7AawjeYG8EYJf0SFzPLIY\nsZxGUp+JkgUHNw2raCPYpWUNpZStzwb0T7mkQHaYReBgCImmzckHnfJOwTWl\nVKP7NJJK9jtcCqL58u1FhujdAF/c+mYxBnGx6LJ7nZvDPwKsKmiJZJGr5L1v\np1SlZJgtAR98RPfbJ6yqw1WA8ALZNRQq2Bo4ctNmKjXAEicXmMfk5e5JUP9K\nZjVV\r\n=xNgX\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_0.0.1_1536499455691_0.6102768079553382" + }, + "_hasShrinkwrap": false + }, + "0.0.2": { + "scripts": { + "postinstall": "./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js", + "fitfont-install": "./bin/install.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "0.0.2", + "main": "index.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4" + }, + "gitHead": "54c2f8dd2383235b6ecc39fe7416f3289261fb3f", + "_id": "fitfont@0.0.2", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-puBxABg+Suc1PXiXxGBI2bzcVpfi+GBTRtHjFkuNEXYngw4wr2c8Rz4SOqQLaSfBvpZEZRGv/voWkpvSMM8Y+A==", + "shasum": "cb3c63fd689c7dd5e41756c151ec17ac973d2bdc", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-0.0.2.tgz", + "fileCount": 10, + "unpackedSize": 32299, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJblYN3CRA9TVsSAnZWagAA4KoP/3U7gORpnDbmTuyXvsPM\nz6feox0orrlU0nkbViwalP9gwC7Ax3L7mvWe3N2hv952Gyo+G6vIvJmJ/eeW\niH3DNA6eIfgxQ6n17Cw6QzTPaykE7q++CRTj3BuKd7jpkdSW1T8CFfTNrL5U\nOcyxm6Q2kc6mx60xWVbaQ798txGGH+94D00mmNCfih1IltR9wI4/HJeXf7vU\nEHehFOiSpjmDo0q4zCUlfiYoy2hcTGr4JzZdDbDSyMV6QUMyt9CE0R+oQW9J\nmBSKUvJiex8yIsh6Scq3enwFwKz8uYpfpONhSyzSHu0dM5m5TPp+AAhAuSno\n8iyg9nW9RYFu6XgZGvxmjkVBBzXDWYsqWCP6IYtALiruTMbioIddpFT877RN\nGIi1IKBeNWzIwFwX9EzL+kuQp31dRcRrQ4KU0Y/S7uFd9iVCTK6YZqAFNxoS\n2ucn40HSkcxYE85lCdAyEwFcGH7MLcUim6Sw3IPjbjvdyJTiiyohqFa2oBCT\nSlN1eqgnzzp6RzdsJ4m7BHtDwpU2tuUU30zxM52c8QOaHunpxljeBJAnt4MI\ncZzoZn+KnijOi2rv/u1ax/FUuIENiNuL7tVH5HlavVkcljjJzTPpRLWLK1UR\nP2gK19IVCEjDbT37E//AhMaVuKTcySTfjFxUsOJqTDVC8+MSCrCZqcZd6Xta\nWQam\r\n=Trgp\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_0.0.2_1536525174509_0.11024666130343586" + }, + "_hasShrinkwrap": false + }, + "1.2.0": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.0", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "3499d4b4ec4296cb8fd5f7425e3f5adf519c4588", + "_id": "fitfont@1.2.0", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-uP9Jv+4HNiYUAD6JWBySqGLcAQk9SJjbWQ7DMUmZdNSp45alTUHFLckQ2Vk7KpaSmmJ0RReUtyKQ67qmDjWEBg==", + "shasum": "ed4d1fbf36b612debff9ed2819b243afabdf136e", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.0.tgz", + "fileCount": 9, + "unpackedSize": 30861, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJblaoUCRA9TVsSAnZWagAAUh8P/iPR8/Za/rqh4m3mlsMw\n6nC7dagOFp+fR7l2Buczg9XWcf1A9mhFNDaZvuYrDhTU+JmSOaPHbVKrDNzS\nXam6zAS4o+sTzdaRBYJSu32/ksq7aDw2HG15RFVOHafLQtU3Vd4AvLj5Eoqa\nBOUYEOYf6dwJzbzb5Gxlwp6XCAIQxma2cFjXxHV/OIjv9la2DXEz7hTajx/n\ncZik0qsPwY0Mr7AMPcMKlahy2wNw1lRLdwHu2eu8dXUtWH35RTYL4IztMFq/\n29t0OaaVzmW1jxtf6TcXxThLcICEPrMqRVWsYWPehXa5DRZw3lkXfWQ8u4kf\nZ29+NHBUtFHG3chyJIc0Y+x6M5fbHzwSJBXmL1CBWHcCsCn2sPEf9d/6OOiN\nwDdhlsiRuT4+3odyh5cdCjauPsbTa0nBWh4g2rQd1yr65ALZwPhTm0G/dpUU\n5pKcVYI5XSww2R2vFnIjnSX12Hi9Su8fVnUbIyQ1GAmYLX/gPwa6DwuhQqE0\n4csiR0RF8PxRxYxsJbPzMUo6Lv2wUum6qDvqtJFFS73CvOhl4oMEoONZmEpL\n+PA2YGn3Q2lUZmMGrgkYDtwvUkQOBgfIcd4WPmTDOsjeQV8TbJLGV0oC+Qz3\nBrhWfPbJAKjNvfIYe5/ODVEQA/Bs1itJad9XRo5yzAmWaW1Iqc12ez+GI3Cc\ni9ot\r\n=p3OO\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.0_1536535060164_0.5761924881591736" + }, + "_hasShrinkwrap": false + }, + "1.2.1": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.1", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "a06f6220a0efc876e80773ffd64ca1e8f0d90088", + "_id": "fitfont@1.2.1", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-XHfSiCRHkGRZUiBPJSKeRpWiOd4+6aSX2KJzpe8BL7/dHWxNhPam3Mh/Gdn1inkR2H2KiZScN965D3aLCElxWQ==", + "shasum": "d01db9565f970d1a6a71d1caa8d4dc971d21d32e", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.1.tgz", + "fileCount": 9, + "unpackedSize": 30870, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJblas8CRA9TVsSAnZWagAAW3IP+wUHLn5vydiy276ibzQP\n8JTFwzY9XJTUF4R9fLTqzrghSCkbQWiLKH1WXMi940w0wD7dz/aPeIcpAjuQ\nPOowEQZzHJDXXc9BWTWEaA7n6UIlk66Kyhq4HD5/3r2FQ9aauIoZtzuj6HSa\nIP4OFGeBxlks7mbatdTwv0kwUsrjl1xCo6yk+BZjp1Xm1sIE3vVH/2M18ILf\nYwhb6BPyfhVfgRHwnr9UrGe220WDw9uN8Ue7lINQjV8jie9C8Mf+AowBvWBY\nkQJy/xnQc55ypdpvSe278FME0BPRLE/A4LsxxT2kG8te/qMKDURkmVZgxOnW\nV/eW/yRevS4dFnE91ura28yltPdBByPhtK8mJwYJElU1a6+ESXo86ukNpnJa\nsbga/2tTUU50TKVuzzRhVJu+bXfB1IRyzhCL5vuti8wMd4r8LIwNARcMPkTD\nDjiR4HkbsiMFX4aFlQ+nBJxeGi278nh7aDYzu/IG0QNL3Lf6jJPyZWQyO87M\nneb30gvIUu4a/wqIK6ogEi2uusEU6+Oc7d62KWSez99JgPtEV4oCIMnhwKQW\nIfhK3Hex1HnQDqa77kwHQJYIGavnYD/fZAxtGCtp1n7rlMYQj6+AfARFqJjs\n5kFf/LpUvlfpCI9DLggwFPovNOpygx5RR/d3QHZMpFQJ+aT9DiwK71ZSAHyQ\nqaMr\r\n=EgYw\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.1_1536535355516_0.30012284760267915" + }, + "_hasShrinkwrap": false + }, + "1.2.2": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.2", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "2ebb034f82713ae79c61f1d147708c11b2ddfac5", + "_id": "fitfont@1.2.2", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-YOaosTQf7HW511x4bTv6VfGNs/UKIDVU9YXWCpElK3u445hnH52J3cPS+tQlQ1mitNcq4ZXH8lb9hN5TEmMhxQ==", + "shasum": "da186bcb57e7e108fe24c79faceac548641da970", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.2.tgz", + "fileCount": 9, + "unpackedSize": 30734, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJblax/CRA9TVsSAnZWagAAvVAP/1asIefoKOjBiEds2BG9\npSsmPevlCMiIcZKDr6xC/z+g4O3Es40xBHJLsfw6PDh+fqiZa5Rm6Fj+oZ8L\n3F7G9HyQzo4GOU+e8YB/GHkpFkQhAkNRc3oNRR/S64XwigzleL87SIkCG10g\nAiQxWjj3rFk9BM7W4t+0+r78RvTInj5DWE90KiG7PKXwm2lGjXFxOGCNdzyp\n5B3KPwquLfQ9jZuBF+kqg3hyWGi92kSdEl+aS3YhHwkL7f0voJ8w6OxDdwlY\nEpzek+XVhFc/EOGYXLBb1pjlaV51bFAb/U339SQm6YZ+dbXqsdlEKnpAzGQj\nbM3B3NquCB3zjqKNppMkspTYYlodigEUd5Zj1Rsae+9s9IS3fW3MHtJrWZR2\njwQDn0zkT2X4KV+dY84NX1JfL2j9eYWvpG6It9VRN4cN7l7fYSOYznaM+sVH\nF7mv7wura1NfcrXWJOiIXypQl+RsNJrfOuX3yQ8KGNtm7UUmoamR2MUjGvlB\njMzwEJpVIeFdD3qhW42U73R1AjWhHk/AQyQb2SeW2zSVD/8U9BOhsriZA84j\n5UlYVDHEHfK7kv2D1gun5bqVp+99mAGG2Bw+FBj8CVx1xaYHZtCboQgLlSba\n4iiDE/ZqTGUKkws7cPe+xoLKBgCVWWiIj4w/WMKCvzT12IFmBjK4hPuC2QVE\nhMTL\r\n=lxph\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.2_1536535679294_0.3985967115691982" + }, + "_hasShrinkwrap": false + }, + "1.2.3": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.3", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "07f0bbd06ee97b8cc85e79de27d39dea2766b03b", + "_id": "fitfont@1.2.3", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-Ni9twpJqzl+I2fCx8Ka/ueTDwyC8DJaRCGMgA7BuSz0GRqws3d/uBFsvioXZPV6V9EAzt+wqA1pcOtTnY3ZpUQ==", + "shasum": "17d5ab368594dad9babdfb981c640a4698d9f805", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.3.tgz", + "fileCount": 9, + "unpackedSize": 30835, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJbmDC4CRA9TVsSAnZWagAAdOQP/RcV0mWga1ijZm3Kax3i\njAae6HSUp3MyFNw3sVd/uJt8oRKqErsgJVjuDIRBHlVskvVI5IeBHt4QP91a\nNFgw/por8Uzp4dabvmvKtb0yPTMcP+21ELI360W/rQSA/z5Yvk7+9v/idpDL\n3aPDVTm5xA8JejQwibkkz8Z9/ZVPj93qL1/jShxrHBWGZy01y5munsa7gVBz\n5qjA3LFVi//t9qYq/mndOF/giQhUhCc8Yupws7EG7tuxLoi+50PKWaUDXsW0\nhasuuXJltP3afv9U0hox6kIuWRycm/0yOVLuT7U/fG1scXl3Pa0kQrAybNae\nvzALNeeaD5hjcu/zig5LLWIfrrcvtcrqArYSRv69r/12MmQ5iOf0jX3bNJfk\nAOhuawvKYot1VClFe1mJmMcGZZD9/tfNntJWLL36yR70/AqFShpKWfrLlKmo\nDH6EbijjNh5pfYDIAYPiY964iQ8ChaTmG8D8QSsel5IgW6tT7szjzvvAe6uY\nL6mSS0Wtx46CGouruMDuvI26pfFNh9TecbVSPJJQV+xEjWU8xAtZNudBEykF\nOgV0ZBw/yEhVcrQdMRugGmJOijc5NdVIv1ohfHd7OwCvVyTQWNkINwbQLUDK\nqJQ3HsUUx1O46QpAWjVwBPGONEndthijaeF5ZlqS6a9WMa5jgubrvmFq1P5J\nWFXF\r\n=uku6\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.3_1536700599865_0.3798776170592211" + }, + "_hasShrinkwrap": false + }, + "1.2.4": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.4", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "9c447b677a57782f0c65bf5c622764279c558a8f", + "_id": "fitfont@1.2.4", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-cNuLPvppDLz3nRThNgb65CD+SnX+teF0Ur/o9+YQc7r5wjVjliVTM1hueJjIvWsng3iEZU6ZZbhiMIMk8/ZCqg==", + "shasum": "40c63286706d704f04b0c83d748fda95ed213bcf", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.4.tgz", + "fileCount": 9, + "unpackedSize": 30998, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJbmEA8CRA9TVsSAnZWagAAS74P/izScTA6Kn33SVehVu/G\n9zQ51uoG1Ga/FAokWAHMN76vL89M2vBV53gpMBCS9YpiVZbTinEQ8I3gIz7Z\n7+EJbKm5L/EhgQiX5wPZM+oP5XUfh/dMZsLsPwMuJWRr1ooMsCRVD3SkrD/7\nb0Pl22PLKmC/vpCKTYnoYcDDOJh1eUApl5oj+Uls6s3GhuTp8qO9xC5zJCjR\nxEYpXTqeh6McU2e/jVD/MRRBinWWYExrb6B/k+0UEjIToCVwWwpIKjSao0FH\nzqhf5gl70Kv/NsTAbbwXUSHvKSMoWtJZbbs7yjXCUqQ528LKpFxKxOgiNVhJ\n4PQzmsXkk9lxynm6ErVMkjtj9UtH2waGkUZP/+o+1ZRjcM2k4Ql1slQX3vhS\n40aY7+jh1cpOPe/74SFInHkolKB1g1anruaGxxm1fQr4Tp/z7Tv95hDL+zqd\nIx1T6EZGQ8tiPITsMBS6xnw84+0ewLg3m0N/HO75yyjbbs0VaRuqC9cdWEZi\ntQKimOijhLoBuykBgVzXeS4ykGFgBuakaqnwsRi7qAjAtGrL4JK0NKTvPKv7\ndH+pz3uLlxEtVTZ0VBC9TJ68QnIBlrnuj6QQNaBaN1P1KeinGI7ImyBng5Zn\nwWQXK1p28PpvDx9+/1e3qU03exh0k2yk8NkcPk2iTlfFoIGAjPaByF1fKKvo\nM7qx\r\n=1Gya\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.4_1536704571698_0.6181887635371259" + }, + "_hasShrinkwrap": false + }, + "1.2.5": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.2.5", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "gitHead": "3baadbeda8d17e71a6edb2b3dfcd59c867fb0a4f", + "_id": "fitfont@1.2.5", + "_npmVersion": "6.4.1", + "_nodeVersion": "8.10.0", + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "dist": { + "integrity": "sha512-mbgCQ5Zh3/dlnlXvN9gsJkes5DH38E2cZ8eVcPGaxhb8dlBd9NX3pDicBMgc4J9oi2XpoLNyDb5m7w5t2RuOsg==", + "shasum": "bff3b8058f4aacce2503efb62d05bbecaca56848", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.2.5.tgz", + "fileCount": 9, + "unpackedSize": 31076, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJboqsRCRA9TVsSAnZWagAAYtgP/1hshuCaevghGgd0TLrt\nnn0b3c5HY4rXWgOoh+8wtBIiYp+2m6rYo/XPGHTKmBAS/bzU39Tly0lpHcZd\nDP8Q7LUBTaNEXHgy73NNQbZQrVzg2b/WldRmr3UYvlEjHwEB6+IUPwSygIiG\np99dNZrPE2HO5O5kNWUTwV20e4tUNqphj6i3R9NHln6E28OO5secyfClewu1\nJ6NwmjeooypIn1e9iesCNNa56u1Hessu2XkV3DMbi6bea9cGPOjBEIQ3/DnD\nxxmDP1jBvoeqhKgEPhT7i5K4cPXc0Wcx81rRheuK6IfJbMq9eM182zo2eYMe\nT+5dFETQ5tVHQg1Bg2sTHb2rRHf3yQtsuXjhqZjgIbrzE5T5p/VOJFGgphgF\nYcyYWVoQ5DEkzDSVzzQnJPYfsjXseR3oO8Ypq4cTcvIP0E7M58hkussNlKs9\n3E4UKAX1ojzrF9Qq4E6mmpsgm9m+Ypl+QNlAxWHds7ljUVcT+vLZ5lu5tsVY\n/X90Hiz/zCwdp05M8IcHRnzfsORtzXuoOzRvWgBHSLyH2yQu38FvVkgZD/nz\nz311decjFbBIIT0HueKZvUkeGudV9JvcbG41gCcpP8PoP6nL5j64F9F5BB9F\nBSDh5PPHJujGRLv3UjEkRbTNe8YEi0//c7mWXUmn9lTD7MIKYveoIC2DAks9\nu0FL\r\n=SDCu\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.2.5_1537387280956_0.6980933336802038" + }, + "_hasShrinkwrap": false + }, + "1.3.0": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.3.0", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "licenseText": "MIT License\n\nCopyright (c) 2018 Grégoire Sage\n\nPermission is hereby granted, free of charge, to any person obtaining a copy\nof this software and associated documentation files (the \"Software\"), to deal\nin the Software without restriction, including without limitation the rights\nto use, copy, modify, merge, publish, distribute, sublicense, and/or sell\ncopies of the Software, and to permit persons to whom the Software is\nfurnished to do so, subject to the following conditions:\n\nThe above copyright notice and this permission notice shall be included in all\ncopies or substantial portions of the Software.\n\nTHE SOFTWARE IS PROVIDED \"AS IS\", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR\nIMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,\nFITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE\nAUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER\nLIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,\nOUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE\nSOFTWARE.\n", + "_id": "fitfont@1.3.0", + "dist": { + "shasum": "b74c79d1c12b3283d9b0066668796361c9741c9d", + "integrity": "sha512-YQaTECVSK2lFeRStSjMg/1KYGS4tk43WqGER8LDY2UpxRo1xkOemqJOC0jSCUMJyYzqwfRI7O2zzxH/SLCGA9w==", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.3.0.tgz", + "fileCount": 23, + "unpackedSize": 42653, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJeEgd/CRA9TVsSAnZWagAAfCoP/2CJmvyEjl32loFmzIEJ\nly25GXNaVwx8Z2lGtUFS7S2N79jSNrJN6ILKJZXst0M2KO2AFOsyk1iAy7Sr\nq8ilh/gsyPrzjxdaCWCbkHUVIWXPNye2nflBAbYb1kUMwBEiERCZRBVj6dQK\nsk89HtZsiWdtisyheRNC0LMqPMICUAthgneapnDT6wxcfXMNTruG7+MYP9vF\nfD4Csrq9TcJRUd2z/nEx2QFXttzV44KOuv6qQHO7eNfcuO1MryhthiGhjqTO\nAlZYhD8KyzZAYaU51CYmqH/K1ccUmaa5eQ3b5B4B4hamX/LfZ+gvBRPWry0Z\nsSJPH68Q83zfLIjle3auDBdzC+h3UVKdHbD0BWvQ+4Vr3XFeiLzYX9Fkix85\nXXb/srMfaO5D57l3Jh1G/cSscN1/hnrXYeE2/uL8LMKXHPyC/zhF+1pgQ1cW\nUqu6tbFKEFw9nD97sjrhBcCqzahZhabxT8aGQbg5sfSqFq1j2NDYrEGdQiEZ\nhcL2WoMudtcyZ60n6fM150/Z6PlRY2gi9qs29UlfU2VqHsxCocaVPy1Z523r\nTxyHwan0AAEddgf8/FIfeAIonI7JK1gKpMqkUm7LoMqZ32OyjxAt2bhgaekR\nTr1hU2+tYAiYZyvmmB4QYiShMfvTBA2y3Dq2XeZaHXNYkw+dUlRfRhAh5IEL\nM1D0\r\n=66E0\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.3.0_1578239870840_0.15126135136832675" + }, + "_hasShrinkwrap": false + }, + "1.3.1": { + "scripts": { + "postinstall": "node ./bin/install.js", + "build": "babel source --out-dir .", + "prepublish": "npm run build" + }, + "babel": { + "presets": [ + [ + "es2015", + { + "modules": false + } + ] + ] + }, + "bin": { + "fitfont-generate": "./bin/generate.js" + }, + "name": "fitfont", + "description": "This library allows you to easily display text with custom fonts.", + "version": "1.3.1", + "main": "fitfont.js", + "devDependencies": { + "babel-cli": "^6.26.0", + "babel-preset-es2015": "^6.24.1" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "keywords": [ + "fitbitdev" + ], + "author": { + "name": "Grégoire Sage" + }, + "license": "MIT", + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "dependencies": { + "python-shell": "^1.0.4", + "readline2": "^1.0.1" + }, + "licenseText": "MIT License\n\nCopyright (c) 2018 Grégoire Sage\n\nPermission is hereby granted, free of charge, to any person obtaining a copy\nof this software and associated documentation files (the \"Software\"), to deal\nin the Software without restriction, including without limitation the rights\nto use, copy, modify, merge, publish, distribute, sublicense, and/or sell\ncopies of the Software, and to permit persons to whom the Software is\nfurnished to do so, subject to the following conditions:\n\nThe above copyright notice and this permission notice shall be included in all\ncopies or substantial portions of the Software.\n\nTHE SOFTWARE IS PROVIDED \"AS IS\", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR\nIMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,\nFITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE\nAUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER\nLIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,\nOUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE\nSOFTWARE.\n", + "_id": "fitfont@1.3.1", + "dist": { + "shasum": "766ef53baea9c03406d52760313617aa31e9bdc6", + "integrity": "sha512-09b0eYCAgZ6l+G1pvkmtJ4YiwADxvTWlDmX/isd7/3qdxdi75+OJQfYJFnT9qRLIkexKssXK2dG1n77Pr/pdYw==", + "tarball": "https://registry.npmjs.org/fitfont/-/fitfont-1.3.1.tgz", + "fileCount": 49, + "unpackedSize": 57902, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJeFlVZCRA9TVsSAnZWagAAtp8P/1SjtqNQMJ92kiJr9Uic\nQiaoiVFvLCp6rgWl8tTTdinJbKEcLtSagoMfBxhiD6ZpsKilXzVLGZ34JOmY\nqE1mxDNIIFHXMGR0z3962zVXV4bmLmdSmIypKjgE2nxr99Qneo41zj+oso65\n8OArXcjvNYkbjj4NpMr8Lb7HR6pq4abRDwJYgcP+YDimn/ORCGW6pqEa3AU5\nl5Inu7qF8vtTnfUhfYSZiefOeFpqnMncepcvkM8KghyDtG/CPWjneXHOV+bH\ntSPSgnvpq0GSWFQl8ER50TVV5LbmS08sJSENSrRxNIvCx5mMqf9YwHnRFghh\n/KYXizzQkmakqkKlLYKhP+mD1ebkNGLx8pMOgji+s4PfmlWzSyaJC1ugk88N\naIzzHurLReQIo0kGhNfBtibmL0cycCXKatknuX3xxqrTCBBBnEOED8+VlB2H\ng5YrGwGtnX6y32GBuoEB8Ak0v42YSUJwsq9o9brmJ0B2ObxPZbQ4mUfSc+QA\njmWp5f+952b6ViQYUC/vFupDyHcHVB4fp1Zngzassa33u0LwST6aOqzeLOIY\nvahpLSj45vWyNLwZ+BCcP2FgFD6hAsnn6U3x2cT6ZkKIJ2Qq/CwA8qLP2wry\nymBwSsMt2MlPsjH2v6RsAaaIwArWaikyqKwHbt4p8e/daJ243DAw/sWhNiRC\nU4Wp\r\n=xE98\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "_npmUser": { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/fitfont_1.3.1_1578521944205_0.14338694411758457" + }, + "_hasShrinkwrap": false + } + }, + "time": { + "created": "2018-09-09T13:24:15.691Z", + "0.0.1": "2018-09-09T13:24:15.868Z", + "modified": "2020-01-08T22:19:07.288Z", + "0.0.2": "2018-09-09T20:32:54.674Z", + "1.2.0": "2018-09-09T23:17:40.293Z", + "1.2.1": "2018-09-09T23:22:35.718Z", + "1.2.2": "2018-09-09T23:27:59.477Z", + "1.2.3": "2018-09-11T21:16:40.057Z", + "1.2.4": "2018-09-11T22:22:51.832Z", + "1.2.5": "2018-09-19T20:01:21.047Z", + "1.3.0": "2020-01-05T15:57:50.949Z", + "1.3.1": "2020-01-08T22:19:04.303Z" + }, + "maintainers": [ + { + "name": "gregoire", + "email": "gregoire.sage@gmail.com" + } + ], + "description": "This library allows you to easily display text with custom fonts.", + "homepage": "https://github.com/gregoiresage/fitfont#readme", + "keywords": [ + "fitbitdev" + ], + "repository": { + "type": "git", + "url": "git+https://github.com/gregoiresage/fitfont.git" + }, + "author": { + "name": "Grégoire Sage" + }, + "bugs": { + "url": "https://github.com/gregoiresage/fitfont/issues" + }, + "license": "MIT", + "readme": "# FitFont for Fitbit\n\nThis library allows you to easily display text with custom fonts.\n\n[![Support via PayPal](https://cdn.rawgit.com/twolfson/paypal-github-button/1.0.0/dist/button.svg)](https://www.paypal.me/gsage/)\n\n![alt text](screenshot.png \"Sorry for the colors\")\n\n## With the Fitbit CLI\n\n### Installation\n\nInstall the library with `npm i fitfont`\n\n The installer will ask you if it can copy the library gui files in your `resources` folder: \n Press `Y` and the `fitfont.gui` file will be created automatically. \n Press `N` if you prefer to copy the file manually afterwards (default choice is Yes) \n \nYou still need to modify the `resources/widgets.gui` file to add the link to `fitfont.gui`:\n``` xml\n\n \n \n \n \n \n \n\n```\n\n### Assets generation\n\nDownload your favourite ttf font and generate the library needed files with:\n```\n npx fitfont-generate [path/to_the_font_file] [font_size] [list_of_characters_to_generate]\n```\ne.g.\n```\n npx fitfont-generate /path/to/my_cool_font.ttf 200 0123456789:.\n```\nThe generated folder (named [font_name]_[font_size]) will be automatically copied in your `resources` folder.\n\nRepeat this for every font/size you need.\n\n\n## Without the Fitbit CLI (shame on you)\n\n### Library integration\n\nCopy the `fitfont.js` file in your `app` folder.\n\nCopy the `fitfont.gui` file in your `resources` folder.\n\nModify the `resources/widgets.gui` file to add the link to `fitfont.gui`:\n``` xml\n\n \n \n \n \n \n \n\n```\n\n### Assets generation\n\nInstall the python dependencies:\n```\n pip install Pillow\n```\n\nDownload your favourite font file and call the python `generator.py` script of this repo.\n```\n python generate.py [path/to_the_font_file] [font_size] [list_of_characters_to_generate]\n```\ne.g.\n```\n python generate.py /path/to/my_cool_font.ttf 200 0123456789:.\n```\n\nCopy the generated folder (named [font_name]_[font_size]) in your project's `resources` folder.\n\nRepeat this for every font/size you need.\n\n# Usage\n\nUse a `fitfont` symbol in your `index.gui` file:\n``` xml\n \n \n \n \n \n \n \n \n \n```\n\nImport and use the library in your `app/index.js`:\n``` javascript\n import { FitFont } from 'fitfont'\n // or if you are not using the CLI : import { FitFont } from './fitfont.js'\n \n const myLabel = new FitFont({ \n id:'myLabel', // id of your symbol in the index.gui, you can also give an element object e.g. id: document.getElementById('foo')\n font:'Market_Saturday_200' // name of the generated font folder\n\n // Optional\n halign: 'start', // horizontal alignment : start / middle / end\n valign: 'baseline', // vertical alignment : baseline / top / middle / bottom\n letterspacing: 0 // letterspacing...\n })\n \n myLabel.text = '12:55' // only the characters generated with the python script will be displayed\n\n // It is also possible to change the halign/valign/letterspacing dynamically\n myLabel.halign = 'middle'\n myLabel.valign = 'top'\n myLabel.letterspacing = -3\n```\n\nLaunch your app and enjoy.\n\n# Demo\n\nA demo is available [here](https://github.com/gregoiresage/fitfont-demo)\n", + "readmeFilename": "README.md" +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/tiny.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/tiny.json new file mode 100644 index 00000000..0c6612db --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/tiny.json @@ -0,0 +1,120 @@ +{ + "_id": "@bamblehorse/tiny", + "_rev": "2-9526bb8fdedb67c1fe82abbad9de9a4f", + "name": "@bamblehorse/tiny", + "dist-tags": { + "latest": "2.0.0" + }, + "versions": { + "1.0.0": { + "name": "@bamblehorse/tiny", + "version": "1.0.0", + "gitHead": "059629d1df46c4cdb40c1f2f1964c7dd84bca991", + "_id": "@bamblehorse/tiny@1.0.0", + "_npmVersion": "5.6.0", + "_nodeVersion": "8.11.3", + "_npmUser": { + "name": "bamblehorse", + "email": "bamblehorse@gmail.com" + }, + "dist": { + "integrity": "sha512-NeqbFrUda1x1sJJF1/4AwWKCvm5x0faRrBZ1GswmzaeggE1zZf8e6Nw5oZZrTxgFpLU1/J7bWvoJ3Zf1N2Mppw==", + "shasum": "aa9862fdb97ee8f08889780e73860dee35fa3de3", + "tarball": "https://registry.npmjs.org/@bamblehorse/tiny/-/tiny-1.0.0.tgz", + "fileCount": 1, + "unpackedSize": 56, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJbVG1pCRA9TVsSAnZWagAAn1EQAIiFIsqsLq6xODFpmsMT\nydYgndpdM2/rOsRQCXoDoS0shDQTrI6cAi9DZ2BJXT+dAwld4zO5tpFgy/Ly\nG/wI2n72wbAemqs/Rf+Lmo5DPswSu6jdkQw9Aiy78G/+5BQbwluXhHLOyEUE\ng17HIy3OF/zZGi4mAFWKLIUlC9PBco4o/dBa1cdLI6HF6M+WUZYB84Xm01nJ\nGDvzYEbGVsXQ6CufYMUneSgXPL+Etm1hJ23qLggeTBCASYa9CxrZNSOf9jE6\ntVk2MD5rKmNCqIWuAayEp6kQVuRHMQviBI0vgSzFXArnmaRj17dBosMWmSW2\nsXTh/Na9UhyzAggmNyFa7YsnzFUfwSpiG5YMSbBaJRW7kLvAdlM7hVEsoDf7\nwkuiu6xvaPImlL4JTx+fY1OonHx5D74ii1WpdvpIl0nOiEGkTP0bY6FCI64W\naJZ1cXyHAUM3pOf+8x9Y14k0CSbKlBufWwXC8JNSsqk5DsPfpAwdf8+6tESA\ncDE7xOoIljIZ3y0q4ft0+lZ4SWiVZt33IE+fvdvCL9EFY9mEsW2EoiYasZys\nEsQrfxeeWd9BPiH/3omymvQS9EL2lpqIXXahirSUjzPmN6QceenRmDx1oY45\nxpaAdvKvF1LSaZuUvt9qDTooceS8Rk2RUKM1joKlzOtUREIIegDggT7cbGfS\nPKpE\r\n=Yrzg\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "bamblehorse", + "email": "bamblehorse@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/tiny_1.0.0_1532259689529_0.44546873546737387" + } + }, + "2.0.0": { + "name": "@bamblehorse/tiny", + "version": "2.0.0", + "description": "Removes all spaces from a string", + "license": "MIT", + "repository": { + "type": "git", + "url": "git+https://github.com/bamblehorse/tiny.git" + }, + "main": "index.js", + "keywords": [ + "tiny", + "npm", + "package", + "bamblehorse" + ], + "gitHead": "2a03af08285d3f989f8f6b4252293ec5e6ef7a26", + "bugs": { + "url": "https://github.com/bamblehorse/tiny/issues" + }, + "homepage": "https://github.com/bamblehorse/tiny#readme", + "_id": "@bamblehorse/tiny@2.0.0", + "_npmVersion": "5.6.0", + "_nodeVersion": "8.11.3", + "_npmUser": { + "name": "bamblehorse", + "email": "bamblehorse@gmail.com" + }, + "dist": { + "integrity": "sha512-l2muGeiKltR0sSAqV8n/qTNFdhuwpN/2cW919q/upK+KrUcVhW+rgH9ipOTjEzPOLUmJs6JH2RABUSGZ7Ly8lA==", + "shasum": "a58d73ac5a4affec8c8bb19824e36d72581db658", + "tarball": "https://registry.npmjs.org/@bamblehorse/tiny/-/tiny-2.0.0.tgz", + "fileCount": 3, + "unpackedSize": 1021, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJbVxXkCRA9TVsSAnZWagAAFNIP/3oYsl1slDfaavKSl9dm\nq18OjzPMZuElhtO+q3bsZKbJlOiivnlKAAQx8MWGjxCgarmNE86vAR1OQQgr\nuI6vUv0XzXTwVU588yRUKolaaso8QlJndGUzaIodeUkrn3UjSi4fOz0UusO+\ny3vqekpl+yvR+s+df8pVFSMX17b4fMdZqMvxEL/lyDvmaLzKmkQkQOMZairA\nvryVpyF9Lx7eiPVgmft9nm9XoA5wcILNCz0uZIiD/asTfnEvWmd5yZW+2XbB\nzKAkPpab6VvPyYK4sne5h6ANSxa3bsrdiisTvC6h7W9zHc81th22YyEq5qc+\nLMmfTmVHZw632tBdDlBDkkG7HmfjZKMvtwYSGdZdSNJmNwSVc86WRiv1YPCU\nRUJ9ql84ADBMcJVn63aMbuAFSzcrTdb5SMDAIJjNU7GaUf+8mV3upVMuTCNO\np5vqpVb2jqIB9fDGCC/27WB8cJwJlihKa3B7+QU1skGLoQLvUNbeekItamcq\ndcytmPOcwBMEVToLam+jU37D6AWfYkDIV8/vcvqP9EONmLcBoW9CgqYywlnP\nGSdz5IHFFuqIOl+3oe2T7fSMg0pluYsbipUixYcFyoomty4vyxkK0BoZ0lE9\nlZdeDglBu5K3HwlEcnIYyRdp/HEjsGn6D724ydBselaJwLvKRJTQVL5bXkp3\naFbb\r\n=d9hK\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "bamblehorse", + "email": "bamblehorse@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/tiny_2.0.0_1532433892870_0.7449554709515787" + } + } + }, + "time": { + "created": "2018-07-22T11:41:29.226Z", + "1.0.0": "2018-07-22T11:41:29.631Z", + "modified": "2018-07-24T12:04:56.497Z", + "2.0.0": "2018-07-24T12:04:52.956Z" + }, + "maintainers": [ + { + "name": "bamblehorse", + "email": "bamblehorse@gmail.com" + } + ], + "readme": "# @bamblehorse/tiny\n\n[![npm (scoped)](https://img.shields.io/npm/v/@bamblehorse/tiny.svg)](https://www.npmjs.com/package/@bamblehorse/tiny)\n[![npm bundle size (minified)](https://img.shields.io/bundlephobia/min/@bamblehorse/tiny.svg)](https://www.npmjs.com/package/@bamblehorse/tiny)\n\nRemoves all spaces from a string.\n\n## Install\n\n```\n$ npm install @bamblehorse/tiny\n```\n\n## Usage\n\n```js\nconst tiny = require(\"@bamblehorse/tiny\");\n\ntiny(\"So much space!\");\n//=> \"Somuchspace!\"\n\ntiny(1337);\n//=> Uncaught TypeError: Tiny wants a string!\n// at tiny (:2:41)\n// at :1:1\n```\n", + "readmeFilename": "README.md", + "description": "Removes all spaces from a string", + "homepage": "https://github.com/bamblehorse/tiny#readme", + "keywords": [ + "tiny", + "npm", + "package", + "bamblehorse" + ], + "repository": { + "type": "git", + "url": "git+https://github.com/bamblehorse/tiny.git" + }, + "bugs": { + "url": "https://github.com/bamblehorse/tiny/issues" + }, + "license": "MIT", + "_attachments": {} +} diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/ts2php.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/ts2php.json new file mode 100644 index 00000000..3d53c79d --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/ts2php.json @@ -0,0 +1,371 @@ +{ + "_id": "ts2php", + "_rev": "84-a99a3e14f1d576d910aafb083cba8673", + "name": "ts2php", + "dist-tags": { + "alpha": "0.0.1-alpha.1", + "latest": "0.19.11", + "beta": "0.13.0-beta.0" + }, + "versions": { + "0.0.1-alpha.1": { + "name": "ts2php", + "version": "0.0.1-alpha.1", + "description": "ts转php", + "main": "dist/index.js", + "scripts": { + "build": "tsc", + "sync": "sync-files -w ./typescript.d.ts ./node_modules/typescript/lib/typescript.d.ts", + "test": "tsc && mocha", + "mocha": "mocha" + }, + "repository": { + "type": "git", + "url": "ssh://git@icode.baidu.com:8235/baidu/atom/ts2php" + }, + "author": { + "name": "meixuguang" + }, + "license": "UNLICENSED", + "dependencies": { + "fs-extra": "^7.0.1", + "hash-sum": "^1.0.2", + "lodash": "^4.17.11", + "typescript": "3.1.3" + }, + "devDependencies": { + "@types/fs-extra": "^5.0.5", + "@types/lodash": "^4.14.117", + "@types/node": "^10.12.0", + "koa": "^2.6.2", + "koa-bodyparser": "^4.2.1", + "koa-router": "^7.4.0", + "koa-static": "^5.0.0", + "mocha": "^5.2.0", + "sync-files": "^1.0.3", + "ts-node": "^7.0.1" + }, + "gitHead": "fb29a2f4c4a0d064f029bdb9e89dbe61a00ad90b", + "_id": "ts2php@0.0.1-alpha.1", + "_npmVersion": "6.5.0", + "_nodeVersion": "8.11.2", + "_npmUser": { + "name": "cxtom", + "email": "cxtom2010@gmail.com" + }, + "dist": { + "integrity": "sha512-jNa0yRwS0zMlzwMxpzyaOZtadDut94gc1kLgS+L/MSBh+227j7VD6RSTnG1XjYqJdrbnAst2bzI7DSlPBQVQcA==", + "shasum": "78f934fdaf7ba2e24a9614efffbdb30b1b280411", + "tarball": "https://registry.npmjs.org/ts2php/-/ts2php-0.0.1-alpha.1.tgz", + "fileCount": 25, + "unpackedSize": 584356, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJca5tlCRA9TVsSAnZWagAASW4P/2vkJkr7cBs7RHSv1iUI\n6AFzmuLXbUM2XZfJ+DHjXE+l2ZGIPRFeeu5b2W/7gyWVNbCN9dHA0ohYTj6R\nAi8mpweo3eeIcvHdf7CxhCYtVbiMHWfjamjsTLeDWOgcWcVnKjcaeOTq4ami\nOIkmEsH73tcWKehn9j759yGd57EsIVDyVuoyBQh+RLJdlkZqcylP9Xed8f1/\nsF1GvW+ACr4ydSwISyHuboC8DWXWzpUmYJpxZsajyl2wIjbc3sb48M53BL/t\noukDr046N1CDlA+6YnThWGc2b9LI7cAeXL7Y5Cy7H8999+jJtuCKFoAHf0H3\nwk4Ka+IYJ8XvJHt85V9KDvgtIwpSCDtRlpD+ouKj/DfRTtVA7G8YY5c2Uz1R\nr3uH6ZmE2qtlDcTHY+ayrhhYVZcbRyiI7WD7KIrZTtmBzMQkEdDHFZ7zBtOj\nvur98IfoUuZFQ0EtxN+LbCMxqxBWJxAaFzOaA20l5okEbrq3MhjM1Llkvq/u\nS43FOQy3TzG+EyjQ6YiQmpCgr7E7GY5nsPPJeLFhm6zNZJp/kkLmbIhVl5rq\n+zsbPknsaqjmVocCBLoK4v+JicP7tE6pswjctLAZAqJ9we58+oI7wEWkNITK\noxHTm4iutuIuklunItsWOncUXQkANTWmqdyJ4RPD+QggkUvECXNdUQtWBu8c\nnzWN\r\n=7Cyd\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "name": "cxtom", + "email": "cxtom2010@gmail.com" + } + ], + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/ts2php_0.0.1-alpha.1_1550556004976_0.9770363962614048" + }, + "_hasShrinkwrap": false + }, + + "0.19.11": { + "name": "ts2php", + "version": "0.19.11", + "description": "TypeScript to PHP Transpiler", + "main": "dist/index.js", + "typings": "types/index.d.ts", + "bin": { + "ts2php": "dist/bin/ts2php.js" + }, + "scripts": { + "prebuild": "rimraf dist", + "build": "tsc && mkdir -p dist/runtime && cp src/runtime/*.php dist/runtime && chmod a+x dist/bin/*", + "sync": "cp ./typescript.d.ts ./node_modules/typescript/lib/typescript.d.ts", + "test": "mocha", + "coverage": "nyc _mocha", + "prepublishOnly": "npm test && npm run build", + "test:feature": "ts-node ./src/bin/ts2php.ts", + "test:runtime": "sh ./test/runtime/run.sh", + "changelog": "conventional-changelog -p angular -i CHANGELOG.md -s -r 0", + "doc": "typedoc --out docs --theme minimal --includeDeclarations --excludeNotExported --excludeExternals --mode file ./types/index.d.ts" + }, + "repository": { + "type": "git", + "url": "git+https://github.com/searchfe/ts2php.git" + }, + "author": { + "name": "meixuguang" + }, + "contributors": [ + { + "name": "cxtom", + "email": "cxtom2008@gmail.com" + } + ], + "license": "MIT", + "dependencies": { + "fs-extra": "^7.0.1", + "lodash": "^4.17.14", + "semver": "^6.2.0", + "ts-morph": "^3.1.0", + "yargs": "^13.2.4" + }, + "nyc": { + "include": [ + "src/*.ts", + "src/**/*.ts" + ], + "exclude": [ + "test/**/*.ts", + "**/*.d.ts" + ], + "extension": [ + ".ts" + ], + "require": [ + "ts-node/register" + ], + "reporter": [ + "text-summary", + "html" + ], + "sourceMap": true, + "instrument": true + }, + "commitlint": { + "extends": [ + "@commitlint/config-conventional" + ] + }, + "husky": { + "hooks": { + "commit-msg": "commitlint -E HUSKY_GIT_PARAMS" + } + }, + "peerDependencies": { + "typescript": "~3.4.5" + }, + "devDependencies": { + "@commitlint/cli": "^7.6.1", + "@commitlint/config-conventional": "^7.6.0", + "@semantic-release/changelog": "^3.0.2", + "@semantic-release/commit-analyzer": "^6.1.0", + "@semantic-release/git": "^7.0.8", + "@semantic-release/npm": "^5.1.8", + "@semantic-release/release-notes-generator": "^7.1.4", + "@types/fs-extra": "^5.1.0", + "@types/lodash": "^4.14.132", + "@types/mocha": "^5.2.7", + "@types/node": "^10.14.7", + "@typescript-eslint/eslint-plugin": "^2.6.1", + "@typescript-eslint/experimental-utils": "^2.6.1", + "@typescript-eslint/parser": "^2.6.1", + "camelcase": "^5.3.1", + "chalk": "^3.0.0", + "conventional-changelog": "^3.1.3", + "conventional-changelog-cli": "^2.0.12", + "eslint": "^6.6.0", + "eslint-plugin-import": "^2.18.2", + "eslint-plugin-jsdoc": "^17.1.1", + "eslint-plugin-no-null": "^1.0.2", + "glob": "^7.1.6", + "husky": "^1.3.1", + "istanbul": "^0.4.5", + "koa": "^2.7.0", + "koa-bodyparser": "^4.2.1", + "koa-router": "^7.4.0", + "koa-static": "^5.0.0", + "mdgator": "^1.1.2", + "mocha": "^6.1.4", + "nyc": "^14.1.1", + "rimraf": "^2.6.3", + "source-map-support": "^0.5.12", + "ts-node": "^8.2.0", + "typedoc": "^0.15.3", + "typescript": "~3.4.5", + "vue": "^2.6.10" + }, + "release": { + "branch": "master", + "plugins": [ + "@semantic-release/commit-analyzer", + "@semantic-release/release-notes-generator", + "@semantic-release/changelog", + "@semantic-release/npm", + [ + "@semantic-release/git", + { + "assets": [ + "package.json", + "CHANGELOG.md" + ], + "message": "chore(release): ${nextRelease.version} [skip ci]\n\n${nextRelease.notes}" + } + ], + "@semantic-release/github" + ] + }, + "gitHead": "f16f0b57edd6d18fec721202dfa076ab037b205b", + "bugs": { + "url": "https://github.com/searchfe/ts2php/issues" + }, + "homepage": "https://github.com/searchfe/ts2php#readme", + "_id": "ts2php@0.19.11", + "_nodeVersion": "10.18.0", + "_npmVersion": "6.13.4", + "dist": { + "integrity": "sha512-y2xfD5YtbOEz9OU619Wo9G3QCk0th0QnKQ5y2N271NbTf1bg6AmxQxj+2MIBjKS01D6v30hZVHyOojpkAgFDpw==", + "shasum": "25f221cd688e1733548a157713cb746b6716c23d", + "tarball": "https://registry.npmjs.org/ts2php/-/ts2php-0.19.11.tgz", + "fileCount": 95, + "unpackedSize": 921808, + "npm-signature": "-----BEGIN PGP SIGNATURE-----\r\nVersion: OpenPGP.js v3.0.4\r\nComment: https://openpgpjs.org\r\n\r\nwsFcBAEBCAAQBQJeANSgCRA9TVsSAnZWagAAjCMP/iHDZjz2zVX6gT6AY/jn\nc/EQTsRH3/+vpqeHIEKiIieTvJh1zbwzSoYVgFdJxbMXFocY0ApFZc2FsA7D\nBLeyJmQteZzUdQA1AkG+n7Hcb5RobWIOCER/tIlQw0x7Ma7gVQChaaFX991/\nt2LAJRt9V+bcxEQA1Qdbg/LNstk8vEm7uQbbht6GobciU0Vm/2NRVGY1a4+L\nf5VOCZYpeCXIKUT3WbeklAGHsmBeB+rrWyD5z0dXHd8iICO4a6/KuY1L+NCq\ns8Xxv+iy7CYb5d2rPq6qPqq7ajvihqHFW/GfpLJH/BN6KGCus7PPPrmMwcwR\ng3Sh7qhtISBhnSCs+8TepYRFPb4eGsbJxlCArx6QQePjNehzHj1nVFIjsewN\nfXptkd/hGIVIp5aTp3D9n59vIDlxQzMxd2pxsHDHkzuhZ281TOZ3YbUVY9iQ\neP8oVAvdsbC01t2TMmJCUyLHb+yX/zWA3n/H3rPuULaaDt6GL1VXB+QrraGC\n85HJR1E7V87oLTUV4n1Lc9UWR/gWLEUJZ9aFxy6xsYSAIv425nw/5T7fCfYi\nldB/QVyO5n83zFaJFd6O4YqKloTmniEOYQvghZBwMYvsWdjPMT3eJyHoO7Nd\n6xnS5nV2k6HG1xSj189tHVe81frA2H+S+JxecifzY3fac27O96Q4nNIltM0S\nb0YQ\r\n=2tOU\r\n-----END PGP SIGNATURE-----\r\n" + }, + "maintainers": [ + { + "email": "cxtom2010@gmail.com", + "name": "cxtom" + }, + { + "email": "meixg@foxmail.com", + "name": "meixg" + } + ], + "_npmUser": { + "name": "meixg", + "email": "meixg@foxmail.com" + }, + "directories": {}, + "_npmOperationalInternal": { + "host": "s3://npm-registry-packages", + "tmp": "tmp/ts2php_0.19.11_1577112735797_0.6937637919869244" + }, + "_hasShrinkwrap": false + } + }, + "time": { + "created": "2019-02-19T06:00:04.974Z", + "0.0.1-alpha.1": "2019-02-19T06:00:05.161Z", + "modified": "2019-12-23T14:52:18.893Z", + "0.0.1-alpha.2": "2019-02-20T11:46:01.275Z", + "0.0.1-alpha.3": "2019-02-21T06:48:31.546Z", + "0.0.1-alpha.4": "2019-02-21T12:49:50.908Z", + "0.0.1-alpha.6": "2019-02-25T12:07:08.830Z", + "0.0.1-alpha.7": "2019-02-26T03:35:03.094Z", + "0.0.1-alpha.8": "2019-02-27T06:11:13.414Z", + "0.0.1-alpha.9": "2019-02-27T07:34:41.024Z", + "0.0.1-alpha.10": "2019-02-27T08:32:15.941Z", + "0.1.0": "2019-02-28T08:13:09.981Z", + "0.1.1": "2019-03-03T07:33:43.056Z", + "0.1.2": "2019-03-03T08:02:21.727Z", + "0.1.3": "2019-03-14T09:54:10.092Z", + "0.1.4": "2019-03-15T03:59:19.296Z", + "0.1.5": "2019-04-09T06:40:24.570Z", + "0.2.1": "2019-04-17T04:51:30.993Z", + "0.2.2": "2019-04-17T11:06:17.482Z", + "0.3.0": "2019-04-18T05:50:25.518Z", + "0.3.1": "2019-04-18T12:11:54.733Z", + "0.4.0": "2019-04-22T07:19:48.144Z", + "0.5.0": "2019-04-23T12:59:54.544Z", + "0.6.0": "2019-04-25T12:24:05.843Z", + "0.6.1": "2019-04-26T11:58:51.226Z", + "0.7.0": "2019-04-29T03:48:11.423Z", + "0.8.0": "2019-04-29T09:52:06.880Z", + "0.8.1": "2019-05-05T10:10:58.988Z", + "0.9.0": "2019-05-07T03:45:05.264Z", + "0.9.1": "2019-05-09T11:20:44.006Z", + "0.9.2": "2019-05-09T12:39:57.707Z", + "0.9.3": "2019-05-10T08:51:01.355Z", + "0.9.4": "2019-05-22T13:34:05.644Z", + "0.9.5": "2019-05-27T02:26:25.910Z", + "0.9.6": "2019-05-30T07:44:57.376Z", + "0.9.7": "2019-06-03T09:58:29.813Z", + "0.9.8": "2019-06-04T07:38:22.169Z", + "0.10.0": "2019-06-05T02:30:07.099Z", + "0.10.1": "2019-06-18T08:25:43.895Z", + "0.10.2": "2019-06-19T11:40:44.116Z", + "0.10.3": "2019-06-20T08:23:04.535Z", + "0.10.4": "2019-06-21T09:39:51.064Z", + "0.10.5": "2019-06-21T10:21:49.268Z", + "0.10.6": "2019-06-25T08:14:16.897Z", + "0.10.7": "2019-06-26T02:14:13.182Z", + "0.10.8": "2019-07-08T06:04:09.312Z", + "0.10.9": "2019-07-08T06:20:09.902Z", + "0.11.0": "2019-07-10T03:56:11.273Z", + "0.11.1": "2019-07-10T05:49:50.783Z", + "0.11.2": "2019-07-22T04:07:01.360Z", + "0.11.3": "2019-07-22T06:11:44.686Z", + "0.12.0-0": "2019-08-05T05:59:53.409Z", + "0.12.0": "2019-08-05T06:34:12.203Z", + "0.12.1": "2019-08-26T08:09:33.064Z", + "0.13.0-beta.0": "2019-09-09T05:11:08.522Z", + "0.12.2": "2019-09-10T03:45:03.336Z", + "0.12.3": "2019-09-26T02:52:33.384Z", + "0.12.4": "2019-10-12T14:56:21.873Z", + "0.12.5": "2019-10-18T06:48:56.019Z", + "0.12.6": "2019-10-21T09:04:42.673Z", + "0.12.7": "2019-10-24T03:13:53.119Z", + "0.12.8": "2019-10-24T06:13:11.518Z", + "0.12.9": "2019-10-28T04:36:55.589Z", + "0.12.10": "2019-10-29T05:44:08.928Z", + "0.12.11": "2019-10-30T09:47:53.414Z", + "0.12.12": "2019-10-31T12:49:48.831Z", + "0.13.0": "2019-11-14T08:21:00.574Z", + "0.14.0": "2019-11-14T09:40:06.032Z", + "0.15.0": "2019-11-18T13:34:00.137Z", + "0.16.0": "2019-11-19T05:10:12.864Z", + "0.16.1": "2019-11-19T07:57:04.131Z", + "0.17.0": "2019-11-20T07:06:37.111Z", + "0.18.0": "2019-11-20T12:18:54.613Z", + "0.19.0": "2019-11-24T02:37:11.290Z", + "0.19.1": "2019-11-27T10:04:19.510Z", + "0.19.2": "2019-11-28T04:09:55.188Z", + "0.19.3": "2019-11-28T07:35:34.312Z", + "0.19.4": "2019-12-01T10:21:41.384Z", + "0.19.5": "2019-12-06T10:45:02.236Z", + "0.19.6": "2019-12-09T12:13:29.254Z", + "0.19.7": "2019-12-11T12:38:28.958Z", + "0.19.8": "2019-12-12T07:01:55.430Z", + "0.19.9": "2019-12-13T07:50:58.615Z", + "0.19.10": "2019-12-23T10:36:43.218Z", + "0.19.11": "2019-12-23T14:52:15.964Z" + }, + "maintainers": [ + { + "email": "cxtom2010@gmail.com", + "name": "cxtom" + }, + { + "email": "meixg@foxmail.com", + "name": "meixg" + } + ], + "description": "TypeScript to PHP Transpiler", + "repository": { + "type": "git", + "url": "git+https://github.com/searchfe/ts2php.git" + }, + "author": { + "name": "meixuguang" + }, + "license": "MIT", + "readme": "# ts2php\n\n**under development**\n\nTypeScript 转 PHP\n\nA Compiler which can compile TypeScript to PHP.\n\n![Language](https://img.shields.io/badge/-TypeScript-blue.svg)\n[![Build Status](https://travis-ci.com/searchfe/ts2php.svg?branch=master)](https://travis-ci.org/searchfe/ts2php)\n[![npm package](https://img.shields.io/npm/v/ts2php.svg)](https://www.npmjs.org/package/ts2php)\n[![npm downloads](http://img.shields.io/npm/dm/ts2php.svg)](https://www.npmjs.org/package/ts2php)\n[![semantic-release](https://img.shields.io/badge/%20%20%F0%9F%93%A6%F0%9F%9A%80-semantic--release-e10079.svg)](https://github.com/searchfe/ts2php)\n\n- [ts2php](#ts2php)\n - [Usage](#usage)\n - [compiler](#compiler)\n - [runtime](#runtime)\n - [CLI](#cli)\n - [update ts2php version](#update-ts2php-version)\n - [Features](#features)\n - [Javascript Syntax](#javascript-syntax)\n - [Core JavaScript API](#core-javascript-api)\n - [Thanks to](#thanks-to)\n\n## Usage\n\n### compiler\n\n```javascript\nimport {compile} from 'ts2php';\n\nconst result = compile(filePath, options);\n```\n\n### runtime\n\n> 部分功能依赖一个 PHP 的类库,需要在 PHP 工程中引入\n\n> Some features are implemented by a PHP helper class, which need to be included in your PHP\n code.\n\n```php\nrequire_once(\"/path/to/ts2php/dist/runtime/Ts2Php_Helper.php\");\n```\n\n### CLI\n\nQuick Start:\n\n```bash\n$ npm i -g ts2php\n$ ts2php ./a.ts # 编译输出到 stdout\n```\n\n使用[配置][options]并输出到文件:\n\n```bash\n$ cat config.js\nmodule.exports = {\n emitHeader: false\n};\n$ ts2php -c config.js src/ -o output/\n```\n\n更多选项:\n\n```bash\n$ ts2php --show-diagnostics # 输出诊断信息\n$ ts2php --emit-header # 输出头部信息\n$ ts2php -h # 更多功能请查看帮助\n```\n\n### update ts2php version\n\nSame TS code with different version of ts2php may result to different PHP code. When updating the version of ts2php, we should check the result PHP code manually. To simplify this process, we recommend to use [ts2php-diff-checker][ts2php-diff-checker]. Specify two version of ts2php, and some source TS code, [ts2php-diff-checker][ts2php-diff-checker] will generate diff info directly.\n\n```sh\nts2php-check [destination]\n```\n\n## Features\n\n### Javascript Syntax\n\n- [`for`/`for of`/`for in`](./test/features/ForStatement.md)\n- [`if`/`else if`/`else`](./test/features/IfStatement.md)\n- [`swtich`](./test/features/SwitchStatement.md)\n- [`while`/`do while`](./test/features/WhileStatement.md)\n- [`Class`](./test/features/Class.md)\n- [`typeof`](./test/features/GlobalApi.md)\n- [`delete`](./test/features/GlobalApi.md)\n- [`destructuring`](./test/features/Destructuring.md)\n- [`template string`](./test/features/template.md)\n- [`object computed property`](./test/features/ComputedPropertyName.md)\n- [`object shorthand property`](./test/features/ShorthandPropertyAssignment.md)\n- [`enum`](./test/features/EnumDeclaration.md)\n- [`anonymous function inherit variables`](./test/features/inheritedVariables.md)\n- [`rest function arguments`](./test/features/spreadExpression.md)\n- [`spread`](./test/features/spreadExpression.md)\n- [`exception`](./test/features/exception.md)\n\nFor more, see feature test markdowns: [Javascript Syntax](./test/features)\n\n### Core JavaScript API\n\n- parseInt **只接收一个参数**\n- parseFloat\n- encodeURIComponent\n- decodeURIComponent\n- encodeURI\n- __dirname\n- __filename\n- Date\n - Date.now\n - Date.prototype.getTime\n - Date.prototype.getDate\n - Date.prototype.getDay\n - Date.prototype.getFullYear\n - Date.prototype.getHours\n - Date.prototype.getMinutes\n - Date.prototype.getMonth\n - Date.prototype.getSeconds\n - Date.prototype.setDate\n - Date.prototype.setFullYear\n - Date.prototype.setHours\n - Date.prototype.setMinutes\n - Date.prototype.setMonth\n - Date.prototype.setSeconds\n - Date.prototype.setTime\n- Object\n - Object.assign\n - Object.keys\n - Object.values\n - Object.freeze\n - Object.prototype.hasOwnProperty\n- JSON\n - JSON.stringify **只接收一个参数**\n - JSON.parse **只接收一个参数**\n- console\n - console.log\n - console.info **转成 var_dump**\n - console.error\n- String\n - String.prototype.replace **第二个参数只支持 string,不支持 Function**\n - String.prototype.trim\n - String.prototype.trimRight\n - String.prototype.trimLeft\n - String.prototype.toUpperCase\n - String.prototype.toLowerCase\n - String.prototype.split\n - String.prototype.indexOf\n - String.prototype.substring\n - String.prototype.repeat\n - String.prototype.startsWidth\n - String.prototype.endsWidth\n - String.prototype.includes\n - String.prototype.padStart\n - String.prototype.match **只支持正则和字符串匹配**\n- Array\n - Array.isArray\n - Array.prototype.length\n - Array.prototype.filter **回调函数只接收第一个参数**\n - Array.prototype.push\n - Array.prototype.pop\n - Array.prototype.shift\n - Array.prototype.unshift\n - Array.prototype.concat\n - Array.prototype.reverse\n - Array.prototype.splice\n - Array.prototype.reverse\n - Array.prototype.map\n - Array.prototype.forEach\n - Array.prototype.indexOf\n - Array.prototype.join\n - Array.prototype.some\n - Array.prototype.every\n - Array.prototype.find\n - Array.prototype.findIndex\n - Array.prototype.sort\n- Number\n - Number.isInterger\n - Number.prototype.toFixed\n- Math\n - Math.abs\n - Math.acos\n - Math.acosh\n - Math.asin\n - Math.asinh\n - Math.atan\n - Math.atanh\n - Math.atan2\n - Math.cbrt\n - Math.ceil\n - Math.clz32\n - Math.cos\n - Math.cosh\n - Math.exp\n - Math.expm1\n - Math.floor\n - Math.hypot\n - Math.log\n - Math.log1p\n - Math.log10\n - Math.max\n - Math.min\n - Math.pow\n - Math.random\n - Math.round\n - Math.sin\n - Math.sinh\n - Math.sqrt\n - Math.tan\n - Math.tanh\n\n## Thanks to\n\nBased on [Typescript](https://github.com/Microsoft/TypeScript) compiler\n\n[options]: https://searchfe.github.io/ts2php/interfaces/ts2phpoptions.html\n[ts2php-diff-checker]: https://github.com/meixg/ts2php-diff-checker\n", + "readmeFilename": "README.md", + "homepage": "https://github.com/searchfe/ts2php#readme", + "contributors": [ + { + "name": "cxtom", + "email": "cxtom2008@gmail.com" + } + ], + "bugs": { + "url": "https://github.com/searchfe/ts2php/issues" + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/unpublished.json b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/unpublished.json new file mode 100644 index 00000000..d6d33f83 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/resources/test-data/unpublished.json @@ -0,0 +1,37 @@ +{ + "_id": "@lizheng11/t1", + "_rev": "19-a7bf0b8b42b6b157b2b0f490a7222227", + "name": "@lizheng11/t1", + "time": { + "0.0.1": "2020-01-06T12:47:16.883Z", + "0.0.2": "2020-01-06T12:47:16.883Z", + "0.0.6": "2020-01-06T12:47:16.883Z", + "created": "2020-01-06T12:52:56.224Z", + "0.0.9": "2020-01-06T12:52:56.512Z", + "modified": "2020-01-06T13:35:24.946Z", + "0.0.10": "2020-01-06T13:00:07.387Z", + "unpublished": { + "name": "standlee", + "time": "2020-01-06T13:07:00.143Z", + "tags": { + "latest": "0.0.10" + }, + "versions": [ + "0.0.10", + "0.0.9" + ], + "maintainers": [ + { + "name": "standlee", + "email": "lizhengnacl@gmail.com" + } + ] + } + }, + "maintainers": [ + { + "email": "lizhengnacl@gmail.com", + "name": "standlee" + } + ] +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSourceTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSourceTest.scala new file mode 100644 index 00000000..ec7b04be --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/operators/NpmReleasesSourceTest.scala @@ -0,0 +1,75 @@ +package org.codefeedr.plugins.npm.operators + +import java.util.Calendar +import org.scalatest.FunSuite + +/** + * Class to test org.tudelft.operators.NpmReleasesSource, + * the class that turns an input stream of NPM packages into a + * Datastream of Protocol information for that specific type of package + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +class NpmReleasesSourceTest extends FunSuite { + + // variables used for (simulation in) testing + val npmReleasesSource = new NpmReleasesSource(NpmSourceConfig(500, -1, 16)) + val mockedString = """["amplify-codegen-appsync-model-plugin","amplify-category-hosting","amplify-frontend-android","amplify-graphql-types-generator","amplify-frontend-javascript","amplify-app","graphql-mapping-template","amplify-frontend-ios","parser-factory","@ngasemjs/server-core","@mrf3/modularize","augmentor","midi-json-parser","eslint-config-posva","els-addon-typed-templates","validator.fn","yoshi","yoshi-template-intro","yoshi-server","yoshi-server-tools","yoshi-server-testing","yoshi-server-react","yoshi-server-client","yoshi-helpers","@vlzh/nest-typeorm-i18n","yoshi-flow-monorepo","@themost/test","yoshi-flow-legacy","yoshi-flow-editor","yoshi-flow-app","yoshi-config","yoshi-common","@let.sh/cli","testcontainers","lemon-core","eslint-config-rapida","quill-blot-formatter-extended","midi-json-parser-broker","@phnq/platform","doomisoft-controller","thirty","@applitools/visual-grid-client","@code-engine/cli","swr","@nkjmsss/bend_editor","broker-factory","skm-lit","@daostack/infra","dynamodb-recs","hydra-validator-e2e","hydra-validator","hydra-validator-analyse","hydra-validator-core","@seaneiros/react-bem","homebridge-rademacher-homepilot","jslint-configs","json-midi-encoder","library-lyj","pubs","muxu-server","postcss-lowercase-text","pimatic-smartmeter-stats","@slarys/ufe-models","json-midi-encoder-broker","react-product-fruits","eslint-config-hardcore","v-comment","json-midi-encoder-worker","react-iaux","spiel-engine","@rajzik/config-babel","@rajzik/lumos","@rajzik/config-webpack","@rajzik/config-jest","@rajzik/config-prettier","@rajzik/config-eslint","@rajzik/config-typescript","@rajzik/lumos-common","@rajzik/config-danger","@infoods/tagnames","@barba/prefetch","@angular-plugins/npm","mega-nice-form","@barba/router","jupyter-offlinenotebook","@barba/css","node-calls-python","@barba/core","json-midi-message-encoder","@ehmicky/dev-tasks","djsz3y","brain-games-liza","tk-one","react-auth-form","@ehmicky/eslint-config","@sigma-db/napi","mapir-react-component","fast-case","react-modern-calendar-datepicker","bean-parser","midi-json-parser-worker","http-proxy-tool","vue-contextmenujs","@goldfishjs/composition-api","@goldfishjs/core","@goldfishjs/plugins","@goldfishjs/route","@goldfishjs/requester","@goldfishjs/reactive","@goldfishjs/pre-build","@goldfishjs/bridge","grandjs","petri-specs","worker-factory","@azure/service-bus","app-response","@abhi18av/ramdafamily","vue-command","ngrx-signalr-core","@azure/identity","@azure/app-configuration","compilerr","vue-tournament-bracket","fe-boltzmann","jvdx","okumura-api.js","@minta/minta-error-handler","bargeron-cloverui-react","eslint-config-tidy-react","ra-data-firestore-client-ada-u","ngrx-signalr","bo-transpile-bo","verovio","@pauliescanlon/gatsby-remark-sticky-table","netmodular-ui","node-red-contrib-modbus","fantasy-content-generator","fast-unique-numbers","antd-form-mate","taupdmap","neat-rich-editor","@typetron/framework","zigmium","@enigmatis/polaris-logs","no-scroll-chains","@azure/eventhubs-checkpointstore-blob","@azure/event-hubs","@yuicer/vuepress-theme-yuicer","@yuicer/vuepress-plugin-sorted-pages","@azure/keyvault-secrets","@azure/keyvault-keys","cordova-plugin-x-socialsharing","@azure/keyvault-certificates","@aomi/wbs","ss-react","@azure/cosmos","json-schema-typed","@azure/storage-queue","@azure/storage-file-datalake","@azure/storage-file-share","@azure/storage-blob","lesh-test-module","publish-svelte","persistent-programming-redis-state","authoring-library-example","@yarnaimo/next-config","vue-autocompletion","jstransformer-lowlight","@vandmo/fake-smtp-server","react-native-dynamic-bundle-loader","@lxjx/react-render-api","@cafebazaar/hod","react-draft-wysiwyg","@yaffle/expression","webpanel-admin","typed-firestore","@gamiphy/service-core","pkg","json-form-data","node-cache","tickplate","infinidesk-portal-lib","node-typescript-template","laravel-vue-form-validator","party-parrots","draftjs-utils","lior-ehrlich-casino-clover-ui","cloudwatch-front-logger","clover-ui-yael","@vitaba/common-ui","lve","@thalesrc/hermes","@binary-constructions/semantic-map","whfp-motion-webcam","@omni-door/cli","eunice","@ant-design/codemod-v4","@react-ui-org/react-ui","colineteam-binarystream","@travi/scaffolder-sub-command","timing-provider"]""" + + test("retrieving the list from the updatestream works correctly ") { + // Act + val jsonString = npmReleasesSource.retrieveUpdateStringFrom(npmReleasesSource.url_updatestream) + // Assert + assert(jsonString.get.isInstanceOf[String]) + } + +// time consuming + test("retrieve from incorrect url will fail") { + // Arrange + val jsonString = npmReleasesSource.retrieveUpdateStringFrom("http://www.idontexisturl.com") + // Assert + assert(jsonString.isEmpty) + } + + // now for other test I'll use the mockedString + test("parsing a string with a normal list of packages works correctly") { + // Arrange + val now = Calendar.getInstance.getTime + // Act + val listReleases = npmReleasesSource.createListOfUpdatedNpmIdsFrom(mockedString, now) + // Assert + assert(listReleases.size == 201) + assert(listReleases.head.name == "amplify-codegen-appsync-model-plugin") + } + + test(" parsing an empty string will result in an empty list of Npm releases") { + // Arrange + val emptyString = "" + val now = Calendar.getInstance.getTime + // Act + val listReleases = npmReleasesSource.createListOfUpdatedNpmIdsFrom(emptyString, now) + // Assert + assert(listReleases.isEmpty) + assert(listReleases == List()) + } + + test("waiting for a PollingInterval takes equal to or more than 1/2 second") { + val startTime = System.currentTimeMillis() + npmReleasesSource.waitPollingInterval() + npmReleasesSource.waitPollingInterval() + val endTime = System.currentTimeMillis() + assert(endTime - startTime >= 2 * 500) + } + + test("default values for the source config should be set") { + val config = NpmSourceConfig() + assert(config.pollingInterval == 10000) + assert(config.maxNumberOfRuns == -1) + } + + test("boilerplate test") { + npmReleasesSource.cancel() + assert(!npmReleasesSource.getIsRunning) + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/protocol/ProtocolTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/protocol/ProtocolTest.scala new file mode 100644 index 00000000..b9b63b25 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/protocol/ProtocolTest.scala @@ -0,0 +1,368 @@ +package org.codefeedr.plugins.npm.protocol + +import java.util.Date + +import org.scalatest.FunSuite +import org.codefeedr.plugins.npm.protocol.Protocol.{Bug, BugPojo, Dependency, DependencyPojo, DependencyPojoExt, NpmProject, NpmProjectPojo, NpmRelease, NpmReleaseExt, NpmReleaseExtPojo, NpmReleasePojo, PersonObject, PersonObjectPojo, PersonObjectPojoExt, Repository, RepositoryPojo, RepositoryPojoExt, TimeObject, TimePojo, TimePojoExt} + +/** + * Class to test the creation of POJO for our SQL Service (since the Datastream[NPM Case Class] will not work + * with field referencing. + * + * Some initial variables are declared and then each conversion method to convert a NPM Case Class into its relevant + * NPM POJO is tested + * + * @author Roald van der Heijden + * Date: 2019 - 12 - 19 (YYYY-MM-DD) + */ +class ProtocolTest extends FunSuite { + + // variables to test the creation of POJOs from their relevant case class counterparts + val timeobj: TimeObject = TimeObject("2019-02-19T06:00:04.974Z", Some("2019-12-13T07:51:00.925Z")) + val timeEmptyobj: TimeObject = TimeObject("2019-02-19T06:00:04.974Z", None) + val bugobj0: Bug = Bug(Some("https:/github.com/nonexistentuser/projectname/issues"), Some("someUser@someDomain.com")) + val bugobj1: Bug = Bug(Some("https://github.com/searchfe/ts2php/issues"), None) + val bugobj2: Bug = Bug(None, Some("nospam@nourl.com")) + val emptybugobj: Bug = Bug(None, None) + val repoobj: Repository = Repository("git", "git+https://github.com/searchfe/ts2php.git", None) + val emptyrepoobj: Repository = Repository("", "", None) + val simplepersonobj: Option[String] = Some("Barney Rubble (http://barnyrubble.tumblr.com/)") + val personobj: PersonObject = PersonObject("cxtom", Some("cxtom2010@gmail.com"), None) + val emptypersonobj: PersonObject = PersonObject("", None, None) + val dependencyobj: Dependency = Dependency("semver", "^6.2.0") + val bigProject: NpmProject = NpmProject("ts2php", Some("82-79c18b748261d1370bd45e0efa753721"), "ts2php", None, + Some(List(PersonObject("cxtom", Some("cxtom2008@gmail.com"), None))), Some("TypeScript to PHP Transpiler"), Some("https://github.com/searchfe/ts2php#readme"), None, Some("MIT"), + Some(List(Dependency("fs-extra", "^7.0.1"), Dependency("lodash", "^4.17.14"), Dependency("semver", "^6.2.0"))), List(PersonObject("cxtom", Some("cxtom2010@gmail.com"), None), PersonObject("meixg", Some("meixg@foxmail.com"), None)), + "some story on how this project came to be", + "indication where to find the above line", Some(Bug(Some("https://github.com/searchfe/ts2php/issues"), None)), + None, Some(Repository("git", "git+https://github.com/searchfe/ts2php.git", None)), TimeObject("2019-02-19T06:00:04.974Z", Some("2019-12-13T07:51:00.925Z")) + ) + val now = new Date(0) + val npmrel: NpmRelease = NpmRelease("ts2php", now) + val npmrele: NpmReleaseExt = NpmReleaseExt("ts2php", now, bigProject) + + val bigProject2: NpmProject = NpmProject("ts2php", Some("82-79c18b748261d1370bd45e0efa753721"), "ts2php", Some(personobj), // cxtom version + Some(List(PersonObject("cxtom", Some("cxtom2008@gmail.com"), None))), Some("TypeScript to PHP Transpiler"), Some("https://github.com/searchfe/ts2php#readme"), + Some(List("testing", "fullcoverage")), + Some("MIT"), + Some(List(Dependency("fs-extra", "^7.0.1"), Dependency("lodash", "^4.17.14"), Dependency("semver", "^6.2.0"))), List(PersonObject("cxtom", Some("cxtom2010@gmail.com"), None), PersonObject("meixg", Some("meixg@foxmail.com"), None)), + "some story on how this project came to be", + "indication where to find the above line", Some(Bug(Some("https://github.com/searchfe/ts2php/issues"), None)), + None, Some(Repository("git", "git+https://github.com/searchfe/ts2php.git", None)), TimeObject("2019-02-19T06:00:04.974Z", Some("2019-12-13T07:51:00.925Z")) + ) + + + // Start of tests + + + test("POJO Test - NpmRelease POJO creation") { + val pojo = NpmReleasePojo.fromNpmRelease(npmrel) + // Assert + assert(pojo.name == "ts2php") + assert(pojo.retrieveDate == 0) + } + + test("POJO Test - NpmReleaseExt POJO creation") { + val result = NpmReleaseExtPojo.fromNpmReleaseExt(npmrele) + // Assert + assert(result.project.name == "ts2php") + assert(result.retrieveDate == 0) + assert(result.project.name == "ts2php") + assert(result.project._id == "ts2php") + assert(result.project._rev == "82-79c18b748261d1370bd45e0efa753721") + assert(result.project.name == "ts2php") + assert(result.project.author == null) + assert(result.project.bugString == null) + assert(result.project.readme == "some story on how this project came to be") + assert(result.project.readmeFilename == "indication where to find the above line") + assert(result.project.contributors.head.email == "cxtom2008@gmail.com") + assert(result.project.dependencies.head.packageName == "fs-extra") + assert(result.project.dependencies.last.packageName == "semver") + assert(result.project.license == "MIT") + assert(result.project.maintainers.head.name == "cxtom") + assert(result.project.maintainers.last.name == "meixg") + assert(result.project.description == "TypeScript to PHP Transpiler") + assert(result.project.homepage == "https://github.com/searchfe/ts2php#readme") + assert(result.project.keywords == null) + assert(result.project.bugs.url == "https://github.com/searchfe/ts2php/issues") + assert(result.project.bugString == null) + assert(result.project.repository.url == "git+https://github.com/searchfe/ts2php.git") + assert(result.project.time.modified == "2019-12-13T07:51:00.925Z") + } + + test("POJO Test - NpmReleaseExt POJO creation - alternative paths") { + val extendedRelease = NpmReleaseExt("ts2php", new Date(1), bigProject2) + val result = NpmReleaseExtPojo.fromNpmReleaseExt(extendedRelease) + // Assert + assert(result.project.name == "ts2php") + assert(result.retrieveDate == 1) + assert(result.project.name == "ts2php") + assert(result.project._id == "ts2php") + assert(result.project._rev == "82-79c18b748261d1370bd45e0efa753721") + assert(result.project.name == "ts2php") + assert(result.project.author.name == "cxtom") + assert(result.project.author.email == "cxtom2010@gmail.com") + assert(result.project.author.url == null) + assert(result.project.bugString == null) + assert(result.project.readme == "some story on how this project came to be") + assert(result.project.readmeFilename == "indication where to find the above line") + assert(result.project.contributors.head.email == "cxtom2008@gmail.com") + assert(result.project.dependencies.head.packageName == "fs-extra") + assert(result.project.dependencies.last.packageName == "semver") + assert(result.project.license == "MIT") + assert(result.project.maintainers.head.name == "cxtom") + assert(result.project.maintainers.last.name == "meixg") + assert(result.project.description == "TypeScript to PHP Transpiler") + assert(result.project.homepage == "https://github.com/searchfe/ts2php#readme") + assert(result.project.keywords.size == 2) + assert(result.project.keywords.head.keyword == "testing") + assert(result.project.keywords.last.keyword == "fullcoverage") + assert(result.project.bugs.url == "https://github.com/searchfe/ts2php/issues") + assert(result.project.bugString == null) + assert(result.project.repository.url == "git+https://github.com/searchfe/ts2php.git") + assert(result.project.time.modified == "2019-12-13T07:51:00.925Z") + } + + test("POJO Test - NpmProject POJO creation") { + val result = NpmProjectPojo.fromNpmProject(bigProject) + // Assert + assert(result._id == "ts2php") + assert(result._rev == "82-79c18b748261d1370bd45e0efa753721") + assert(result.name == "ts2php") + assert(result.author == null) + assert(result.author == null) + assert(result.bugString == null) + assert(result.readme == "some story on how this project came to be") + assert(result.readmeFilename == "indication where to find the above line") + assert(result.contributors.head.email == "cxtom2008@gmail.com") + assert(result.dependencies.head.packageName == "fs-extra") + assert(result.dependencies.last.packageName == "semver") + assert(result.license == "MIT") + assert(result.maintainers.head.name == "cxtom") + assert(result.maintainers.last.name == "meixg") + assert(result.description == "TypeScript to PHP Transpiler") + assert(result.homepage == "https://github.com/searchfe/ts2php#readme") + assert(result.keywords == null) + assert(result.bugs.url == "https://github.com/searchfe/ts2php/issues") + assert(result.bugString == null) + assert(result.repository.url == "git+https://github.com/searchfe/ts2php.git") + assert(result.time.modified == "2019-12-13T07:51:00.925Z") + } + + test("POJO Test - Npmproject Pojo creation - alternative paths") { + // Arrange + val alternativePathProject = Protocol.NpmProject("project_id", Some("_rev0.1"), "Harald", None, None, None, None, None, None, None, + List(PersonObject("Roald", Some("roaldheijden@nospam.com"), Some("https://github.com/roaldvanderheijden"))), + "readme: this is a short readme", + "this is the link to the readme filename", None, None, None, TimeObject("2020-01-12T07:51:00.925Z", Some("2020-01-13T00:33:00.925Z"))) + // Act + val result = NpmProjectPojo.fromNpmProject(alternativePathProject) + // assert + assert(result._id == "project_id") + assert(result._rev == "_rev0.1") + assert(result.name == "Harald") + assert(result.author == null) + assert(result.contributors == null) + assert(result.description == null) + assert(result.homepage == null) + assert(result.keywords == null) + assert(result.license == null) + assert(result.dependencies == null) + assert(result.maintainers.size == 1) + assert(result.maintainers.head.name == "Roald") + assert(result.maintainers.head.email == "roaldheijden@nospam.com") + assert(result.maintainers.head.url == "https://github.com/roaldvanderheijden") + assert(result.readme == "readme: this is a short readme") + assert(result.readmeFilename == "this is the link to the readme filename") + assert(result.contributors == null) + assert(result.bugs == null) + assert(result.bugString == null) + assert(result.repository == null) + assert(result.time.created == "2020-01-12T07:51:00.925Z") + assert(result.time.modified == "2020-01-13T00:33:00.925Z") + } + + test("POJO Test - Dependency POJO creation") { + val result = DependencyPojo.fromDependency(dependencyobj) + // Assert + assert(result.packageName == "semver") + assert(result.version == "^6.2.0") + } + + test("POJO Test - Person POJO creation") { + val result = PersonObjectPojo.fromPersonObject(personobj) + // Assert + assert(result.name == "cxtom") + assert(result.email == "cxtom2010@gmail.com") + assert(result.url == null) + } + + test("POJO Test - empty Person POJO creation") { + val result = PersonObjectPojo.fromPersonObject(emptypersonobj) + // Assert + assert(result.name == "") + assert(result.email == null) + assert(result.url == null) + } + + test("POJO Test - partially filled repository POJO creation") { + val result = RepositoryPojo.fromRepository(repoobj) + // Assert + assert(result.`type` == "git") + assert(result.url == "git+https://github.com/searchfe/ts2php.git") + assert(result.directory == null) + } + + test("POJO Test - empty repository POJO creation") { + val result = RepositoryPojo.fromRepository(emptyrepoobj) + // Assert + assert(result.`type` == "") + assert(result.url == "") + assert(result.directory == null) + } + + test("POJO Test - fully filled BugObject Pojo creation") { + val result = BugPojo.fromBug(bugobj0) + // Assert + assert(result.url == "https:/github.com/nonexistentuser/projectname/issues") + assert(result.email == "someUser@someDomain.com") + } + + test("POJO Test - partially filled BugObject POJO creation") { + val result1 = BugPojo.fromBug(bugobj1) + val result2 = BugPojo.fromBug(bugobj2) + // Assert + assert(result1.url == "https://github.com/searchfe/ts2php/issues") + assert(result1.email == null) + + assert(result2.url == null) + assert(result2.email == "nospam@nourl.com") + } + + test("POJO Test - empty BugObject POJO creation") { + val result = BugPojo.fromBug(emptybugobj) + // Assert + assert(result.url == null) + assert(result.email == null) + } + + test("POJO Test - filled TimeObject POJO creation") { + val result = TimePojo.fromTime(timeobj) + // Assert + assert(result.created == "2019-02-19T06:00:04.974Z") + assert(result.modified == "2019-12-13T07:51:00.925Z") + } + + test("POJO Test - empty TimeObject POJO creation") { + val result = TimePojo.fromTime(timeEmptyobj) + // Assert + assert(result.created == "2019-02-19T06:00:04.974Z") + assert(result.modified == null) + } + + test("POJO Test - PersonObjectPojoExt creation") { + val result = new PersonObjectPojoExt() + result.id = "ts2php" + result.name = "Roald" + result.email = "roaldheijden@nospam.com" + result.url = "https://github.com" + assert(result.id == "ts2php") + assert(result.name == "Roald") + assert(result.email == "roaldheijden@nospam.com") + assert(result.url == "https://github.com") + } + + test("POJO TEST - TimePojoExt creation") { + val result = new TimePojoExt() + result.id = "bslet" + result.created = "2020-01-12" + result.modified = "2020-01-13" + assert(result.id == "bslet") + assert(result.created == "2020-01-12") + assert(result.modified == "2020-01-13") + } + + test("POJO Test - DependencyPojoExt creation") { + val result = new DependencyPojoExt() + result.id = "upload.js" + result.packageName = "semver" + result.version = "6.0.3" + assert(result.id == "upload.js") + assert(result.packageName == "semver") + assert(result.version == "6.0.3") + } + + test("POJO Test - RepositoryPojoExt creation") { + val result = new RepositoryPojoExt() + result.id = "root" + result.`type` = "git" + result.url = "https://github.com/" + result.directory = "roaldvanderheijden" + assert(result.id == "root") + assert(result.`type` == "git") + assert(result.url == "https://github.com/") + assert(result.directory == "roaldvanderheijden") + } + + // Boilerplate tests (?) in an attempt to reach 100% coverage + + test("Unapply Test - NpmRelease case class") { + assert(NpmRelease.unapply(npmrel).get == ("ts2php", new Date(0))) + } + + test("Unapply Test - NpmReleaseExt case class") { + val myExtendedRelease = NpmReleaseExt("someName", new Date(3), bigProject) + // Assert + assert(NpmReleaseExt.unapply(myExtendedRelease).get == ("someName", new Date(3), bigProject)) + } + + test("Unapply Test - NpmProject case class") { + assert(NpmProject.unapply(bigProject2).get == + ( + "ts2php", + Some("82-79c18b748261d1370bd45e0efa753721"), + "ts2php", + Some(personobj), // cxtom version + Some(List(PersonObject("cxtom", Some("cxtom2008@gmail.com"), None))), + Some("TypeScript to PHP Transpiler"), + Some("https://github.com/searchfe/ts2php#readme"), + Some(List("testing", "fullcoverage")), + Some("MIT"), + Some(List(Dependency("fs-extra", "^7.0.1"), Dependency("lodash", "^4.17.14"), Dependency("semver", "^6.2.0"))), + List(PersonObject("cxtom", Some("cxtom2010@gmail.com"), None), PersonObject("meixg", Some("meixg@foxmail.com"), None)), + "some story on how this project came to be", + "indication where to find the above line", + Some(Bug(Some("https://github.com/searchfe/ts2php/issues"), None)), + None, + Some(Repository("git", "git+https://github.com/searchfe/ts2php.git", None)), + TimeObject("2019-02-19T06:00:04.974Z", Some("2019-12-13T07:51:00.925Z")) + ) + ) + } + + test("Unapply Test - Dependency case class") { + val depo = Dependency("somepackagename", "0.0.1") + assert(Dependency.unapply(depo).get == (("somepackagename", "0.0.1"))) + } + + test("Unapply Test - PersonObject case class") { + assert(PersonObject.unapply(personobj).get == ("cxtom", Some("cxtom2010@gmail.com"), None)) + } + + test("Unapply test - Repository case class") { + assert(Repository.unapply(repoobj).get == ("git", "git+https://github.com/searchfe/ts2php.git", None)) + } + + test("Unapply Test - Bug case class") { + val fullBug = Bug(Some("somewebpage.com/issues"), Some("user@domain.com")) + assert(Bug.unapply(fullBug).get == (Some("somewebpage.com/issues"), Some("user@domain.com"))) + } + + test("Unapply Test - time case class") { + assert(TimeObject.unapply(timeobj).get == ("2019-02-19T06:00:04.974Z", Some("2019-12-13T07:51:00.925Z"))) + } + + test("A basic or obscure test - that's the question - Testing toString & hashcode on Object Protocol") { + assert(Protocol.toString() == "Protocol companion object") + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStageTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStageTest.scala new file mode 100644 index 00000000..c466dd5f --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStageTest.scala @@ -0,0 +1,83 @@ +package org.codefeedr.plugins.npm.stages + +import java.text.SimpleDateFormat +import java.util +import java.util.{Calendar, Date} + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.scalatest.FunSuite +import org.codefeedr.plugins.npm.protocol.Protocol.{NpmRelease, NpmReleaseExt} + +/** + * Integration tests for ReleaseExtStage + * + * Adapted from W.R. Zonnveld's Maven plugin tests + * + * @author Roald van der Heijden + * Date: 2019-12-01 + */ +class NpmReleasesExtStageTest extends FunSuite { + + test("NpmReleasesExtStage integration empty project test"){ + val now = Calendar.getInstance().getTime + val sdf = new SimpleDateFormat("E, dd MMM yyyy HH:mm:ss Z") + sdf.format(now) + + val release = NpmRelease("nonexistingpackageRoald", now) + implicit val typeInfo: TypeInformation[NpmRelease] = TypeInformation.of(classOf[NpmRelease]) + + new PipelineBuilder() + .appendSource(x => x.fromCollection(List(release))(typeInfo)) + .append(new NpmReleasesExtStage()) + .append { x: DataStream[NpmReleaseExt] => + x.addSink(new CollectReleases) + } + .build() + .startMock() + + assert(CollectReleases.result.size() == 0) + } + + test("NpmReleasesExtStage integration test"){ + val now = Calendar.getInstance().getTime + val sdf = new SimpleDateFormat("E, dd MMM yyyy HH:mm:ss Z") + sdf.format(now) + + val release = NpmRelease("@microservices/cli", now) + implicit val typeInfo: TypeInformation[NpmRelease] = TypeInformation.of(classOf[NpmRelease]) + + new PipelineBuilder() + .appendSource(x => x.fromCollection(List(release))(typeInfo)) + .append(new NpmReleasesExtStage()) + .append { x: DataStream[NpmReleaseExt] => + x.addSink(new CollectReleases) + } + .build() + .startMock() + + assert(CollectReleases.result.size() == 1) + } + +} + +/** + * Adapted from W.R. Zonnveld's Maven plugin tests + * Date: 2019-12-01 + */ +object CollectReleases { + val result = new util.ArrayList[NpmReleaseExt]() +} + +/** + * Adapted from W.R. Zonnveld's Maven plugin tests + * Date: 2019-12-01 + */ +class CollectReleases extends SinkFunction[NpmReleaseExt] { + override def invoke(value: NpmReleaseExt, + context: SinkFunction.Context[_]): Unit = { + CollectReleases.result.add(value) + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStageTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStageTest.scala new file mode 100644 index 00000000..09443021 --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStageTest.scala @@ -0,0 +1,67 @@ +package org.codefeedr.plugins.npm.stages + +import org.apache.flink.streaming.api.functions.sink.SinkFunction +import org.apache.flink.streaming.api.scala.DataStream +import org.codefeedr.pipeline.PipelineBuilder +import org.codefeedr.stages.OutputStage +import org.scalatest.FunSuite +import org.codefeedr.plugins.npm.operators.NpmSourceConfig +import org.codefeedr.plugins.npm.protocol.Protocol.NpmRelease + +/** + * Class to test NpmReleasesStage class + * + * @author Roald van der Heijden + * Date: 2019-12-01 (YYYY-MM-DD) + */ +class NpmReleasesStageTest extends FunSuite { + test("NpmReleasesIntegrationTest") { + val source = new NpmReleasesStage(sourceConfig = NpmSourceConfig(1000,12,8)) + val sink = new LimitingSinkStage(4) + + val pipeline: Unit = new PipelineBuilder() + .append(source) + .append(sink) + .build() + .startMock() + } +} + +/** + * Simple Sink Pipeline Object that limits the output to a certain number + * and is able to get a list of all the items that were received in the sink + * Adapted from: W.R. Zonneveld's Maven plugin for CodeFeedr + * + * @author Roald van der Heijden + * Date: 2019-12-01 + */ +class LimitingSinkStage(elements: Int = -1) + extends OutputStage[NpmRelease] + with Serializable { + var sink: LimitingSink = _ + + override def main(source: DataStream[NpmRelease]): Unit = { + sink = new LimitingSink(elements) + source.addSink(sink).setParallelism(1) + } +} + +/** + * Adapted from: W.R. Zonneveld's Maven plugin for CodeFeedr + */ +class LimitingSink(elements: Int) extends SinkFunction[NpmRelease] { + var count = 0 + var items: List[NpmRelease] = List() + + override def invoke(value: NpmRelease, + context: SinkFunction.Context[_]): Unit = { + count += 1 + items = value :: items + + //println(count) // annoyed by this during testing, commented out + + if (elements != -1 && count >= elements) { + throw new RuntimeException() + } + } +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/AllfieldTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/AllfieldTest.scala new file mode 100644 index 00000000..3b7cd4de --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/AllfieldTest.scala @@ -0,0 +1,85 @@ +package org.codefeedr.plugins.npm.util + +import org.scalatest.FunSuite +import org.codefeedr.plugins.npm.protocol.Protocol.{Bug, PersonObject, Repository, TimeObject} +import scala.io.Source + +class AllfieldTest extends FunSuite { + test("convertProjectFrom(\"ts2php\" - All fields") { + // Arrange + val source = Source.fromFile(getClass.getResource("/test-data/ts2php.json").getPath) + + val jsonString = source.getLines().mkString + // Act + val result = NpmService.convertProjectFrom(jsonString).get + // Assert + assert(result._id == "ts2php") + assert(result._rev.contains("84-a99a3e14f1d576d910aafb083cba8673")) + assert(result.name == "ts2php") + assert(result.author.get.name == "meixuguang") + assert(result.author.get.email.isEmpty) + assert(result.author.get.url.isEmpty) + assert(result.contributors.get.size == 1) + assert(result.contributors.get.head.name == "cxtom") + assert(result.contributors.get.head.email.contains("cxtom2008@gmail.com")) + assert(result.contributors.get.head.url.isEmpty) + assert(result.description.contains("TypeScript to PHP Transpiler")) + assert(result.homepage.contains("https://github.com/searchfe/ts2php#readme")) + assert(result.keywords.isEmpty) + assert(result.license.contains("MIT")) + assert(result.dependencies.isEmpty) // is intended, since this will be built up in buildExtendedReleaseUsing + assert(result.maintainers.size == 2) + assert(result.maintainers.head.name == "cxtom") + assert(result.maintainers.head.email.contains("cxtom2010@gmail.com")) + assert(result.maintainers.head.url.isEmpty) + assert(result.maintainers.last.name == "meixg") + assert(result.maintainers.last.email.contains("meixg@foxmail.com")) + assert(result.maintainers.last.url.isEmpty) + // skipped testing readme itself + assert(result.readmeFilename == "README.md") + assert(result.bugs.get.url.get=="https://github.com/searchfe/ts2php/issues") + assert(result.bugs.get.email.isEmpty) + assert(result.bugString.isEmpty) + assert(result.repository.get.`type` == "git") + assert(result.repository.get.url == "git+https://github.com/searchfe/ts2php.git") + assert(result.repository.get.directory.isEmpty) + assert(result.time.created == "2019-02-19T06:00:04.974Z") + assert(result.time.modified.contains("2019-12-23T14:52:18.893Z")) + + source.close() + } + + test("convertProjectfrom(\"fitfont\") - All fields") { + // Arrange + val source = Source.fromFile(getClass.getResource("/test-data/fitfont-haskeywords.json").getPath) + + val jsonString = source.getLines().mkString + // Act + val result = NpmService.convertProjectFrom(jsonString).get + // Assert + assert(result._id == "fitfont") + assert(result._rev.contains("9-8893e3db2492f122c3bf2293ec25d2e0")) + assert(result.name == "fitfont") + assert(result.author.get.name == "Grégoire Sage") + assert(result.author.get.email.isEmpty) + assert(result.author.get.url.isEmpty) + assert(result.contributors.isEmpty) + assert(result.description.contains("This library allows you to easily display text with custom fonts.")) + assert(result.homepage.contains("https://github.com/gregoiresage/fitfont#readme")) + assert(result.keywords.get.size == 1) // BUG + assert(result.keywords.get.head == "fitbitdev") + assert(result.license.contains("MIT")) + assert(result.dependencies.isEmpty) // is intended, since this will be built up in buildExtendedReleaseUsing + assert(result.maintainers.size == 1) + assert(result.maintainers.head == PersonObject("gregoire",Some("gregoire.sage@gmail.com"),None)) + // skipped readMe + assert(result.readmeFilename == "README.md") + assert(result.bugs.get == Bug(Some("https://github.com/gregoiresage/fitfont/issues"),None)) + assert(result.bugString.isEmpty) + assert(result.repository.get == Repository("git","git+https://github.com/gregoiresage/fitfont.git",None)) + assert(result.time == TimeObject("2018-09-09T13:24:15.691Z",Some("2020-01-08T22:19:07.288Z"))) + + source.close() + } + +} diff --git a/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/NpmServiceTest.scala b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/NpmServiceTest.scala new file mode 100644 index 00000000..04c984bb --- /dev/null +++ b/codefeedr-plugins/codefeedr-npm/src/test/scala/org/codefeedr/plugins/npm/util/NpmServiceTest.scala @@ -0,0 +1,336 @@ +package org.codefeedr.plugins.npm.util + +import java.io.File + +import org.codefeedr.plugins.npm.protocol.Protocol +import org.json4s.JsonAST.JNothing +import org.json4s.ext.JavaTimeSerializers +import org.json4s.jackson.JsonMethods.parse +import org.json4s.jackson.Serialization.read +import org.json4s.{DefaultFormats, Formats} +import org.scalatest.FunSuite + +import scala.io.Source + +/** + * Tests the NpmService + * + * @author Roald van der Heijden + * Date: 2019-12-08 (YYYY-MM-DD) + */ +class NpmServiceTest extends FunSuite { + + // variables to test conditions for getProjectRaw & convertProjectFrom + val correct_base_url = "http://registry.npmjs.com/" + val incorrect_base_url = "http://somenonexistentdomain.com" + val nonExistingProject = "roaldroaldroaldroaldroald8D" + val unPublishedProject = "/@lizheng11/t1" + val existingProject = "ts2php" + + // variables simulating certain conditions in JSON for testing dependencies extraction method + val jsonStringWithNoDistTags = """{"_id": "@bamblehorse/tiny","_rev": "2-9526bb8fdedb67c1fe82abbad9de9a4f","name": "@bamblehorse/tiny" }""" + + // variables simulating incorrect JSON (used e.g. in convertProjectFrom + val incorrectJsonString = """{"_id": "@bamblehorse/tiny","_rev": }""" + + // variables simulating certain conditions in JSON for testing time extraction method + val jsonNoTime = """{ "_id": "@roald/test" } """ + val jsonIncorrectTimeType = """{"name":"luca", "id": "1q2w3e4r5t", "time": 26, "url":"http://www.nosqlnocry.wordpress.com"}""" + val jsonEmptyTime = """{"_id": "@lizheng11/t1","_rev": "19-a7bf0b8b42b6b157b2b0f490a7222227","name": "@lizheng11/t1","time": {}}""" + val jsonTimeOk = """{"time": {"created": "2019-07-22T11:41:29.226Z","1.0.0": "2019-07-22T11:41:29.631Z","modified": "2019-07-24T12:04:56.497Z","2.0.0": "2019-07-24T12:04:52.956Z"}}""" + val jsonTimeNoCreated = """{"time": {"1.0.0": "2011-07-22T11:41:29.631Z","modified": "2011-07-24T12:04:56.497Z","2.0.0": "2011-07-24T12:04:52.956Z"}}""" + val jsonTimeNoModified = """{"time": {"created": "2001-07-22T11:41:29.226Z","1.0.0": "2001-07-22T11:41:29.631Z","2.0.0": "2001-07-24T12:04:52.956Z"}}""" + val jsonTimeBothFieldsMissing = """{"time": {"1.0.0": "1980-07-22T11:41:29.631Z","2.0.0": "1980-07-24T12:04:52.956Z"}}""" + + // variable simulating conditions in JSON for testing author extraction method + val jsonAuthorString = """{ + "author": "Barney Rubble < b @rubble.com > (http: //barnyrubble.tumblr.com/)" +}""" + + // test for withConfiguredHeaders + + + test("Configuration of headers was done correct") { + val headersList = NpmService.withConfiguredHeaders + // Assert + assert(headersList.size == 1) + assert(headersList.head._1 == "User-Agent") + assert(headersList.head._2 == "CodeFeedr-Npm/1.0 Contact: zonneveld.noordwijk@gmail.com") + } + + // tests for getProjectRaw + + test("getProjectRAW - fetching a from a NONEXISTING domain returns None") { + val result = NpmService.getProjectRaw(incorrect_base_url, nonExistingProject, NpmService.timeout) + assert(result.isEmpty) + } + + test("getProjectRAW - fetching a NONEXISTING Npm package returns a JSON ERROR string") { + val result = NpmService.getProjectRaw(correct_base_url, nonExistingProject, NpmService.timeout) + assert(result.get=="""{"error":"Not found"}""") + } + + test("getProjectRaw - fetching an UNPUBLISHED Npm package returns some JSON string") { + // Act + val result = NpmService.getProjectRaw(correct_base_url, unPublishedProject, NpmService.timeout) + val json = parse(result.get) + val unpublishedTimeField = (json \ "time") \ "unpublished" + + // Assert + // maybe too broad of an assumption? + // could also check for \ "time" and then check it's a JString ... but then would the string + // be a date format? Check again? + assert(unpublishedTimeField != JNothing) + } + + test("getProjectRAW - fetching a EXISTING Npm package returns a good JSON string") { + val optionString = NpmService.getProjectRaw(correct_base_url, existingProject, NpmService.timeout) + assert(optionString.isInstanceOf[Option[String]]) + } + + + // tests for getProject + + test("getProject - fetching an UNPUBLISHED Npm package yields None") { + val result = NpmService.getProject(unPublishedProject) + assert(result.isEmpty) + } + + test("getProject - fetching an EXISTING project works correctly") { + // Act + val result = NpmService.getProject(existingProject) + // Assert + assert(result.isDefined) + assert(result.get.project._id == "ts2php") + assert(result.get.project.license.get == "MIT") + assert(result.get.project.bugs.get.url.contains("https://github.com/searchfe/ts2php/issues")) + } + + + // tests for createJsonString + + + test("createjsonString - working ok") { + // Act 1 + val jsonString = NpmService.createJsonStringFor(correct_base_url , existingProject) + + // Assert 1 + assert(jsonString.isInstanceOf[Option[String]]) + // Act 2 + implicit val formats: Formats = new DefaultFormats {} ++ JavaTimeSerializers.all + val result = read[Protocol.NpmProject](jsonString.get) + // Assert 2 + assert(result._id == "ts2php") + assert(result.dependencies.isEmpty) // upon first reading this is Empty, we need to extract this from somehwere in the time field + assert(result.contributors.get.size==1) + } + + test("createjsonString - unpublished npm package results in None") { + // Act 1 + val jsonString = NpmService.createJsonStringFor(correct_base_url, unPublishedProject) + // Assert 1 + assert(jsonString.isEmpty) + } + + test("createJsonString - incorrectly specified npm package url results in None") { + val jsonString = NpmService.createJsonStringFor(correct_base_url, nonExistingProject) + assert(jsonString.isEmpty) + } + + + // Time extraction tests + + + test("Extracing time - with incorrectly typed subfields should return unknown/None") { + // Arrange + val json = parse(jsonIncorrectTimeType) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "unknown") + assert(result.modified.isEmpty) + } + + test("Extracting time - from a json string with NO time field returns unknown/None") { + // Arrange + val json = parse(jsonNoTime) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "unknown") + assert(result.modified.isEmpty) + } + + test("Extracting time - from a json string with no created and no modified field returns unknown/None") { + // Arrange + val json = parse(jsonTimeBothFieldsMissing) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "unknown") + assert(result.modified.isEmpty) + } + + test("Extracting time - from a json string with no created field returns unknown/Some(...)") { + // Arrange + val json = parse(jsonTimeNoCreated) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "unknown") + assert(result.modified.contains("2011-07-24T12:04:56.497Z")) + } + + test("Extracting time - from a json string with no modified field returns value/None") { + // Arrange + val json = parse(jsonTimeNoModified) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "2001-07-22T11:41:29.226Z") + assert(result.modified.isEmpty) + } + + test("Extracting time - from a json string with both created/modified present returns value/Some(...)") { + // Arrange + val json = parse(jsonTimeOk) + // Act + val result = NpmService.extractTimeFrom(json) + // Assert + assert(result.created == "2019-07-22T11:41:29.226Z") + assert(result.modified.contains("2019-07-24T12:04:56.497Z")) + } + + + // Dependencies extraction tests + + + test("Extracting dependencies - Homophone: Fire the Nun") { + // Fires the special case resulting in None in the method, just before flatMapping and returning the result + // Arrange + val file = new File(getClass.getResource("/test-data/firetheNun.json").getPath) + // Act + val json = parse(file) + val result = NpmService.extractDependenciesFrom(json) + // Assert + assert(result.isEmpty) + } + + test("Extracting dependencies - failing due to the absence of dist - tags from a JSON String fails and results in Nil") { + // Arrange + val json = parse(jsonStringWithNoDistTags) + // Act + val projectWithoutDistTags = NpmService.extractDependenciesFrom(json) + // Assert + assert(projectWithoutDistTags == Nil) + } + + test("Extracting dependencies - failing on nonexistent dependencies returns Nil") { + // Arrange + val file = new File(getClass.getResource("/test-data/tiny.json").getPath) + val json = parse(file) + // Act + val projectWithoutDependencies = NpmService.extractDependenciesFrom(json) + // Assert + assert(projectWithoutDependencies==Nil) + } + + test("Extracting dependencies - failing on existing but empty dependencies field returns Nil") { + // Arrange + val file = new File(getClass.getResource("/test-data/bslet.json").getPath) + val json = parse(file) + // Act + val projectWithoutDependencies = NpmService.extractDependenciesFrom(json) + // Assert + assert(projectWithoutDependencies==Nil) + } + + test("Extracting existing dependencies works ok") { + // Arrange + val file = new File(getClass.getResource("/test-data/ts2php.json").getPath) + val json = parse(file) + // Act + val projectWithoutDependencies: List[Protocol.Dependency] = NpmService.extractDependenciesFrom(json) + // Assert + assert(projectWithoutDependencies.size == 5) + assert(projectWithoutDependencies.head.packageName=="fs-extra") + assert(projectWithoutDependencies.last.packageName=="yargs") + } + + + // tests for author + + test("debug test author") { + val file = new File(getClass.getResource("/test-data/ts2php.json").getPath) + val json = parse(file) + // Act + val author = NpmService.extractAuthorFrom(json) + // Assert + assert(author.get.name == "meixuguang") + assert(author.get.email.isEmpty) + assert(author.get.url.isEmpty) + + } + + // test for buildrelextusing + + test("buildNpmReleaseExt - simple test works") { + // Arrange + val source = Source.fromFile(getClass.getResource("/test-data/bslet.json").getPath) + + val jsonString = source.getLines().mkString + // Act + val npmProject = NpmService.convertProjectFrom(jsonString).get + val result = NpmService.buildNpmReleaseExtUsing("bslet", jsonString, npmProject) + // Assert + assert(result.name == "bslet") + assert(result.project.maintainers.head.name == "mrmurphy") + assert(result.project.license.get == "MIT") + + source.close() + } + + // Test for convertProjectFrom + + + test("convertProjectFrom - an UNPUBLISHED npmproject json string will return None") { + // Arrange + val source = Source.fromFile(getClass.getResource("/test-data/unpublished.json").getPath) + val jsonString = source.getLines().mkString + // Act + val npmProject = NpmService.convertProjectFrom(jsonString) + // Assert + assert(npmProject.isEmpty) + + source.close() + } + + test("convertProjectFrom - Incorrect json String will fail and result in None") { + // Act + val json = NpmService.convertProjectFrom(incorrectJsonString) + // Assert + assert(json.isEmpty) + } + + test("convertProjectFrom - converting a correct json String will result in an NpmProject") { + // Arrange + val source = Source.fromFile(getClass.getResource("/test-data/bslet.json").getPath) + + val jsonString = source.getLines().mkString + // Act + val npmProject = NpmService.convertProjectFrom(jsonString) + // Assert + assert(npmProject.isDefined) + assert(npmProject.get.isInstanceOf[Protocol.NpmProject]) + assert(npmProject.get.maintainers.head.name == "mrmurphy") + + source.close() + } + + test("boilerplate test - testing toString of NpmService Companion Object") { + // Assert + assert(NpmService.toString == "NpmService Companion Object") + } + + +} \ No newline at end of file diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/operators/PyPiReleasesSource.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/operators/PyPiReleasesSource.scala index 243eeb59..e0f1196f 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/operators/PyPiReleasesSource.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/operators/PyPiReleasesSource.scala @@ -2,60 +2,25 @@ package org.codefeedr.plugins.pypi.operators import java.text.SimpleDateFormat -import org.apache.flink.api.common.accumulators.LongCounter -import org.apache.flink.api.common.state.{ListState, ListStateDescriptor} -import org.apache.flink.configuration.Configuration -import org.apache.flink.runtime.state.{ - FunctionInitializationContext, - FunctionSnapshotContext -} -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction -import org.apache.flink.streaming.api.functions.source.{ - RichSourceFunction, - SourceFunction -} +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.codefeedr.pipeline.{PluginReleasesSource, PluginSourceConfig} import org.codefeedr.plugins.pypi.protocol.Protocol.PyPiRelease import org.codefeedr.stages.utilities.{HttpRequester, RequestException} import scalaj.http.Http -import scala.collection.JavaConverters._ import scala.xml.XML case class PyPiSourceConfig(pollingInterval: Int = 1000, maxNumberOfRuns: Int = -1) + extends PluginSourceConfig class PyPiReleasesSource(config: PyPiSourceConfig = PyPiSourceConfig()) - extends RichSourceFunction[PyPiRelease] - with CheckpointedFunction { + extends PluginReleasesSource[PyPiRelease](config) { /** Format and URL of RSS Feed. */ val dateFormat = "EEE, dd MMM yyyy HH:mm:ss ZZ" val url = "https://pypi.org/rss/updates.xml" - /** Some track variables of this source. */ - private var isRunning = false - private var runsLeft = 0 - private var lastItem: Option[PyPiRelease] = None - @transient - private var checkpointedState: ListState[PyPiRelease] = _ - - def getIsRunning: Boolean = isRunning - - /** Accumulator for the amount of processed releases. */ - val releasesProcessed = new LongCounter() - - /** Opens this source. */ - override def open(parameters: Configuration): Unit = { - isRunning = true - runsLeft = config.maxNumberOfRuns - } - - /** Close the source. */ - override def cancel(): Unit = { - isRunning = false - - } - /** Runs the source. * * @param ctx the source the context. @@ -79,13 +44,9 @@ class PyPiReleasesSource(config: PyPiSourceConfig = PyPiSourceConfig()) val validSortedItems = sortAndDropDuplicates(items) validSortedItems.foreach(x => ctx.collectWithTimestamp(x, x.pubDate.getTime)) - releasesProcessed.add(validSortedItems.size) - if (validSortedItems.nonEmpty) { - lastItem = Some(validSortedItems.last) - } - // Wait until the next poll - waitPollingInterval() + // call parent run + super.runPlugin(ctx, validSortedItems) } catch { case _: Throwable => } @@ -142,51 +103,12 @@ class PyPiReleasesSource(config: PyPiSourceConfig = PyPiSourceConfig()) */ def xmlToPyPiRelease(node: scala.xml.Node): PyPiRelease = { val title = (node \ "title").text - val description = (node \ "description").text val link = (node \ "link").text + val description = (node \ "description").text val formatter = new SimpleDateFormat(dateFormat) val pubDate = formatter.parse((node \ "pubDate").text) - PyPiRelease(title, description, link, pubDate) - } - - /** - * If there is a limit to the amount of runs decrease by 1 - */ - def decreaseRunsLeft(): Unit = { - if (runsLeft > 0) { - runsLeft -= 1 - } - } - - /** - * Wait a certain amount of times the polling interval - * @param times Times the polling interval should be waited - */ - def waitPollingInterval(times: Int = 1): Unit = { - Thread.sleep(times * config.pollingInterval) - } - - /** Make a snapshot of the current state. */ - override def snapshotState(context: FunctionSnapshotContext): Unit = { - if (lastItem.isDefined) { - checkpointedState.clear() - checkpointedState.add(lastItem.get) - } - } - - /** Initializes state by reading from a checkpoint or creating an empty one. */ - override def initializeState(context: FunctionInitializationContext): Unit = { - val descriptor = - new ListStateDescriptor[PyPiRelease]("last_element", classOf[PyPiRelease]) - - checkpointedState = context.getOperatorStateStore.getListState(descriptor) - - if (context.isRestored) { - checkpointedState.get().asScala.foreach { x => - lastItem = Some(x) - } - } + PyPiRelease(title, link, description, pubDate) } } diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala index 6f1a6512..73fe44cb 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala @@ -2,6 +2,8 @@ package org.codefeedr.plugins.pypi.stages import java.util.concurrent.TimeUnit +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.api.datastream.{ AsyncDataStream => JavaAsyncDataStream @@ -11,8 +13,10 @@ import org.codefeedr.plugins.pypi.protocol.Protocol.{ PyPiReleaseExt } import org.codefeedr.stages.TransformStage -import org.apache.flink.api.scala._ import org.codefeedr.plugins.pypi.operators.RetrieveProjectAsync +import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor + +import scala.language.higherKinds /** Transform a [[PyPiRelease]] to [[PyPiReleaseExt]]. * @@ -38,4 +42,9 @@ class PyPiReleaseExtStage(stageId: String = "pypi_releases") new org.apache.flink.streaming.api.scala.DataStream(async) } + + override def getSchema: Schema = { + implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[PyPiReleaseExt] + } } diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala index f27bedef..7c16e2e1 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala @@ -1,18 +1,19 @@ package org.codefeedr.plugins.pypi.stages +import com.sksamuel.avro4s.AvroSchema +import org.apache.avro.Schema +import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.DataStream import org.codefeedr.pipeline.Context -import org.codefeedr.plugins.pypi.protocol.Protocol.PyPiRelease -import org.codefeedr.stages.InputStage -import org.apache.flink.api.scala._ -import org.apache.flink.streaming.api.TimeCharacteristic -import org.apache.flink.streaming.api.scala.function.ProcessAllWindowFunction -import org.apache.flink.streaming.api.windowing.time.Time -import org.apache.flink.util.Collector import org.codefeedr.plugins.pypi.operators.{ PyPiReleasesSource, PyPiSourceConfig } +import org.codefeedr.plugins.pypi.protocol.Protocol.PyPiRelease +import org.codefeedr.stages.InputStage +import org.codefeedr.stages.utilities.DefaultTypeMapper._ + +import scala.language.higherKinds /** Fetches real-time releases from PyPi. */ class PyPiReleasesStage(stageId: String = "pypi_releases_min", @@ -27,4 +28,9 @@ class PyPiReleasesStage(stageId: String = "pypi_releases_min", override def main(context: Context): DataStream[PyPiRelease] = context.env .addSource(new PyPiReleasesSource(sourceConfig)) + + override def getSchema: Schema = { + implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + AvroSchema[PyPiRelease] + } } diff --git a/codefeedr-plugins/codefeedr-rabbitmq/src/test/scala/org/codefeedr/plugins/rabbitmq/stages/RabbitMQInputOutputTest.scala b/codefeedr-plugins/codefeedr-rabbitmq/src/test/scala/org/codefeedr/plugins/rabbitmq/stages/RabbitMQInputOutputTest.scala index a6a550f2..d697eb0c 100644 --- a/codefeedr-plugins/codefeedr-rabbitmq/src/test/scala/org/codefeedr/plugins/rabbitmq/stages/RabbitMQInputOutputTest.scala +++ b/codefeedr-plugins/codefeedr-rabbitmq/src/test/scala/org/codefeedr/plugins/rabbitmq/stages/RabbitMQInputOutputTest.scala @@ -21,23 +21,17 @@ import java.util import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.functions.sink.SinkFunction -import org.apache.flink.streaming.api.functions.sink.SinkFunction.Context -import org.apache.flink.streaming.api.scala.{ - DataStream, - StreamExecutionEnvironment -} +import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} import org.apache.flink.streaming.connectors.rabbitmq.RMQSource import org.codefeedr.pipeline -import org.codefeedr.pipeline.Context +import org.codefeedr.plugins.rabbitmq.RMQSinkDurable import org.codefeedr.stages.utilities.StringType import org.scalatest.FunSuite import org.scalatest.mockito.MockitoSugar import org.mockito.Matchers._ import org.mockito.Mockito._ -import org.apache.flink.api.scala._ -import org.codefeedr.plugins.rabbitmq.RMQSinkDurable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ class RabbitMQInputOutputTest extends FunSuite with MockitoSugar { @@ -106,7 +100,7 @@ object RMQStringCollectSink { numEventTimes = 0 } - def asList: List[String] = result.toList + def asList: List[String] = result.asScala.toList } class RMQStringCollectSink extends SinkFunction[StringType] { diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposer.scala b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposer.scala similarity index 97% rename from codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposer.scala rename to codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposer.scala index 617fe244..b6c8f154 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposer.scala +++ b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposer.scala @@ -16,7 +16,7 @@ * limitations under the License. * */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import java.net.URI diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposer.scala b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/SchemaExposer.scala similarity index 95% rename from codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposer.scala rename to codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/SchemaExposer.scala index 89c4024b..6cda763f 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposer.scala +++ b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/SchemaExposer.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import org.apache.avro.Schema @@ -57,7 +57,7 @@ trait SchemaExposer { val schema = new Schema.Parser().parse(schemaString) Some(schema) } catch { - case x: Throwable => None + case _: Throwable => None } } } diff --git a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposer.scala b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposer.scala similarity index 86% rename from codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposer.scala rename to codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposer.scala index 63b3a6aa..2f1f7f0c 100644 --- a/codefeedr-core/src/main/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposer.scala +++ b/codefeedr-util/schema-exposure/src/main/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposer.scala @@ -16,7 +16,7 @@ * limitations under the License. * */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import org.apache.avro.Schema import org.apache.zookeeper._ @@ -40,9 +40,7 @@ class ZookeeperSchemaExposer(host: String, root: String = "/codefeedr:schemas") /** Connect with ZK server. */ private def connect(): Unit = { - client = new ZooKeeper(host, 5000, new Watcher { - override def process(event: WatchedEvent): Unit = {} - }) //we don't care about the watchevent + client = new ZooKeeper(host, 5000, (_: WatchedEvent) => {}) //we don't care about the WatchedEvent //if parent doesn't exist create it val exists = client.exists(root, false) @@ -94,7 +92,7 @@ class ZookeeperSchemaExposer(host: String, root: String = "/codefeedr:schemas") //parse the schema and return parse(new String(data)) } catch { - case x: Throwable => None //if path is not found + case _: Throwable => None //if path is not found } } @@ -107,7 +105,7 @@ class ZookeeperSchemaExposer(host: String, root: String = "/codefeedr:schemas") try { client.delete(s"$root/$subject", -1) } catch { - case x: Throwable => + case _: Throwable => return false //if path doesn't exist or there is no data } @@ -131,4 +129,20 @@ class ZookeeperSchemaExposer(host: String, root: String = "/codefeedr:schemas") //delete root afterwards client.delete(s"$root", -1) } + + /** + * Getter for all children in zookeeper path. + * @return list of all children in zookeeper path + */ + def getAllChildren: List[String] = { + val exists = client.exists(s"$root", false) + + //if not exists then return + if (exists == null) return null + + //get all children + val children = client.getChildren(s"$root", false) + + children.asScala.toList + } } diff --git a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposerTest.scala b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposerTest.scala similarity index 84% rename from codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposerTest.scala rename to codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposerTest.scala index 60196fbe..64b23d84 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/RedisSchemaExposerTest.scala +++ b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/RedisSchemaExposerTest.scala @@ -16,30 +16,30 @@ * limitations under the License. * */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import com.github.sebruck.EmbeddedRedis import org.scalatest.BeforeAndAfterAll import redis.embedded.RedisServer class RedisSchemaExposerTest - extends SchemaExposerTest + extends SchemaExposerTest with BeforeAndAfterAll with EmbeddedRedis { - var redis: RedisServer = null + var redis: RedisServer = _ var redisPort: Int = 0 // Before all tests, setup an embedded redis - override def beforeAll() = { + override def beforeAll(): Unit = { redis = startRedis() redisPort = redis.ports().get(0) } // After all tests, stop embedded redis - override def afterAll() = { + override def afterAll(): Unit = { stopRedis(redis) } - override def getSchemaExposer(): SchemaExposer = + override def getSchemaExposer: SchemaExposer = new RedisSchemaExposer(s"redis://localhost:$redisPort") } diff --git a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposerTest.scala b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/SchemaExposerTest.scala similarity index 92% rename from codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposerTest.scala rename to codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/SchemaExposerTest.scala index 5ba26002..61bb9866 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/SchemaExposerTest.scala +++ b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/SchemaExposerTest.scala @@ -16,7 +16,7 @@ * limitations under the License. * */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import org.apache.avro.Schema import org.scalatest.{BeforeAndAfter, FunSuite} @@ -59,10 +59,10 @@ abstract class SchemaExposerTest extends FunSuite with BeforeAndAfter { val subject = "testSubject" - def getSchemaExposer(): SchemaExposer + def getSchemaExposer: SchemaExposer before { - exposer = getSchemaExposer() + exposer = getSchemaExposer parsedSchema = exposer .parse(schema) @@ -118,18 +118,18 @@ abstract class SchemaExposerTest extends FunSuite with BeforeAndAfter { } test("Get a schema on a non existent subject should return None") { - assert(exposer.get("IDoNoTeXiSt") == None) + assert(exposer.get("IDoNoTeXiSt").isEmpty) } test("An invalid schema should return None") { - assert(exposer.parse("iNVaLIdScHemA{}$%:)") == None) + assert(exposer.parse("iNVaLIdScHemA{}$%:)").isEmpty) } test("All schema's should be properly deleted") { //put the schema assert(exposer.put(parsedSchema, subject)) exposer.deleteAll() - assert(exposer.get(subject) == None) + assert(exposer.get(subject).isEmpty) } test("All schema's should be properly deleted even if called twice") { @@ -137,7 +137,7 @@ abstract class SchemaExposerTest extends FunSuite with BeforeAndAfter { assert(exposer.put(parsedSchema, subject)) exposer.deleteAll() exposer.deleteAll() - assert(exposer.get(subject) == None) + assert(exposer.get(subject).isEmpty) } } diff --git a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposerTest.scala b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposerTest.scala similarity index 85% rename from codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposerTest.scala rename to codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposerTest.scala index c884e9fb..bf9b67ad 100644 --- a/codefeedr-core/src/test/scala/org/codefeedr/buffer/serialization/schema_exposure/ZookeeperSchemaExposerTest.scala +++ b/codefeedr-util/schema-exposure/src/test/scala/org/codefeedr/util/schema_exposure/ZookeeperSchemaExposerTest.scala @@ -16,18 +16,18 @@ * limitations under the License. * */ -package org.codefeedr.buffer.serialization.schema_exposure +package org.codefeedr.util.schema_exposure import net.manub.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig} import org.scalatest.BeforeAndAfterAll class ZookeeperSchemaExposerTest - extends SchemaExposerTest + extends SchemaExposerTest with EmbeddedKafka with BeforeAndAfterAll { override def beforeAll(): Unit = { - implicit val config = EmbeddedKafkaConfig(zooKeeperPort = 2181) + implicit val config: EmbeddedKafkaConfig = EmbeddedKafkaConfig(zooKeeperPort = 2181) EmbeddedKafka.start() } @@ -35,6 +35,6 @@ class ZookeeperSchemaExposerTest EmbeddedKafka.stop() } - override def getSchemaExposer(): SchemaExposer = + override def getSchemaExposer: SchemaExposer = new ZookeeperSchemaExposer("localhost:2181") } diff --git a/project/assembly.sbt b/project/assembly.sbt deleted file mode 100644 index 652a3b93..00000000 --- a/project/assembly.sbt +++ /dev/null @@ -1 +0,0 @@ -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.6") diff --git a/project/plugins.sbt b/project/plugins.sbt index 019ed63b..8ad67da8 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,3 +3,4 @@ addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.2.4") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.1.1") addSbtPlugin("com.geirsson" % "sbt-scalafmt" % "1.5.1") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.6") \ No newline at end of file From 1b030d8461cab4134251e0c395514f36aa64fb77 Mon Sep 17 00:00:00 2001 From: AbeleMM Date: Tue, 21 Jul 2020 14:48:48 +0200 Subject: [PATCH 2/3] Update Avro4s version & refactor Avro4s handling of Date --- build.sbt | 2 +- .../stages/utilities/DefaultTypeMapper.scala | 26 ------------------- .../stages/ClearlyDefinedReleasesStage.scala | 8 ++---- .../plugins/maven/protocol/Protocol.scala | 4 +++ .../maven/stages/MavenReleasesStage.scala | 8 +++--- .../plugins/npm/protocol/Protocol.scala | 7 ++++- .../npm/stages/NpmReleasesExtStage.scala | 7 ++--- .../plugins/npm/stages/NpmReleasesStage.scala | 8 +++--- .../plugins/pypi/protocol/Protocol.scala | 4 +++ .../pypi/stages/PyPiReleaseExtStage.scala | 7 ++--- .../pypi/stages/PyPiReleasesStage.scala | 8 +++--- 11 files changed, 40 insertions(+), 49 deletions(-) delete mode 100644 codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala diff --git a/build.sbt b/build.sbt index 2ad6b143..87644698 100644 --- a/build.sbt +++ b/build.sbt @@ -282,7 +282,7 @@ lazy val dependencies = //val embeddedRabbitMQ = "io.arivera.oss" %% "embedded-rabbitmq" % "1.3.0" % Test val avro = "org.apache.avro" % "avro" % "1.8.2" - val avro4s = "com.sksamuel.avro4s" %% "avro4s-core" % "3.1.0" + val avro4s = "com.sksamuel.avro4s" %% "avro4s-core" % "3.1.1" val flinkTablePlanner = "org.apache.flink" %% "flink-table-planner" % flinkVersion diff --git a/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala b/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala deleted file mode 100644 index 607babde..00000000 --- a/codefeedr-core/src/main/scala/org/codefeedr/stages/utilities/DefaultTypeMapper.scala +++ /dev/null @@ -1,26 +0,0 @@ -package org.codefeedr.stages.utilities - -import java.util.Date - -import com.sksamuel.avro4s._ -import org.apache.avro.Schema - -object DefaultTypeMapper { - - /** - * Generate an Avro Schema for java.util.Date. - * - * @param isRowtime determines if Date field is used as a rowtime attribute - */ - class DateSchemaFor(val isRowtime: Boolean = false) extends SchemaFor[Date] { - - override def schema(fieldMapper: FieldMapper): Schema = { - val sc = Schema.create(Schema.Type.STRING) - if (isRowtime) { - sc.addProp("isRowtime", isRowtime) - } - sc - } - } - -} diff --git a/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala index 04197d97..c6f00e7e 100644 --- a/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala +++ b/codefeedr-plugins/codefeedr-clearlydefined/src/main/scala/org/codefeedr/plugins/clearlydefined/stages/ClearlyDefinedReleasesStage.scala @@ -5,13 +5,12 @@ import org.apache.avro.Schema import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.DataStream import org.codefeedr.pipeline.Context -import org.codefeedr.stages.InputStage import org.codefeedr.plugins.clearlydefined.operators.{ ClearlyDefinedReleasesSource, ClearlyDefinedSourceConfig } import org.codefeedr.plugins.clearlydefined.protocol.Protocol.ClearlyDefinedRelease -import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor +import org.codefeedr.stages.InputStage import scala.language.higherKinds @@ -34,8 +33,5 @@ class ClearlyDefinedReleasesStage(stageId: String = .addSource(new ClearlyDefinedReleasesSource(sourceConfig))(typeInfo) } - override def getSchema: Schema = { - implicit val NpmSchema: DateSchemaFor = new DateSchemaFor(true) - AvroSchema[ClearlyDefinedRelease] - } + override def getSchema: Schema = AvroSchema[ClearlyDefinedRelease] } diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala index 91f416c0..7895a5db 100644 --- a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/protocol/Protocol.scala @@ -3,6 +3,8 @@ package org.codefeedr.plugins.maven.protocol import java.io.Serializable import java.util.Date +import com.sksamuel.avro4s.AvroProp + object Protocol { /** @@ -309,6 +311,7 @@ object Protocol { title: String, link: String, description: String, + @AvroProp("rowtime", "true") pubDate: Date, guid: Guid ) @@ -319,6 +322,7 @@ object Protocol { title: String, link: String, description: String, + @AvroProp("rowtime", "true") pubDate: Date, guid: Guid, project: MavenProject diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala index 67a17da9..2c6ff230 100644 --- a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesStage.scala @@ -1,6 +1,8 @@ package org.codefeedr.plugins.maven.stages -import com.sksamuel.avro4s.AvroSchema +import java.util.Date + +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} import org.apache.avro.Schema import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.DataStream @@ -11,7 +13,6 @@ import org.codefeedr.plugins.maven.operators.{ MavenReleasesSource, MavenSourceConfig } -import org.codefeedr.stages.utilities.DefaultTypeMapper._ import scala.language.higherKinds @@ -34,7 +35,8 @@ class MavenReleasesStage(stageId: String = "maven_releases_min", } override def getSchema: Schema = { - implicit val MavenSchema: DateSchemaFor = new DateSchemaFor(true) + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) AvroSchema[MavenRelease] } } diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala index 2b91e100..28e7cff4 100644 --- a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/protocol/Protocol.scala @@ -2,6 +2,8 @@ package org.codefeedr.plugins.npm.protocol import java.util.Date +import com.sksamuel.avro4s.AvroProp + /** * Contains all the case classes and POJO equivalent classes to represent a NPM package release * @@ -10,9 +12,12 @@ import java.util.Date */ object Protocol { - case class NpmRelease(name: String, retrieveDate: Date) // using ingestion time + case class NpmRelease( + name: String, + @AvroProp("rowtime", "true") retrieveDate: Date) // using ingestion time case class NpmReleaseExt(name: String, + @AvroProp("rowtime", "true") retrieveDate: Date, project: NpmProject) diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala index a78aa651..5dada7de 100644 --- a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesExtStage.scala @@ -1,8 +1,9 @@ package org.codefeedr.plugins.npm.stages +import java.util.Date import java.util.concurrent.TimeUnit -import com.sksamuel.avro4s.AvroSchema +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} import org.apache.avro.Schema import org.apache.flink.streaming.api.datastream.{ AsyncDataStream => JavaAsyncDataStream @@ -11,7 +12,6 @@ import org.apache.flink.streaming.api.scala.DataStream import org.codefeedr.plugins.npm.operators.RetrieveProjectAsync import org.codefeedr.plugins.npm.protocol.Protocol.{NpmRelease, NpmReleaseExt} import org.codefeedr.stages.TransformStage -import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor import scala.language.higherKinds @@ -54,7 +54,8 @@ class NpmReleasesExtStage(stageId: String = "npm_releases") } override def getSchema: Schema = { - implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) AvroSchema[NpmReleaseExt] } } diff --git a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala index bf4044dd..b8e566a5 100644 --- a/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala +++ b/codefeedr-plugins/codefeedr-npm/src/main/scala/org/codefeedr/plugins/npm/stages/NpmReleasesStage.scala @@ -1,6 +1,8 @@ package org.codefeedr.plugins.npm.stages -import com.sksamuel.avro4s.AvroSchema +import java.util.Date + +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} import org.apache.avro.Schema import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.scala.DataStream @@ -8,7 +10,6 @@ import org.codefeedr.pipeline.Context import org.codefeedr.stages.InputStage import org.codefeedr.plugins.npm.operators.{NpmReleasesSource, NpmSourceConfig} import org.codefeedr.plugins.npm.protocol.Protocol.NpmRelease -import org.codefeedr.stages.utilities.DefaultTypeMapper._ import scala.language.higherKinds @@ -36,7 +37,8 @@ class NpmReleasesStage(stageId: String = "npm_releases_min", } override def getSchema: Schema = { - implicit val NpmSchema: DateSchemaFor = new DateSchemaFor(true) + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) AvroSchema[NpmRelease] } } diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/protocol/Protocol.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/protocol/Protocol.scala index f9dfe2c8..e918fdac 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/protocol/Protocol.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/protocol/Protocol.scala @@ -2,16 +2,20 @@ package org.codefeedr.plugins.pypi.protocol import java.util.Date +import com.sksamuel.avro4s.AvroProp + object Protocol { case class PyPiRelease(title: String, link: String, description: String, + @AvroProp("rowtime", "true") pubDate: Date) case class PyPiReleaseExt(title: String, link: String, description: String, + @AvroProp("rowtime", "true") pubDate: Date, project: PyPiProject) diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala index 73fe44cb..43b987bd 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleaseExtStage.scala @@ -1,8 +1,9 @@ package org.codefeedr.plugins.pypi.stages +import java.util.Date import java.util.concurrent.TimeUnit -import com.sksamuel.avro4s.AvroSchema +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} import org.apache.avro.Schema import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.api.datastream.{ @@ -14,7 +15,6 @@ import org.codefeedr.plugins.pypi.protocol.Protocol.{ } import org.codefeedr.stages.TransformStage import org.codefeedr.plugins.pypi.operators.RetrieveProjectAsync -import org.codefeedr.stages.utilities.DefaultTypeMapper.DateSchemaFor import scala.language.higherKinds @@ -44,7 +44,8 @@ class PyPiReleaseExtStage(stageId: String = "pypi_releases") } override def getSchema: Schema = { - implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) AvroSchema[PyPiReleaseExt] } } diff --git a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala index 7c16e2e1..eb777784 100644 --- a/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala +++ b/codefeedr-plugins/codefeedr-pypi/src/main/scala/org/codefeedr/plugins/pypi/stages/PyPiReleasesStage.scala @@ -1,6 +1,8 @@ package org.codefeedr.plugins.pypi.stages -import com.sksamuel.avro4s.AvroSchema +import java.util.Date + +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} import org.apache.avro.Schema import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.DataStream @@ -11,7 +13,6 @@ import org.codefeedr.plugins.pypi.operators.{ } import org.codefeedr.plugins.pypi.protocol.Protocol.PyPiRelease import org.codefeedr.stages.InputStage -import org.codefeedr.stages.utilities.DefaultTypeMapper._ import scala.language.higherKinds @@ -30,7 +31,8 @@ class PyPiReleasesStage(stageId: String = "pypi_releases_min", .addSource(new PyPiReleasesSource(sourceConfig)) override def getSchema: Schema = { - implicit val dateSchema: DateSchemaFor = new DateSchemaFor(true) + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) AvroSchema[PyPiRelease] } } From b60eb0d531ae5bb9c25a9f5482e92e97daeae715 Mon Sep 17 00:00:00 2001 From: AbeleMM Date: Tue, 21 Jul 2020 17:32:45 +0200 Subject: [PATCH 3/3] Enable schema registration for MavenReleasesExtStage --- .../plugins/maven/stages/MavenReleasesExtStage.scala | 11 +++++++++++ project/plugins.sbt | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala index aabc127c..02afc97c 100644 --- a/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala +++ b/codefeedr-plugins/codefeedr-maven/src/main/scala/org/codefeedr/plugins/maven/stages/MavenReleasesExtStage.scala @@ -1,7 +1,10 @@ package org.codefeedr.plugins.maven.stages +import java.util.Date import java.util.concurrent.TimeUnit +import com.sksamuel.avro4s.{AvroSchema, SchemaFor} +import org.apache.avro.Schema import org.apache.flink.streaming.api.datastream.{ AsyncDataStream => JavaAsyncDataStream } @@ -13,6 +16,8 @@ import org.codefeedr.plugins.maven.protocol.Protocol.{ MavenReleaseExt } +import scala.language.higherKinds + /** Transform a [[MavenRelease]] to [[MavenReleaseExt]]. * * @param stageId the name of this stage. @@ -37,4 +42,10 @@ class MavenReleasesExtStage(stageId: String = "maven_releases") new org.apache.flink.streaming.api.scala.DataStream(async) } + + override def getSchema: Schema = { + implicit val dateSchemaFor: AnyRef with SchemaFor[Date] = + SchemaFor[Date](Schema.create(Schema.Type.STRING)) + AvroSchema[MavenReleaseExt] + } } diff --git a/project/plugins.sbt b/project/plugins.sbt index 8ad67da8..180496ef 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,4 +3,4 @@ addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.2.4") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3") addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.1.1") addSbtPlugin("com.geirsson" % "sbt-scalafmt" % "1.5.1") -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.6") \ No newline at end of file +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.6")