From 0bc6432c89cf1d08313080727f7dc4f783017638 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 10 Mar 2019 23:06:48 +0000 Subject: [PATCH 01/89] Added reference and build.sbt files --- build.sbt | 3 +++ influxdb/src/main/resources/reference.conf | 24 ++++++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 influxdb/src/main/resources/reference.conf diff --git a/build.sbt b/build.sbt index 74e9c8e157..bbed66f26a 100644 --- a/build.sbt +++ b/build.sbt @@ -16,6 +16,7 @@ lazy val modules: Seq[ProjectReference] = Seq( googleFcm, hbase, hdfs, + influxdb, ironmq, jms, jsonStreaming, @@ -171,6 +172,8 @@ lazy val hbase = alpakkaProject("hbase", "hbase", Dependencies.HBase, fork in Te lazy val hdfs = alpakkaProject("hdfs", "hdfs", Dependencies.Hdfs, parallelExecution in Test := false) +lazy val influxdb = alpakkaProject("influxdb", "influxdb", Dependencies.InfluxDB) + lazy val ironmq = alpakkaProject("ironmq", "ironmq", Dependencies.IronMq, diff --git a/influxdb/src/main/resources/reference.conf b/influxdb/src/main/resources/reference.conf new file mode 100644 index 0000000000..46f0acb398 --- /dev/null +++ b/influxdb/src/main/resources/reference.conf @@ -0,0 +1,24 @@ +akka.stream.alpakka.influxdb{ + + # The InfluxDB host + host = "" + + # The InfluxDB port, https is used as default scheme + port = -1 + + # Use Transport Level Security (https) + tls = true + + # Max number of in flight requests from the InfluxDBClient - must be a power of 2 + parallelism = 32 + + # Optional max number of outstanding requests to the underlying host connection pool. + # If unspecified, will be the same as the parallelism value. Must be a power of 2 + #max-open-requests = + + # Optional Credentials. Used to define static credentials rather than use the DefaultAWSCredentialsProviderChain + #credentials { + # access-key-id = "dummy-access-key" + # secret-key-id = "dummy-secret-key" + #} +} From 190ae833805edb4ef3dcfa32b22a683b10995b02 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 14 Apr 2019 17:59:49 +0100 Subject: [PATCH 02/89] Added result mapper helper and source for influxdb --- .../impl/InfluxDBResultMapperHelper.java | 43 +++++++++++ .../influxdb/InfluxDBClientSettings.scala | 7 ++ .../alpakka/influxdb/InfluxDBSettings.scala | 29 +++++++ .../influxdb/InfuxDBOperationType.scala | 27 +++++++ .../influxdb/impl/InfluxDBSourceStage.scala | 75 +++++++++++++++++++ .../influxdb/javadsl/InfluxDBSource.scala | 25 +++++++ .../influxdb/scaladsl/InfluxDBSource.scala | 53 +++++++++++++ 7 files changed, 259 insertions(+) create mode 100644 influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java new file mode 100644 index 0000000000..bf64b4e1fe --- /dev/null +++ b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java @@ -0,0 +1,43 @@ +package org.influxdb.impl; + +import java.lang.reflect.Field; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; + +import org.influxdb.InfluxDBMapperException; + +public class InfluxDBResultMapperHelper { + + private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); + + public void cacheClassFields(final Class clazz) { + influxDBResultMapper.cacheMeasurementClass(clazz); + } + + public T parseRowAs(final Class clazz, List columns,final List values, TimeUnit precision) { + + try { + ConcurrentMap fieldMap = influxDBResultMapper.getColNameAndFieldMap(clazz); + + T object = null; + + for (int i = 0; i < columns.size(); i++) { + Field correspondingField = fieldMap.get(columns.get(i)); + + if (correspondingField != null) { + if (object == null) { + object = clazz.newInstance(); + } + influxDBResultMapper.setFieldValue(object, correspondingField, columns.get(i), precision); + } + + } + + return object; + } catch (InstantiationException | IllegalAccessException e) { + throw new InfluxDBMapperException(e); + } + } + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala new file mode 100644 index 0000000000..f239be1a23 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala @@ -0,0 +1,7 @@ +package akka.stream.alpakka.influxdb + +abstract class InfluxDBClientSettings { + val host: String + val port: Int + val parallelism: Int +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala new file mode 100644 index 0000000000..5fdc2dd242 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -0,0 +1,29 @@ +package akka.stream.alpakka.influxdb + +import org.influxdb.InfluxDBFactory + +final class InfluxDBSettings private ( +val host: String, +val port: Int, +val parallelism: Int, +) extends InfluxDBClientSettings { + + require(host.nonEmpty, "A host name must be provided.") + require(port > -1, "A port number must be provided.") + + def withHost(value: String): InfluxDBSettings = copy(host = value) + def withPort(value: Int): InfluxDBSettings = copy(port = value) + def withParallelism(value: Int): InfluxDBSettings = copy(parallelism = value) + + private def copy( + host: String = host, + port: Int = port, + parallelism: Int = parallelism + ): InfluxDBSettings = new InfluxDBSettings( + host = host, + port = port, + parallelism = parallelism, + ) + + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala new file mode 100644 index 0000000000..ad139a0d34 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala @@ -0,0 +1,27 @@ +package akka.stream.alpakka.influxdb + +import akka.annotation.InternalApi + +/** + * INTERNAL API + */ +@InternalApi +private[influxdb] sealed abstract class OperationType(val command: String) { + override def toString: String = command +} + +/** + * INTERNAL API + */ +@InternalApi +private[influxdb] object InfuxDBOperationType { + object CreateDatabase extends OperationType("create_database") + object CreateRetentionPolicy extends OperationType("create_retention_policy") + object AlterRetentionPolicy extends OperationType("alter_retention_policy") + object DropDatabase extends OperationType("drop_database") + object DropSeries extends OperationType("drop_series") + object DropRetentionPolicy extends OperationType("drop_retention_policy") + object DropMeasurement extends OperationType("drop_measurement") + object DropShard extends OperationType("drop_shard") + object Delete extends OperationType("delete") +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala new file mode 100644 index 0000000000..6f5fbd25f8 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -0,0 +1,75 @@ +package akka.stream.alpakka.influxdb.impl + +import akka.annotation.InternalApi +import akka.stream.stage._ +import akka.stream.{ActorMaterializer, Attributes, Outlet, SourceShape} +import org.influxdb.InfluxDB +import org.influxdb.dto.{Query, QueryResult} +import scala.collection.JavaConverters._ + +/** + * INTERNAL API + */ +@InternalApi +private[influxdb] trait MessageReader[T] { + def convert(json: QueryResult): T +} + +/** + * INTERNAL API + */ +@InternalApi +private[influxdb] trait RowReader[T] { + def convert(columns: java.util.List[String],row: java.util.List[AnyRef]): T +} + + +/** + * INTERNAL API + */ +@InternalApi +private[influxdb] final class InfluxDBSourceStage[T](query: Query, + client: InfluxDB, + reader: RowReader[T]) + extends GraphStage[SourceShape[T]] { + + val out: Outlet[T] = Outlet("InfluxDBSourceStage.out") + override val shape: SourceShape[T] = SourceShape(out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new InfluxDBSourceLogic[T](query,client, out,shape) + +} + +/** + * INTERNAL API + */ +private[influxdb] final class InfluxDBSourceLogic[T](query: Query, + client: InfluxDB, + outlet: Outlet[T], + shape: SourceShape[T], + reader: RowReader[T]) + extends GraphStageLogic(shape) + with OutHandler { + + private var columns: java.util.List[String] = null; + private var rows : Iterator[java.util.List[AnyRef]] = Iterator() + + override def preStart(): Unit = { + val results = client.query(query) + columns = results.getResults.get(0).getSeries().get(0).getColumns; + rows = results.getResults.get(0).getSeries.get(0).getValues.asScala.toIterator + } + + setHandler(outlet,this) + + override def onPull(): Unit = { + if(rows.hasNext) { + val row = rows.next(); + val measure = reader.convert(columns,row) + emit(outlet, measure) + } else { + completeStage() + } + } +} + diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala new file mode 100644 index 0000000000..5ef67533cb --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala @@ -0,0 +1,25 @@ +package akka.stream.alpakka.influxdb.javadsl + +import akka.NotUsed +import akka.stream.alpakka.influxdb.InfluxDBClientSettings +import akka.stream.javadsl.Source +import org.influxdb.InfluxDB +import org.influxdb.dto.Query + +import scala.concurrent.Future + +/** + * Java API to create InfluxDB sources. + */ +object InfluxDBSource { + + /* + def create[M]( + query: Query, + influxDB: InfluxDB, + influxDBClientSettings: InfluxDBClientSettings): Source[M,NotUsed] = + null + */ + + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala new file mode 100644 index 0000000000..ef7b71130b --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -0,0 +1,53 @@ +package akka.stream.alpakka.influxdb.scaladsl + +import java.util.concurrent.TimeUnit + +import akka.NotUsed +import akka.stream.scaladsl.Source +import org.influxdb.InfluxDB +import org.influxdb.dto.{Query, QueryResult} +import akka.stream.alpakka.influxdb.impl +import akka.stream.alpakka.influxdb.impl.InfluxDBSourceStage +import org.influxdb.impl.{InfluxDBResultMapper, InfluxDBResultMapperHelper} + +import scala.collection.JavaConverters._ + +/** + * Scala API to create InfluxDB sources. + */ +object InfluxDBSource { + + /** + * Scala API: creates a [[InfluxDBSourceStage]] from a given statement. + */ + def apply[T](query: Query)(implicit influxDB: InfluxDB): Source[T, NotUsed] = + Source.fromGraph( + new impl.InfluxDBSourceStage[T](query,influxDB, new RowReaderImpl[T]) + ) + + private final class QueryResultReader[T] extends impl.MessageReader[List[T]] { + override def convert(queryResult: QueryResult): List[T] = { + val influxDBResultMapper = new InfluxDBResultMapper(); + influxDBResultMapper.toPOJO(queryResult,classOf[T]) + .asScala + .toList + } + } + + private final class RowReaderImpl[T] extends impl.RowReader[T] { + + val mapperHelper = init() + + def init(): InfluxDBResultMapperHelper = { + val resultMapperHelper = new InfluxDBResultMapperHelper + resultMapperHelper.cacheClassFields(classOf[T]) + resultMapperHelper + } + + override def convert(column: java.util.List[String],row: java.util.List[AnyRef]): T = { + mapperHelper.parseRowAs(classOf[T],column,row,TimeUnit.MILLISECONDS) + } + } + +} + From 55a3a39258943d3672d4b9b47e207d7726809f0d Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 14 Apr 2019 19:14:35 +0100 Subject: [PATCH 03/89] Fixed influxdb source --- .../influxdb/javadsl/InfluxDBSource.scala | 34 +++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala index 5ef67533cb..be0745b0ef 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala @@ -1,25 +1,39 @@ package akka.stream.alpakka.influxdb.javadsl +import java.util.concurrent.TimeUnit + import akka.NotUsed -import akka.stream.alpakka.influxdb.InfluxDBClientSettings import akka.stream.javadsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.Query - -import scala.concurrent.Future +import akka.stream.alpakka.influxdb.impl.InfluxDBSourceStage +import akka.stream.alpakka.influxdb.impl +import org.influxdb.impl.InfluxDBResultMapperHelper /** * Java API to create InfluxDB sources. */ object InfluxDBSource { - /* - def create[M]( - query: Query, - influxDB: InfluxDB, - influxDBClientSettings: InfluxDBClientSettings): Source[M,NotUsed] = - null - */ + /** + * Java API: creates a [[InfluxDBSourceStage]] from a given statement. + */ + def create[T](query: Query, influxDB: InfluxDB,clazz: Class[T]): Source[T, NotUsed] = + akka.stream.javadsl.Source.fromGraph(new InfluxDBSourceStage[T](query, influxDB,new RowReaderImpl[T](clazz))) + + private final class RowReaderImpl[T](clazz: Class[T]) extends impl.RowReader[T] { + + val mapperHelper = init() + + def init(): InfluxDBResultMapperHelper = { + val resultMapperHelper = new InfluxDBResultMapperHelper + resultMapperHelper.cacheClassFields(classOf[T]) + resultMapperHelper + } + override def convert(column: java.util.List[String],row: java.util.List[AnyRef]): T = { + mapperHelper.parseRowAs(classOf[T],column,row,TimeUnit.MILLISECONDS) + } + } } From f7ba613688f0cb153b2d0f1f2ec8742fe6cb6d53 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:29:45 +0100 Subject: [PATCH 04/89] Added models for influxdb write operations --- .../influxdb/InfluxDBWriteMessage.scala | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala new file mode 100644 index 0000000000..264b4c836e --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala @@ -0,0 +1,22 @@ +package akka.stream.alpakka.influxdb + +import akka.NotUsed + +object InfluxDBWriteMessage { + // Apply method to use when not using passThrough + def apply[T](point: T): InfluxDBWriteMessage[T, NotUsed] = + InfluxDBWriteMessage(point, NotUsed) + + // Java-api - without passThrough + def create[T](point: T): InfluxDBWriteMessage[T, NotUsed] = + InfluxDBWriteMessage(point, NotUsed) + + // Java-api - with passThrough + def create[T, C](point: T, passThrough: C) = + InfluxDBWriteMessage(point, passThrough) +} + +final case class InfluxDBWriteMessage[T, C](point: T, passThrough: C) + +final case class InfluxDBWriteResult[T,C](writeMessage : InfluxDBWriteMessage[T,C], error: Option[String]) + From 778c91ceba184b495844ccd7b06173821839483c Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:30:17 +0100 Subject: [PATCH 05/89] Added influxdb source stage --- .../influxdb/impl/InfluxDBSourceStage.scala | 125 +++++++++++++----- 1 file changed, 90 insertions(+), 35 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 6f5fbd25f8..e187c97789 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -1,75 +1,130 @@ package akka.stream.alpakka.influxdb.impl +import java.util.concurrent.TimeUnit + import akka.annotation.InternalApi -import akka.stream.stage._ -import akka.stream.{ActorMaterializer, Attributes, Outlet, SourceShape} +import akka.stream.alpakka.influxdb.InfluxDBSettings +import akka.stream.{Attributes, Outlet, SourceShape} +import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} +import org.influxdb.impl.InfluxDBResultMapperHelper + import scala.collection.JavaConverters._ + + /** * INTERNAL API */ @InternalApi -private[influxdb] trait MessageReader[T] { - def convert(json: QueryResult): T +private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB, + query: Query, + ) extends GraphStage[SourceShape[T]] { + + val out: Outlet[T] = Outlet("InfluxDB.out") + override val shape = SourceShape(out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new InfluxDBSourceLogic[T](clazz, settings, influxDB, query, out, shape) + } /** - * INTERNAL API - */ + * INTERNAL API + */ @InternalApi -private[influxdb] trait RowReader[T] { - def convert(columns: java.util.List[String],row: java.util.List[AnyRef]): T -} +private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB, + query: Query, + outlet: Outlet[T], + shape: SourceShape[T]) + extends GraphStageLogic(shape) + with OutHandler { + + setHandler(outlet, this) + + var dataRetrieved: Option[QueryResult] = Option.empty + var resultMapperHelper: InfluxDBResultMapperHelper = _ + + override def preStart(): Unit = { + resultMapperHelper = new InfluxDBResultMapperHelper + resultMapperHelper.cacheClassFields(clazz) + + val queryResult = influxDB.query(query) + if(!queryResult.hasError) { + dataRetrieved = Option(queryResult) + } + } + + override def onPull(): Unit = { + if (dataRetrieved.isEmpty) + completeStage() + else { + var queryResult = dataRetrieved.get + for { + result <- queryResult.getResults.asScala + series <- result.getSeries.asScala + } ( + emitMultiple(outlet, resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) + ) + dataRetrieved = Option.empty + } + } + +} /** - * INTERNAL API - */ + * INTERNAL API + */ @InternalApi -private[influxdb] final class InfluxDBSourceStage[T](query: Query, - client: InfluxDB, - reader: RowReader[T]) - extends GraphStage[SourceShape[T]] { +private[influxdb] final class InfluxDBRawSourceStage(query: Query, + influxDB: InfluxDB + ) extends GraphStage[SourceShape[QueryResult]] { - val out: Outlet[T] = Outlet("InfluxDBSourceStage.out") - override val shape: SourceShape[T] = SourceShape(out) + val out: Outlet[QueryResult] = Outlet("InfluxDB.out") + override val shape = SourceShape(out) - override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new InfluxDBSourceLogic[T](query,client, out,shape) + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new InfluxDBSourceRawLogic(query, influxDB, out, shape) } /** * INTERNAL API */ -private[influxdb] final class InfluxDBSourceLogic[T](query: Query, - client: InfluxDB, - outlet: Outlet[T], - shape: SourceShape[T], - reader: RowReader[T]) +@InternalApi +private[influxdb] final class InfluxDBSourceRawLogic(query: Query, + influxDB: InfluxDB, + outlet: Outlet[QueryResult], + shape: SourceShape[QueryResult]) extends GraphStageLogic(shape) with OutHandler { - private var columns: java.util.List[String] = null; - private var rows : Iterator[java.util.List[AnyRef]] = Iterator() + setHandler(outlet, this) + + var dataRetrieved: Option[QueryResult] = Option.empty override def preStart(): Unit = { - val results = client.query(query) - columns = results.getResults.get(0).getSeries().get(0).getColumns; - rows = results.getResults.get(0).getSeries.get(0).getValues.asScala.toIterator + val queryResult = influxDB.query(query) + if(!queryResult.hasError) { + dataRetrieved = Option(queryResult) + } } - setHandler(outlet,this) - override def onPull(): Unit = { - if(rows.hasNext) { - val row = rows.next(); - val measure = reader.convert(columns,row) - emit(outlet, measure) - } else { + if(dataRetrieved.isEmpty) { completeStage() + } else { + emit(outlet,dataRetrieved.get) + dataRetrieved = Option.empty } } + } + From f4d9f306b1030c278b387e54d3ec21ab89c8586a Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:30:37 +0100 Subject: [PATCH 06/89] Added influxdb flow stage --- .../influxdb/impl/InfluxDBFlowStage.scala | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala new file mode 100644 index 0000000000..7c4996e260 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -0,0 +1,87 @@ +package akka.stream.alpakka.influxdb.impl + +import akka.stream.{Attributes, FlowShape, Inlet, Outlet} +import akka.stream.alpakka.influxdb.{InfluxDBWriteMessage, InfluxDBWriteResult} +import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} +import org.influxdb.InfluxDB +import org.influxdb.impl.InfluxDBMapper + +import scala.collection.immutable +import org.influxdb.BatchOptions +import org.influxdb.dto.Point + +/** + * INTERNAL API + */ +private[influxdb] class InfluxDBFlowStage[T,C]( + clazz: Option[Class[T]], + influxDB: InfluxDB + ) +extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutable.Seq[InfluxDBWriteResult[T, C]]]] +{ + private val in = Inlet[immutable.Seq[InfluxDBWriteMessage[T, C]]]("in") + private val out = Outlet[immutable.Seq[InfluxDBWriteResult[T, C]]]("out") + + override val shape = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + clazz match { + case Some(c) => new InfluxDBMapperRecordLogic + case None => new InfluxDBRecordLogic + } + + sealed abstract class InfluxDBLogic extends GraphStageLogic(shape) with InHandler with OutHandler { + + protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit + + setHandlers(in, out, this) + + override def onPull(): Unit = if (!isClosed(in) && !hasBeenPulled(in)) pull(in) + + override def onPush(): Unit = { + val messages = grab(in) + if (messages.nonEmpty) { + + influxDB.enableBatch(BatchOptions.DEFAULTS) + write(messages) + val writtenMessages = messages.map(m => new InfluxDBWriteResult(m, Option.empty)) + influxDB.close() + push(out, writtenMessages) + } + + tryPull(in) + } + } + + final class InfluxDBRecordLogic extends InfluxDBLogic { + + override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = { + messages.foreach { + case InfluxDBWriteMessage(point: Point, _) => { + influxDB.write(point) + println(point.toString) + } + case InfluxDBWriteMessage(others: AnyRef, _) => + failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) + } + } + + } + + final class InfluxDBMapperRecordLogic extends InfluxDBLogic { + + protected var influxDBMapper: InfluxDBMapper = _ + + override def preStart(): Unit = { + influxDBMapper = new InfluxDBMapper(influxDB) + } + + override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = { + messages.foreach { + case InfluxDBWriteMessage(typeMetric: Any, _) => + influxDBMapper.save(typeMetric) + } + } + } + +} From c4a420bbeccb2e51fda9a957dd2a8aa7d738ee84 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:31:08 +0100 Subject: [PATCH 07/89] Added influxdb source implementation for scala --- .../influxdb/scaladsl/InfluxDBSource.scala | 61 ++++++++----------- 1 file changed, 26 insertions(+), 35 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala index ef7b71130b..c5e37cc7e1 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -1,53 +1,44 @@ package akka.stream.alpakka.influxdb.scaladsl -import java.util.concurrent.TimeUnit - import akka.NotUsed +import akka.stream.alpakka.influxdb.InfluxDBSettings +import akka.stream.alpakka.influxdb.impl.{InfluxDBRawSourceStage, InfluxDBSourceStage} import akka.stream.scaladsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} -import akka.stream.alpakka.influxdb.impl -import akka.stream.alpakka.influxdb.impl.InfluxDBSourceStage -import org.influxdb.impl.{InfluxDBResultMapper, InfluxDBResultMapperHelper} - -import scala.collection.JavaConverters._ /** - * Scala API to create InfluxDB sources. + * Scala API. */ object InfluxDBSource { /** - * Scala API: creates a [[InfluxDBSourceStage]] from a given statement. + * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDBRawSourceStage]] from a given statement. */ - def apply[T](query: Query)(implicit influxDB: InfluxDB): Source[T, NotUsed] = + def apply(influxDB: InfluxDB, + query: Query + ): Source[QueryResult, NotUsed] = Source.fromGraph( - new impl.InfluxDBSourceStage[T](query,influxDB, new RowReaderImpl[T]) + new InfluxDBRawSourceStage( + query, + influxDB + ) ) - private final class QueryResultReader[T] extends impl.MessageReader[List[T]] { - override def convert(queryResult: QueryResult): List[T] = { - val influxDBResultMapper = new InfluxDBResultMapper(); - influxDBResultMapper.toPOJO(queryResult,classOf[T]) - .asScala - .toList - } - } - - private final class RowReaderImpl[T] extends impl.RowReader[T] { - - val mapperHelper = init() - - def init(): InfluxDBResultMapperHelper = { - val resultMapperHelper = new InfluxDBResultMapperHelper - resultMapperHelper.cacheClassFields(classOf[T]) - resultMapperHelper - } - - override def convert(column: java.util.List[String],row: java.util.List[AnyRef]): T = { - mapperHelper.parseRowAs(classOf[T],column,row,TimeUnit.MILLISECONDS) - } - } + /** + * Read elements of `T` from `className` or by `query`. + */ + def typed[T](clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB, + query: Query): Source[T, NotUsed] = + Source.fromGraph( + new InfluxDBSourceStage[T]( + clazz, + settings, + influxDB, + query + ) + ) } - From 2786a12ce52e4732bebc3c96b6163982116896d3 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:31:26 +0100 Subject: [PATCH 08/89] Added settings for influxdb database --- .../alpakka/influxdb/InfluxDBSettings.scala | 35 +++++++++---------- 1 file changed, 17 insertions(+), 18 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala index 5fdc2dd242..91f3c81f8d 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -1,29 +1,28 @@ package akka.stream.alpakka.influxdb -import org.influxdb.InfluxDBFactory +import java.util.concurrent.TimeUnit -final class InfluxDBSettings private ( -val host: String, -val port: Int, -val parallelism: Int, -) extends InfluxDBClientSettings { +import scala.concurrent.duration.TimeUnit - require(host.nonEmpty, "A host name must be provided.") - require(port > -1, "A port number must be provided.") +object InfluxDBSettings { + val Default = new InfluxDBSettings(batchSize= 10, TimeUnit.MILLISECONDS) - def withHost(value: String): InfluxDBSettings = copy(host = value) - def withPort(value: Int): InfluxDBSettings = copy(port = value) - def withParallelism(value: Int): InfluxDBSettings = copy(parallelism = value) + def apply(): InfluxDBSettings = Default + +} + +final class InfluxDBSettings private( +val batchSize: Int, val precision: TimeUnit +) { + + def withBatchSize(value: Int): InfluxDBSettings = copy(batchSize = value) private def copy( - host: String = host, - port: Int = port, - parallelism: Int = parallelism + batchSize: Int = batchSize, + precision: TimeUnit = precision ): InfluxDBSettings = new InfluxDBSettings( - host = host, - port = port, - parallelism = parallelism, + batchSize = batchSize, + precision = precision ) - } From c9a0b06273b98622c1b8780e1f93fa83f08c2360 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:31:54 +0100 Subject: [PATCH 09/89] Added scala flow for influxdb connector --- .../influxdb/scaladsl/InfluxDBFlow.scala | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala new file mode 100644 index 0000000000..d04190db4f --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -0,0 +1,93 @@ +package akka.stream.alpakka.influxdb.scaladsl + +import akka.NotUsed +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult, impl} +import akka.stream.scaladsl.Flow +import org.influxdb.InfluxDB + +import scala.collection.immutable +/** + * Scala API to create InfluxDB flows. + */ +object InfluxDBFlow { + + def create[T](settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + Flow[InfluxDBWriteMessage[T, NotUsed]] + .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .via( + new impl.InfluxDBFlowStage[T, NotUsed]( + Option.empty, + influxDB + ) + ).mapConcat(identity) + + def typed[T](clazz: Class[T], settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + Flow[InfluxDBWriteMessage[T, NotUsed]] + .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .via( + new impl.InfluxDBFlowStage[T,NotUsed]( + Option(clazz), + influxDB + ) + ) + .mapConcat(identity) + + def createWithPassThrough[T, C](settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + Flow[InfluxDBWriteMessage[T, C]] + .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .via( + new impl.InfluxDBFlowStage[T, C]( + Option.empty, + influxDB + ) + ) + .mapConcat(identity) + + def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + Flow[InfluxDBWriteMessage[T, C]] + .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .via( + new impl.InfluxDBFlowStage[T, C]( + Option(clazz), + influxDB + ) + ) + .mapConcat(identity) + + def createWithContext[T, C](settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = { + Flow[(InfluxDBWriteMessage[T, NotUsed], C)] + .map { + case (wm, pt) => + InfluxDBWriteMessage(wm.point,pt) + } + .via(createWithPassThrough(settings)) + .map { wr => + (wr, wr.writeMessage.passThrough) + } + } + + def typedWithContext[T, C](clazz: Class[T], settings: InfluxDBSettings)( + implicit influxDB: InfluxDB + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = { + Flow[(InfluxDBWriteMessage[T, NotUsed], C)] + .map { + case (wm, pt) => + InfluxDBWriteMessage(wm.point,pt) + } + .via(typedWithPassThrough(clazz, settings)) + .map { wr => + (wr, wr.writeMessage.passThrough) + } + } + +} From dfcce7f3ec418e23ef60a283bb2ad2e1b3d029a2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:32:23 +0100 Subject: [PATCH 10/89] Added scala dsl sink for influxdb connector --- .../influxdb/scaladsl/InfluxDBSink.scala | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala new file mode 100644 index 0000000000..72851d7701 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala @@ -0,0 +1,24 @@ +package akka.stream.alpakka.influxdb.scaladsl + +import akka.{Done, NotUsed} +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage} +import akka.stream.scaladsl.{Keep, Sink} +import org.influxdb.InfluxDB +import org.influxdb.dto.Point + +import scala.concurrent.Future + +object InfluxDBSink { + + def apply(settings: InfluxDBSettings)(implicit influxDB: InfluxDB): + Sink[InfluxDBWriteMessage[Point,NotUsed],Future[Done]] = + InfluxDBFlow.create[Point](settings).toMat(Sink.ignore)(Keep.right) + + def typed[T]( + clazz: Class[T], settings: InfluxDBSettings + ) (implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[T, NotUsed], Future[Done]] = + InfluxDBFlow + .typed(clazz, settings) + .toMat(Sink.ignore)(Keep.right) + +} From 9f4965715a568cfdd2aadac25cfa88d561c342b0 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:32:52 +0100 Subject: [PATCH 11/89] Added influxdb flow sink and source for java dsl --- .../influxdb/javadsl/InfluxDBFlow.scala | 27 ++++++++++++ .../influxdb/javadsl/InfluxDBSink.scala | 25 +++++++++++ .../influxdb/javadsl/InfluxDBSource.scala | 44 +++++++++---------- 3 files changed, 74 insertions(+), 22 deletions(-) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala new file mode 100644 index 0000000000..d8c2eba1a9 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala @@ -0,0 +1,27 @@ +package akka.stream.alpakka.influxdb.javadsl + +import akka.NotUsed +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} +import org.influxdb.InfluxDB + +object InfluxDBFlow { + + def create[T](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.create(settings)(influxDB).asJava + + def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typed(clazz, settings)(influxDB).asJava + + def createWithPassThrough[T, C](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithPassThrough(settings)(influxDB).asJava + + def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava + + def createWithContext[T,C](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithContext(settings)(influxDB).asJava + + def typedWithContext[T, C](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithContext(clazz, settings)(influxDB).asJava + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala new file mode 100644 index 0000000000..f042562beb --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala @@ -0,0 +1,25 @@ +package akka.stream.alpakka.influxdb.javadsl + +import java.util.concurrent.CompletionStage + +import akka.{Done, NotUsed} +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} +import akka.stream.javadsl.{Keep, Sink} +import org.influxdb.InfluxDB +import org.influxdb.dto.Point + + +/** + * Java API. + */ +object InfluxDBSink { + + def create(settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[Point,NotUsed], CompletionStage[Done]] = + InfluxDBFlow.create(settings, influxDB) + .toMat(Sink.ignore[InfluxDBWriteResult[Point,NotUsed]], Keep.right[NotUsed,CompletionStage[Done]]) + + def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[T, NotUsed], CompletionStage[Done]] = + InfluxDBFlow.typed(clazz,settings,influxDB) + .toMat(Sink.ignore[InfluxDBWriteResult[T,NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala index be0745b0ef..23a240bd15 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala @@ -1,14 +1,11 @@ package akka.stream.alpakka.influxdb.javadsl -import java.util.concurrent.TimeUnit - import akka.NotUsed +import akka.stream.alpakka.influxdb.InfluxDBSettings import akka.stream.javadsl.Source import org.influxdb.InfluxDB -import org.influxdb.dto.Query -import akka.stream.alpakka.influxdb.impl.InfluxDBSourceStage -import akka.stream.alpakka.influxdb.impl -import org.influxdb.impl.InfluxDBResultMapperHelper +import org.influxdb.dto.{Query, QueryResult} +import akka.stream.alpakka.influxdb.impl.{InfluxDBRawSourceStage, InfluxDBSourceStage} /** * Java API to create InfluxDB sources. @@ -16,24 +13,27 @@ import org.influxdb.impl.InfluxDBResultMapperHelper object InfluxDBSource { /** - * Java API: creates a [[InfluxDBSourceStage]] from a given statement. + * Java API: creates an [[InfluxDBRawSourceStage]] from a given statement. */ - def create[T](query: Query, influxDB: InfluxDB,clazz: Class[T]): Source[T, NotUsed] = - akka.stream.javadsl.Source.fromGraph(new InfluxDBSourceStage[T](query, influxDB,new RowReaderImpl[T](clazz))) - - private final class RowReaderImpl[T](clazz: Class[T]) extends impl.RowReader[T] { - - val mapperHelper = init() + def create(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = + Source.fromGraph(new InfluxDBRawSourceStage(query, influxDB)) - def init(): InfluxDBResultMapperHelper = { - val resultMapperHelper = new InfluxDBResultMapperHelper - resultMapperHelper.cacheClassFields(classOf[T]) - resultMapperHelper - } + /** + * Java API: creates an [[InfluxDBSourceStage]] of elements of `T` from `query`. + */ + def typed[T](clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB, + query: Query + ): Source[T, NotUsed] = + Source.fromGraph( + new InfluxDBSourceStage[T]( + clazz, + settings, + influxDB, + query + ) + ) - override def convert(column: java.util.List[String],row: java.util.List[AnyRef]): T = { - mapperHelper.parseRowAs(classOf[T],column,row,TimeUnit.MILLISECONDS) - } - } } From 4e4a3eefb24521adeafe211491301a928906749e Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:33:25 +0100 Subject: [PATCH 12/89] Influxdb connector tests and utils for java dsl --- influxdb/src/test/java/docs/javadsl/Cpu.java | 66 +++++++++ .../java/docs/javadsl/InfluxDBSourceTest.java | 103 ++++++++++++++ .../test/java/docs/javadsl/InfluxDBTest.java | 133 ++++++++++++++++++ .../test/java/docs/javadsl/TestConstants.java | 11 ++ .../src/test/java/docs/javadsl/TestUtils.java | 70 +++++++++ 5 files changed, 383 insertions(+) create mode 100644 influxdb/src/test/java/docs/javadsl/Cpu.java create mode 100644 influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java create mode 100644 influxdb/src/test/java/docs/javadsl/InfluxDBTest.java create mode 100644 influxdb/src/test/java/docs/javadsl/TestConstants.java create mode 100644 influxdb/src/test/java/docs/javadsl/TestUtils.java diff --git a/influxdb/src/test/java/docs/javadsl/Cpu.java b/influxdb/src/test/java/docs/javadsl/Cpu.java new file mode 100644 index 0000000000..4eb8fcb4f7 --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/Cpu.java @@ -0,0 +1,66 @@ +package docs.javadsl; + +import java.time.Instant; +import java.util.concurrent.TimeUnit; + +import org.influxdb.annotation.Column; +import org.influxdb.annotation.Measurement; + +import static docs.javadsl.TestConstants.DATABASE_NAME; + +@Measurement(name = "cpu", database= DATABASE_NAME, retentionPolicy="autogen", timeUnit = TimeUnit.MILLISECONDS) +public class Cpu { + + @Column(name = "time") + private Instant time; + @Column(name = "host", tag = true) + private String hostname; + @Column(name = "region", tag = true) + private String region; + @Column(name = "idle") + private Double idle; + @Column(name = "happydevop") + private Boolean happydevop; + @Column(name = "uptimesecs") + private Long uptimeSecs; + + public Cpu() { + } + + public Cpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + this.time = time; + this.hostname = hostname; + this.region = region; + this.idle = idle; + this.happydevop = happydevop; + this.uptimeSecs = uptimeSecs; + } + + public Instant getTime() { + return time; + } + + public String getHostname() { + return hostname; + } + + public String getRegion() { + return region; + } + + public Double getIdle() { + return idle; + } + + public Boolean getHappydevop() { + return happydevop; + } + + public Long getUptimeSecs() { + return uptimeSecs; + } + + public Cpu cloneAt(Instant time) { + return new Cpu(time,hostname,region,idle,happydevop,uptimeSecs); + } +} diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java new file mode 100644 index 0000000000..7ba8d58bff --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java @@ -0,0 +1,103 @@ +package docs.javadsl; + +import java.util.List; +import java.util.concurrent.CompletionStage; + +import org.influxdb.InfluxDB; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import akka.actor.ActorSystem; +import akka.japi.Pair; +import akka.stream.ActorMaterializer; +import akka.stream.Materializer; +import akka.stream.alpakka.influxdb.InfluxDBSettings; +import akka.stream.alpakka.influxdb.javadsl.InfluxDBSource; +import akka.stream.javadsl.Sink; +import akka.stream.testkit.javadsl.StreamTestKit; +import akka.testkit.javadsl.TestKit; +import static docs.javadsl.TestConstants.DATABASE_NAME; +import static docs.javadsl.TestUtils.cleanDatabase; +import static docs.javadsl.TestUtils.dropDatabase; +import static docs.javadsl.TestUtils.populateDatabase; +import static docs.javadsl.TestUtils.setupConnection; + +public class InfluxDBSourceTest { + + private static ActorSystem system; + private static Materializer materializer; + private static InfluxDB influxDB; + + private static Pair setupMaterializer() { + // #init-mat + final ActorSystem system = ActorSystem.create(); + final Materializer materializer = ActorMaterializer.create(system); + // #init-mat + return Pair.create(system, materializer); + } + + @BeforeClass + public static void setupDatabase() { + final Pair sysmat = setupMaterializer(); + system = sysmat.first(); + materializer = sysmat.second(); + + influxDB = setupConnection(); + } + + @AfterClass + public static void teardown() { + dropDatabase(influxDB); + TestKit.shutdownActorSystem(system); + } + + @Before + public void setUp() throws Exception { + populateDatabase(influxDB); + } + + @After + public void cleanUp() { + cleanDatabase(influxDB); + StreamTestKit.assertAllStagesStopped(materializer); + } + + @Test + public void streamQueryResult() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + CompletionStage> rows = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + .runWith(Sink.seq(),materializer); + + List cpus = rows.toCompletableFuture().get(); + + Assert.assertEquals(2,cpus.size()); + } + + @Test + public void streamRawQueryResult() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + CompletionStage> completionStage = InfluxDBSource.create(influxDB, query) + .runWith(Sink.seq(), materializer); + + List queryResults = completionStage.toCompletableFuture().get(); + QueryResult queryResult = queryResults.get(0); + + Assert.assertFalse(queryResult.hasError()); + + final int resultSize = queryResult.getResults() + .get(0).getSeries() + .get(0).getValues() + .size(); + + Assert.assertEquals(2, resultSize); + } + +} diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java new file mode 100644 index 0000000000..71be8507df --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -0,0 +1,133 @@ +package docs.javadsl; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletionStage; + +import org.influxdb.InfluxDB; +import org.influxdb.dto.Point; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import akka.Done; +import akka.NotUsed; +import akka.actor.ActorSystem; +import akka.japi.Pair; +import akka.stream.ActorMaterializer; +import akka.stream.Materializer; +import akka.stream.alpakka.influxdb.InfluxDBSettings; +import akka.stream.alpakka.influxdb.InfluxDBWriteMessage; +import akka.stream.alpakka.influxdb.javadsl.InfluxDBSink; +import akka.stream.alpakka.influxdb.javadsl.InfluxDBSource; +import akka.stream.javadsl.Sink; +import akka.stream.testkit.javadsl.StreamTestKit; +import akka.testkit.javadsl.TestKit; +import static docs.javadsl.TestConstants.DATABASE_NAME; +import static docs.javadsl.TestUtils.cleanDatabase; +import static docs.javadsl.TestUtils.dropDatabase; +import static docs.javadsl.TestUtils.populateDatabase; +import static docs.javadsl.TestUtils.resultToPoint; +import static docs.javadsl.TestUtils.setupConnection; + +public class InfluxDBTest { + + private static ActorSystem system; + private static Materializer materializer; + private static InfluxDB influxDB; + + private static Pair setupMaterializer() { + // #init-mat + final ActorSystem system = ActorSystem.create(); + final Materializer materializer = ActorMaterializer.create(system); + // #init-mat + return Pair.create(system, materializer); + } + + @BeforeClass + public static void setupDatabase() { + final Pair sysmat = setupMaterializer(); + system = sysmat.first(); + materializer = sysmat.second(); + + influxDB = setupConnection(); + } + + @AfterClass + public static void teardown() { + dropDatabase(influxDB); + TestKit.shutdownActorSystem(system); + } + + @Before + public void setUp() throws Exception { + populateDatabase(influxDB); + } + + @After + public void cleanUp() { + cleanDatabase(influxDB); + StreamTestKit.assertAllStagesStopped(materializer); + } + + @Test + public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + CompletionStage completionStage = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(),influxDB, query) + .map(cpu -> { + Cpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); + return new InfluxDBWriteMessage<>(clonedCpu, NotUsed.notUsed()); + }).runWith(InfluxDBSink.typed(Cpu.class, InfluxDBSettings.Default(), influxDB),materializer); + + Assert.assertNotNull(completionStage.toCompletableFuture().get()); + + CompletionStage> sources = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(),influxDB, query) + .runWith(Sink.seq(), materializer); + + Assert.assertEquals(4, sources.toCompletableFuture().get().size()); + } + + @Test + public void testConsumeAndPublishMeasurements() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + CompletionStage completionStage = InfluxDBSource.create(influxDB,query) + .map(queryResult -> points(queryResult)) + .mapConcat(i -> i) + .runWith(InfluxDBSink.create(InfluxDBSettings.Default(),influxDB),materializer); + + Assert.assertNotNull(completionStage.toCompletableFuture().get()); + + List queryResult = InfluxDBSource.create(influxDB, query) + .runWith(Sink.seq(), materializer) + .toCompletableFuture() + .get(); + final int resultSize = queryResult.get(0).getResults() + .get(0).getSeries() + .get(0).getValues() + .size(); + + Assert.assertEquals(4, resultSize); + } + + private List> points(QueryResult queryResult) { + List> points = new ArrayList<>(); + + for(QueryResult.Result result: queryResult.getResults()) { + for(QueryResult.Series serie: result.getSeries()) { + for(List rows: serie.getValues()) { + InfluxDBWriteMessage influxDBWriteMessage = new InfluxDBWriteMessage<>(resultToPoint(serie,rows),NotUsed.notUsed()); + points.add(influxDBWriteMessage); + } + } + } + + return points; + } + +} diff --git a/influxdb/src/test/java/docs/javadsl/TestConstants.java b/influxdb/src/test/java/docs/javadsl/TestConstants.java new file mode 100644 index 0000000000..bcc14028f7 --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/TestConstants.java @@ -0,0 +1,11 @@ +package docs.javadsl; + +public class TestConstants { + + public static final String INFLUXDB_URL = "http://localhost:8086"; + public static final String USERNAME = "root"; + public static final String PASSWORD = "root"; + + public static final String DATABASE_NAME = "influxdb_stream_scala_test"; + +} diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java new file mode 100644 index 0000000000..ae3355440b --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -0,0 +1,70 @@ +package docs.javadsl; + +import java.time.Instant; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.influxdb.InfluxDB; +import org.influxdb.InfluxDBFactory; +import org.influxdb.dto.Point; +import org.influxdb.dto.Query; +import org.influxdb.dto.QueryResult; +import org.influxdb.impl.InfluxDBMapper; + +import static docs.javadsl.TestConstants.DATABASE_NAME; +import static docs.javadsl.TestConstants.INFLUXDB_URL; +import static docs.javadsl.TestConstants.PASSWORD; +import static docs.javadsl.TestConstants.USERNAME; + +public class TestUtils { + + public static InfluxDB setupConnection() { + final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); + influxDB.setDatabase(DATABASE_NAME); + influxDB.setLogLevel(InfluxDB.LogLevel.FULL); + influxDB.query(new Query("CREATE DATABASE " + DATABASE_NAME, DATABASE_NAME)); + return influxDB; + } + + public static void populateDatabase(InfluxDB influxDB) { + InfluxDBMapper influxDBMapper = new InfluxDBMapper(influxDB); + Cpu firstCore = new Cpu(Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123l); + influxDBMapper.save(firstCore); + Cpu secondCore = new Cpu(Instant.now().minusSeconds(500),"local_2","eu-west-2",1.4d,true,123l); + influxDBMapper.save(secondCore); + } + + public static void cleanDatabase(InfluxDB influxDB) { + influxDB.query(new Query("DROP MEASUREMENT cpu", DATABASE_NAME)); + } + + public static void dropDatabase(InfluxDB influxDB) { + influxDB.query(new Query("DROP DATABASE "+DATABASE_NAME)); + } + + public static Point resultToPoint(QueryResult.Series serie, List values) { + Point.Builder builder = Point.measurement(serie.getName()); + + for(int i = 0; i < serie.getColumns().size(); i++) { + String column = serie.getColumns().get(i); + Object value = values.get(i); + + if(column.equals("time")) { + builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS); + } else if(column.equals("host") || column.equals("region")) { + builder.tag(column,value.toString()); + } else if(column.equals("uptimesecs")) { + builder.addField(column,((Double) value).longValue()); + } else { + if(value instanceof Long) builder.addField(column,(Long) value); + else if(value instanceof Double) builder.addField(column, (Double) value); + else if(value instanceof Number) builder.addField(column, (Number) value); + else if(value instanceof String) builder.addField(column, (String) value); + else if(value instanceof Boolean) builder.addField(column, (Boolean) value); + } + } + + return builder.build(); + } + +} From 0c30e8b65c265a8312521ebeea9980e109985f7a Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:33:48 +0100 Subject: [PATCH 13/89] Influxdb connector tests for scala dsl --- .../docs/scaladsl/InfluxDBSourceSpec.scala | 51 +++++++++ .../scala/docs/scaladsl/InfluxDBSpec.scala | 106 ++++++++++++++++++ 2 files changed, 157 insertions(+) create mode 100644 influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala create mode 100644 influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala new file mode 100644 index 0000000000..85f3af6445 --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -0,0 +1,51 @@ +package docs.scaladsl + +import akka.actor.ActorSystem +import akka.stream.ActorMaterializer +import akka.stream.alpakka.influxdb.scaladsl.InfluxDBSource +import akka.stream.scaladsl.Sink +import akka.testkit.TestKit +import org.influxdb.InfluxDB +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} +import org.scalatest.concurrent.ScalaFutures +import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped +import docs.javadsl.TestUtils._ +import docs.javadsl.TestConstants._ +import org.influxdb.dto.Query + +class InfluxDBSourceSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { + + implicit val system = ActorSystem() + implicit val mat = ActorMaterializer() + + implicit var influxDB: InfluxDB = _ + + override protected def beforeAll(): Unit = { + influxDB = setupConnection() + } + + override protected def afterAll(): Unit = { + TestKit.shutdownActorSystem(system) + } + + override def beforeEach(): Unit = { + populateDatabase(influxDB) + } + + override def afterEach() = { + cleanDatabase(influxDB) + } + + "support source" in assertAllStagesStopped { + // #run-typed + val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + val influxDBResult = InfluxDBSource(influxDB ,query).runWith(Sink.seq) + val resultToAssert = influxDBResult.futureValue.head + + val values = resultToAssert.getResults.get(0).getSeries().get(0).getValues + + values.size() mustBe 2 + } + +} diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala new file mode 100644 index 0000000000..3a3ea4ced0 --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -0,0 +1,106 @@ +package docs.scaladsl + +import java.util.concurrent.TimeUnit + +import akka.actor.ActorSystem +import akka.stream.ActorMaterializer +import org.influxdb.InfluxDB +import org.influxdb.dto.{Point, Query, QueryResult} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} +import org.scalatest.concurrent.ScalaFutures +import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped +import docs.javadsl.TestConstants.DATABASE_NAME +import akka.{Done, NotUsed} +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage} +import akka.stream.alpakka.influxdb.scaladsl.{InfluxDBSink, InfluxDBSource} +import akka.testkit.TestKit +import docs.javadsl.Cpu +import docs.javadsl.TestUtils._ +import akka.stream.scaladsl.Sink + +import scala.collection.JavaConverters._ + + +class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { + + implicit val system = ActorSystem() + implicit val mat = ActorMaterializer() + + implicit var influxDB: InfluxDB = _ + + override protected def beforeAll(): Unit = { + influxDB = setupConnection() + } + + override protected def afterAll(): Unit = { + TestKit.shutdownActorSystem(system) + } + + override def beforeEach(): Unit = { + populateDatabase(influxDB) + } + + override def afterEach() = { + cleanDatabase(influxDB) + } + + "support typed source" in assertAllStagesStopped { + val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + val measurements = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(),influxDB, query).runWith(Sink.seq) + + measurements.futureValue.map(_.getHostname) mustBe List("local_1", "local_2") + } + + "InfluxDBFlow" should { + + "consume and publish measurements using typed" in assertAllStagesStopped { + val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + val f1 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query) + .map { + cpu: Cpu => { + val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) + InfluxDBWriteMessage(clonedCpu) + } + }.runWith(InfluxDBSink.typed(classOf[Cpu], InfluxDBSettings())) + + f1.futureValue mustBe Done + + val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + + f2.futureValue.length mustBe 4 + } + + "consume and publish measurements" in assertAllStagesStopped { + val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + + val f1 = InfluxDBSource(influxDB, query) + .map { + queryReqult => resultToPoints(queryReqult) + } + .mapConcat(identity) + .runWith(InfluxDBSink(InfluxDBSettings())) + + f1.futureValue mustBe Done + + val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(),influxDB, query).runWith(Sink.seq) + + f2.futureValue.length mustBe 4 + } + + def resultToPoints(queryResult: QueryResult) : List[InfluxDBWriteMessage[Point,NotUsed]] = { + val points = for { + results <- queryResult.getResults.asScala + serie <- results.getSeries.asScala + values <- serie.getValues.asScala + } yield ( + InfluxDBWriteMessage(resultToPoint(serie, values)) + ) + points.toList + } + + + } + +} From 6ccf9b2b7663993823a8b633d7e1ce354df318c4 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:41:27 +0100 Subject: [PATCH 14/89] Added influxdb --- docker-compose.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker-compose.yml b/docker-compose.yml index d10c2061f7..948542e58d 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -186,6 +186,10 @@ services: image: mongo ports: - "27017:27017" + influxdb: + image: influxdb + ports: + - "8086:8086" mqtt: image: toke/mosquitto ports: From 32b94c4f41edf23fef29e5dbe43b269a2b87ad10 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 10:50:56 +0100 Subject: [PATCH 15/89] Influxdb connector result mapper helper --- .../influxdb/impl/InfluxDBResultMapperHelper.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java index bf64b4e1fe..6e8f705d90 100644 --- a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java +++ b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java @@ -1,11 +1,14 @@ package org.influxdb.impl; import java.lang.reflect.Field; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.influxdb.InfluxDBMapperException; +import org.influxdb.dto.QueryResult; public class InfluxDBResultMapperHelper { @@ -15,6 +18,13 @@ public void cacheClassFields(final Class clazz) { influxDBResultMapper.cacheMeasurementClass(clazz); } + public List parseSeriesAs(final Class clazz, final QueryResult.Series series, final TimeUnit precision) { + influxDBResultMapper.cacheMeasurementClass(clazz); + return series.getValues().stream() + .map(v-> parseRowAs(clazz, series.getColumns(), v, precision)) + .collect(Collectors.toList()); + } + public T parseRowAs(final Class clazz, List columns,final List values, TimeUnit precision) { try { @@ -29,7 +39,8 @@ public T parseRowAs(final Class clazz, List columns,final List Date: Sun, 5 May 2019 10:53:10 +0100 Subject: [PATCH 16/89] Influxdb connector reference --- influxdb/src/main/resources/reference.conf | 23 +++------------- .../influxdb/InfluxDBClientSettings.scala | 7 ----- .../influxdb/InfuxDBOperationType.scala | 27 ------------------- 3 files changed, 4 insertions(+), 53 deletions(-) delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala diff --git a/influxdb/src/main/resources/reference.conf b/influxdb/src/main/resources/reference.conf index 46f0acb398..ff71f4b780 100644 --- a/influxdb/src/main/resources/reference.conf +++ b/influxdb/src/main/resources/reference.conf @@ -1,24 +1,9 @@ akka.stream.alpakka.influxdb{ - # The InfluxDB host - host = "" + #batch write size + batchSize = 10 - # The InfluxDB port, https is used as default scheme - port = -1 + # The InfluxDB write precision + precision = MILLISECONDS - # Use Transport Level Security (https) - tls = true - - # Max number of in flight requests from the InfluxDBClient - must be a power of 2 - parallelism = 32 - - # Optional max number of outstanding requests to the underlying host connection pool. - # If unspecified, will be the same as the parallelism value. Must be a power of 2 - #max-open-requests = - - # Optional Credentials. Used to define static credentials rather than use the DefaultAWSCredentialsProviderChain - #credentials { - # access-key-id = "dummy-access-key" - # secret-key-id = "dummy-secret-key" - #} } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala deleted file mode 100644 index f239be1a23..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBClientSettings.scala +++ /dev/null @@ -1,7 +0,0 @@ -package akka.stream.alpakka.influxdb - -abstract class InfluxDBClientSettings { - val host: String - val port: Int - val parallelism: Int -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala deleted file mode 100644 index ad139a0d34..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfuxDBOperationType.scala +++ /dev/null @@ -1,27 +0,0 @@ -package akka.stream.alpakka.influxdb - -import akka.annotation.InternalApi - -/** - * INTERNAL API - */ -@InternalApi -private[influxdb] sealed abstract class OperationType(val command: String) { - override def toString: String = command -} - -/** - * INTERNAL API - */ -@InternalApi -private[influxdb] object InfuxDBOperationType { - object CreateDatabase extends OperationType("create_database") - object CreateRetentionPolicy extends OperationType("create_retention_policy") - object AlterRetentionPolicy extends OperationType("alter_retention_policy") - object DropDatabase extends OperationType("drop_database") - object DropSeries extends OperationType("drop_series") - object DropRetentionPolicy extends OperationType("drop_retention_policy") - object DropMeasurement extends OperationType("drop_measurement") - object DropShard extends OperationType("drop_shard") - object Delete extends OperationType("delete") -} From 777b2b733a36145bf2bd432e9e87f30167898e66 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 19:42:01 +0100 Subject: [PATCH 17/89] Influxdb Dependencies --- project/Dependencies.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 706535223f..5b593947a3 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -13,6 +13,8 @@ object Dependencies { case _ => "2.5.21" } + val InfluxDBJavaVersion = "2.15" + val AwsSdkVersion = "1.11.476" val AwsSdk2Version = "2.5.0" val AkkaHttpVersion = "10.1.7" @@ -231,6 +233,12 @@ object Dependencies { ) ) + val InfluxDB = Seq( + libraryDependencies ++= Seq( + "org.influxdb" % "influxdb-java" % InfluxDBJavaVersion // MIT + ) + ) + val IronMq = Seq( libraryDependencies ++= Seq( "com.typesafe.akka" %% "akka-http" % AkkaHttpVersion, From 8230b655fa55875feab1992dfcfaef937b9369b1 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 20:14:39 +0100 Subject: [PATCH 18/89] Applied formatting on Influxdb connector --- .../alpakka/influxdb/InfluxDBSettings.scala | 7 ++-- .../influxdb/InfluxDBWriteMessage.scala | 3 +- .../influxdb/impl/InfluxDBFlowStage.scala | 19 ++++----- .../influxdb/impl/InfluxDBSourceStage.scala | 42 ++++++++----------- .../influxdb/javadsl/InfluxDBFlow.scala | 33 ++++++++++++--- .../influxdb/javadsl/InfluxDBSink.scala | 20 +++++---- .../influxdb/javadsl/InfluxDBSource.scala | 7 +--- .../influxdb/scaladsl/InfluxDBFlow.scala | 22 +++++----- .../influxdb/scaladsl/InfluxDBSink.scala | 10 +++-- .../influxdb/scaladsl/InfluxDBSource.scala | 9 +--- 10 files changed, 90 insertions(+), 82 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala index 91f3c81f8d..084ce0cb95 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -5,14 +5,15 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.TimeUnit object InfluxDBSettings { - val Default = new InfluxDBSettings(batchSize= 10, TimeUnit.MILLISECONDS) + val Default = new InfluxDBSettings(batchSize = 10, TimeUnit.MILLISECONDS) def apply(): InfluxDBSettings = Default } -final class InfluxDBSettings private( -val batchSize: Int, val precision: TimeUnit +final class InfluxDBSettings private ( + val batchSize: Int, + val precision: TimeUnit ) { def withBatchSize(value: Int): InfluxDBSettings = copy(batchSize = value) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala index 264b4c836e..1b4c95b8ae 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala @@ -18,5 +18,4 @@ object InfluxDBWriteMessage { final case class InfluxDBWriteMessage[T, C](point: T, passThrough: C) -final case class InfluxDBWriteResult[T,C](writeMessage : InfluxDBWriteMessage[T,C], error: Option[String]) - +final case class InfluxDBWriteResult[T, C](writeMessage: InfluxDBWriteMessage[T, C], error: Option[String]) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 7c4996e260..0d741b7c09 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -13,12 +13,10 @@ import org.influxdb.dto.Point /** * INTERNAL API */ -private[influxdb] class InfluxDBFlowStage[T,C]( - clazz: Option[Class[T]], - influxDB: InfluxDB - ) -extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutable.Seq[InfluxDBWriteResult[T, C]]]] -{ +private[influxdb] class InfluxDBFlowStage[T, C]( + clazz: Option[Class[T]], + influxDB: InfluxDB +) extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutable.Seq[InfluxDBWriteResult[T, C]]]] { private val in = Inlet[immutable.Seq[InfluxDBWriteMessage[T, C]]]("in") private val out = Outlet[immutable.Seq[InfluxDBWriteResult[T, C]]]("out") @@ -55,7 +53,7 @@ extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutabl final class InfluxDBRecordLogic extends InfluxDBLogic { - override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = { + override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages.foreach { case InfluxDBWriteMessage(point: Point, _) => { influxDB.write(point) @@ -64,7 +62,6 @@ extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutabl case InfluxDBWriteMessage(others: AnyRef, _) => failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) } - } } @@ -72,16 +69,14 @@ extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutabl protected var influxDBMapper: InfluxDBMapper = _ - override def preStart(): Unit = { + override def preStart(): Unit = influxDBMapper = new InfluxDBMapper(influxDB) - } - override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = { + override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages.foreach { case InfluxDBWriteMessage(typeMetric: Any, _) => influxDBMapper.save(typeMetric) } - } } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index e187c97789..31138a3dbc 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -12,8 +12,6 @@ import org.influxdb.impl.InfluxDBResultMapperHelper import scala.collection.JavaConverters._ - - /** * INTERNAL API */ @@ -22,7 +20,7 @@ private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query, - ) extends GraphStage[SourceShape[T]] { +) extends GraphStage[SourceShape[T]] { val out: Outlet[T] = Outlet("InfluxDB.out") override val shape = SourceShape(out) @@ -33,8 +31,8 @@ private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], } /** - * INTERNAL API - */ + * INTERNAL API + */ @InternalApi private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], settings: InfluxDBSettings, @@ -42,7 +40,7 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], query: Query, outlet: Outlet[T], shape: SourceShape[T]) - extends GraphStageLogic(shape) + extends GraphStageLogic(shape) with OutHandler { setHandler(outlet, this) @@ -55,12 +53,12 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], resultMapperHelper.cacheClassFields(clazz) val queryResult = influxDB.query(query) - if(!queryResult.hasError) { + if (!queryResult.hasError) { dataRetrieved = Option(queryResult) } } - override def onPull(): Unit = { + override def onPull(): Unit = if (dataRetrieved.isEmpty) completeStage() else { @@ -68,13 +66,13 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], for { result <- queryResult.getResults.asScala series <- result.getSeries.asScala - } ( - emitMultiple(outlet, resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) + }( + emitMultiple(outlet, + resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) ) dataRetrieved = Option.empty } - } } @@ -82,9 +80,8 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], * INTERNAL API */ @InternalApi -private[influxdb] final class InfluxDBRawSourceStage(query: Query, - influxDB: InfluxDB - ) extends GraphStage[SourceShape[QueryResult]] { +private[influxdb] final class InfluxDBRawSourceStage(query: Query, influxDB: InfluxDB) + extends GraphStage[SourceShape[QueryResult]] { val out: Outlet[QueryResult] = Outlet("InfluxDB.out") override val shape = SourceShape(out) @@ -95,14 +92,14 @@ private[influxdb] final class InfluxDBRawSourceStage(query: Query, } /** - * INTERNAL API - */ + * INTERNAL API + */ @InternalApi private[influxdb] final class InfluxDBSourceRawLogic(query: Query, influxDB: InfluxDB, outlet: Outlet[QueryResult], shape: SourceShape[QueryResult]) - extends GraphStageLogic(shape) + extends GraphStageLogic(shape) with OutHandler { setHandler(outlet, this) @@ -111,20 +108,17 @@ private[influxdb] final class InfluxDBSourceRawLogic(query: Query, override def preStart(): Unit = { val queryResult = influxDB.query(query) - if(!queryResult.hasError) { + if (!queryResult.hasError) { dataRetrieved = Option(queryResult) } } - override def onPull(): Unit = { - if(dataRetrieved.isEmpty) { + override def onPull(): Unit = + if (dataRetrieved.isEmpty) { completeStage() } else { - emit(outlet,dataRetrieved.get) + emit(outlet, dataRetrieved.get) dataRetrieved = Option.empty } - } } - - diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala index d8c2eba1a9..acfd49d17d 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala @@ -6,22 +6,43 @@ import org.influxdb.InfluxDB object InfluxDBFlow { - def create[T](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + def create[T]( + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.create(settings)(influxDB).asJava - def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + def typed[T]( + clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typed(clazz, settings)(influxDB).asJava - def createWithPassThrough[T, C](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + def createWithPassThrough[T, C]( + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithPassThrough(settings)(influxDB).asJava - def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + def typedWithPassThrough[T, C]( + clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava - def createWithContext[T,C](settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + def createWithContext[T, C]( + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithContext(settings)(influxDB).asJava - def typedWithContext[T, C](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + def typedWithContext[T, C]( + clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithContext(clazz, settings)(influxDB).asJava } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala index f042562beb..e5c196654e 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala @@ -8,18 +8,24 @@ import akka.stream.javadsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point - /** * Java API. */ object InfluxDBSink { - def create(settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[Point,NotUsed], CompletionStage[Done]] = - InfluxDBFlow.create(settings, influxDB) - .toMat(Sink.ignore[InfluxDBWriteResult[Point,NotUsed]], Keep.right[NotUsed,CompletionStage[Done]]) + def create( + settings: InfluxDBSettings, + influxDB: InfluxDB + ): akka.stream.javadsl.Sink[InfluxDBWriteMessage[Point, NotUsed], CompletionStage[Done]] = + InfluxDBFlow + .create(settings, influxDB) + .toMat(Sink.ignore[InfluxDBWriteResult[Point, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) - def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[T, NotUsed], CompletionStage[Done]] = - InfluxDBFlow.typed(clazz,settings,influxDB) - .toMat(Sink.ignore[InfluxDBWriteResult[T,NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + def typed[T](clazz: Class[T], + settings: InfluxDBSettings, + influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[T, NotUsed], CompletionStage[Done]] = + InfluxDBFlow + .typed(clazz, settings, influxDB) + .toMat(Sink.ignore[InfluxDBWriteResult[T, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala index 23a240bd15..08b393b6db 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala @@ -21,11 +21,7 @@ object InfluxDBSource { /** * Java API: creates an [[InfluxDBSourceStage]] of elements of `T` from `query`. */ - def typed[T](clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB, - query: Query - ): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( new InfluxDBSourceStage[T]( clazz, @@ -35,5 +31,4 @@ object InfluxDBSource { ) ) - } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index d04190db4f..774413fee9 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -1,11 +1,12 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult, impl} +import akka.stream.alpakka.influxdb.{impl, InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB import scala.collection.immutable + /** * Scala API to create InfluxDB flows. */ @@ -21,20 +22,21 @@ object InfluxDBFlow { Option.empty, influxDB ) - ).mapConcat(identity) + ) + .mapConcat(identity) def typed[T](clazz: Class[T], settings: InfluxDBSettings)( - implicit influxDB: InfluxDB + implicit influxDB: InfluxDB ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } .via( - new impl.InfluxDBFlowStage[T,NotUsed]( + new impl.InfluxDBFlowStage[T, NotUsed]( Option(clazz), influxDB ) ) - .mapConcat(identity) + .mapConcat(identity) def createWithPassThrough[T, C](settings: InfluxDBSettings)( implicit influxDB: InfluxDB @@ -64,30 +66,28 @@ object InfluxDBFlow { def createWithContext[T, C](settings: InfluxDBSettings)( implicit influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = { + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = Flow[(InfluxDBWriteMessage[T, NotUsed], C)] .map { case (wm, pt) => - InfluxDBWriteMessage(wm.point,pt) + InfluxDBWriteMessage(wm.point, pt) } .via(createWithPassThrough(settings)) .map { wr => (wr, wr.writeMessage.passThrough) } - } def typedWithContext[T, C](clazz: Class[T], settings: InfluxDBSettings)( implicit influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = { + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = Flow[(InfluxDBWriteMessage[T, NotUsed], C)] .map { case (wm, pt) => - InfluxDBWriteMessage(wm.point,pt) + InfluxDBWriteMessage(wm.point, pt) } .via(typedWithPassThrough(clazz, settings)) .map { wr => (wr, wr.writeMessage.passThrough) } - } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala index 72851d7701..ab562a7c56 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala @@ -10,13 +10,15 @@ import scala.concurrent.Future object InfluxDBSink { - def apply(settings: InfluxDBSettings)(implicit influxDB: InfluxDB): - Sink[InfluxDBWriteMessage[Point,NotUsed],Future[Done]] = + def apply( + settings: InfluxDBSettings + )(implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[Point, NotUsed], Future[Done]] = InfluxDBFlow.create[Point](settings).toMat(Sink.ignore)(Keep.right) def typed[T]( - clazz: Class[T], settings: InfluxDBSettings - ) (implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[T, NotUsed], Future[Done]] = + clazz: Class[T], + settings: InfluxDBSettings + )(implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[T, NotUsed], Future[Done]] = InfluxDBFlow .typed(clazz, settings) .toMat(Sink.ignore)(Keep.right) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala index c5e37cc7e1..dc315094b3 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -15,9 +15,7 @@ object InfluxDBSource { /** * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDBRawSourceStage]] from a given statement. */ - def apply(influxDB: InfluxDB, - query: Query - ): Source[QueryResult, NotUsed] = + def apply(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = Source.fromGraph( new InfluxDBRawSourceStage( query, @@ -28,10 +26,7 @@ object InfluxDBSource { /** * Read elements of `T` from `className` or by `query`. */ - def typed[T](clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB, - query: Query): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( new InfluxDBSourceStage[T]( clazz, From 9f92fe112b6ec0c56b121cc6af6b3eb5adb49231 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 5 May 2019 21:02:06 +0100 Subject: [PATCH 19/89] Fixed source stage --- .../alpakka/influxdb/impl/InfluxDBSourceStage.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 31138a3dbc..f59e70b3de 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.impl import java.util.concurrent.TimeUnit @@ -19,8 +23,8 @@ import scala.collection.JavaConverters._ private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, - query: Query, -) extends GraphStage[SourceShape[T]] { + query: Query) + extends GraphStage[SourceShape[T]] { val out: Outlet[T] = Outlet("InfluxDB.out") override val shape = SourceShape(out) @@ -62,7 +66,7 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], if (dataRetrieved.isEmpty) completeStage() else { - var queryResult = dataRetrieved.get + val queryResult = dataRetrieved.get for { result <- queryResult.getResults.asScala series <- result.getSeries.asScala @@ -113,12 +117,13 @@ private[influxdb] final class InfluxDBSourceRawLogic(query: Query, } } - override def onPull(): Unit = + override def onPull(): Unit = { if (dataRetrieved.isEmpty) { completeStage() } else { emit(outlet, dataRetrieved.get) dataRetrieved = Option.empty } + } } From 66949156652954039cbe21701deadbd553b95962 Mon Sep 17 00:00:00 2001 From: Christopher Hunt Date: Sun, 5 May 2019 21:22:22 +0100 Subject: [PATCH 20/89] InfluxDB connector compile for 2.11.12 --- .../impl/InfluxDBResultMapperHelper.java | 64 +++--- .../alpakka/influxdb/InfluxDBSettings.scala | 4 + .../influxdb/InfluxDBWriteMessage.scala | 4 + .../influxdb/impl/InfluxDBFlowStage.scala | 4 + .../influxdb/impl/InfluxDBSourceStage.scala | 5 +- .../influxdb/javadsl/InfluxDBFlow.scala | 4 + .../influxdb/javadsl/InfluxDBSink.scala | 4 + .../influxdb/javadsl/InfluxDBSource.scala | 4 + .../influxdb/scaladsl/InfluxDBFlow.scala | 4 + .../influxdb/scaladsl/InfluxDBSink.scala | 4 + .../influxdb/scaladsl/InfluxDBSource.scala | 4 + influxdb/src/test/java/docs/javadsl/Cpu.java | 124 +++++++----- .../java/docs/javadsl/InfluxDBSourceTest.java | 109 +++++----- .../test/java/docs/javadsl/InfluxDBTest.java | 189 +++++++++--------- .../test/java/docs/javadsl/TestConstants.java | 13 +- .../src/test/java/docs/javadsl/TestUtils.java | 85 ++++---- .../docs/scaladsl/InfluxDBSourceSpec.scala | 27 ++- .../scala/docs/scaladsl/InfluxDBSpec.scala | 44 ++-- 18 files changed, 389 insertions(+), 307 deletions(-) diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java index 6e8f705d90..36bd1eb60d 100644 --- a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java +++ b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package org.influxdb.impl; import java.lang.reflect.Field; @@ -12,43 +16,45 @@ public class InfluxDBResultMapperHelper { - private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); - - public void cacheClassFields(final Class clazz) { - influxDBResultMapper.cacheMeasurementClass(clazz); - } + private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); - public List parseSeriesAs(final Class clazz, final QueryResult.Series series, final TimeUnit precision) { - influxDBResultMapper.cacheMeasurementClass(clazz); - return series.getValues().stream() - .map(v-> parseRowAs(clazz, series.getColumns(), v, precision)) - .collect(Collectors.toList()); - } + public void cacheClassFields(final Class clazz) { + influxDBResultMapper.cacheMeasurementClass(clazz); + } - public T parseRowAs(final Class clazz, List columns,final List values, TimeUnit precision) { + public List parseSeriesAs( + final Class clazz, final QueryResult.Series series, final TimeUnit precision) { + influxDBResultMapper.cacheMeasurementClass(clazz); + return series + .getValues() + .stream() + .map(v -> parseRowAs(clazz, series.getColumns(), v, precision)) + .collect(Collectors.toList()); + } - try { - ConcurrentMap fieldMap = influxDBResultMapper.getColNameAndFieldMap(clazz); + public T parseRowAs( + final Class clazz, List columns, final List values, TimeUnit precision) { - T object = null; + try { + ConcurrentMap fieldMap = influxDBResultMapper.getColNameAndFieldMap(clazz); - for (int i = 0; i < columns.size(); i++) { - Field correspondingField = fieldMap.get(columns.get(i)); + T object = null; - if (correspondingField != null) { - if (object == null) { - object = clazz.newInstance(); - } + for (int i = 0; i < columns.size(); i++) { + Field correspondingField = fieldMap.get(columns.get(i)); - influxDBResultMapper.setFieldValue(object, correspondingField, values.get(i), precision); - } + if (correspondingField != null) { + if (object == null) { + object = clazz.newInstance(); + } - } - - return object; - } catch (InstantiationException | IllegalAccessException e) { - throw new InfluxDBMapperException(e); + influxDBResultMapper.setFieldValue(object, correspondingField, values.get(i), precision); } - } + } + return object; + } catch (InstantiationException | IllegalAccessException e) { + throw new InfluxDBMapperException(e); + } + } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala index 084ce0cb95..5d9db02cc2 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb import java.util.concurrent.TimeUnit diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala index 1b4c95b8ae..dbec946d90 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb import akka.NotUsed diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 0d741b7c09..06baa4e380 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.impl import akka.stream.{Attributes, FlowShape, Inlet, Outlet} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index f59e70b3de..3bf06d2d84 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -24,7 +24,7 @@ private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query) - extends GraphStage[SourceShape[T]] { + extends GraphStage[SourceShape[T]] { val out: Outlet[T] = Outlet("InfluxDB.out") override val shape = SourceShape(out) @@ -117,13 +117,12 @@ private[influxdb] final class InfluxDBSourceRawLogic(query: Query, } } - override def onPull(): Unit = { + override def onPull(): Unit = if (dataRetrieved.isEmpty) { completeStage() } else { emit(outlet, dataRetrieved.get) dataRetrieved = Option.empty } - } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala index acfd49d17d..3d5d79920d 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala index e5c196654e..9c8db5cfa8 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.javadsl import java.util.concurrent.CompletionStage diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala index 08b393b6db..9145e33c78 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index 774413fee9..3054745406 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala index ab562a7c56..faf9c9b41a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.scaladsl import akka.{Done, NotUsed} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala index dc315094b3..18e61c28d7 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed diff --git a/influxdb/src/test/java/docs/javadsl/Cpu.java b/influxdb/src/test/java/docs/javadsl/Cpu.java index 4eb8fcb4f7..ef09bb4785 100644 --- a/influxdb/src/test/java/docs/javadsl/Cpu.java +++ b/influxdb/src/test/java/docs/javadsl/Cpu.java @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.javadsl; import java.time.Instant; @@ -8,59 +12,73 @@ import static docs.javadsl.TestConstants.DATABASE_NAME; -@Measurement(name = "cpu", database= DATABASE_NAME, retentionPolicy="autogen", timeUnit = TimeUnit.MILLISECONDS) +@Measurement( + name = "cpu", + database = DATABASE_NAME, + retentionPolicy = "autogen", + timeUnit = TimeUnit.MILLISECONDS) public class Cpu { - @Column(name = "time") - private Instant time; - @Column(name = "host", tag = true) - private String hostname; - @Column(name = "region", tag = true) - private String region; - @Column(name = "idle") - private Double idle; - @Column(name = "happydevop") - private Boolean happydevop; - @Column(name = "uptimesecs") - private Long uptimeSecs; - - public Cpu() { - } - - public Cpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { - this.time = time; - this.hostname = hostname; - this.region = region; - this.idle = idle; - this.happydevop = happydevop; - this.uptimeSecs = uptimeSecs; - } - - public Instant getTime() { - return time; - } - - public String getHostname() { - return hostname; - } - - public String getRegion() { - return region; - } - - public Double getIdle() { - return idle; - } - - public Boolean getHappydevop() { - return happydevop; - } - - public Long getUptimeSecs() { - return uptimeSecs; - } - - public Cpu cloneAt(Instant time) { - return new Cpu(time,hostname,region,idle,happydevop,uptimeSecs); - } + @Column(name = "time") + private Instant time; + + @Column(name = "host", tag = true) + private String hostname; + + @Column(name = "region", tag = true) + private String region; + + @Column(name = "idle") + private Double idle; + + @Column(name = "happydevop") + private Boolean happydevop; + + @Column(name = "uptimesecs") + private Long uptimeSecs; + + public Cpu() {} + + public Cpu( + Instant time, + String hostname, + String region, + Double idle, + Boolean happydevop, + Long uptimeSecs) { + this.time = time; + this.hostname = hostname; + this.region = region; + this.idle = idle; + this.happydevop = happydevop; + this.uptimeSecs = uptimeSecs; + } + + public Instant getTime() { + return time; + } + + public String getHostname() { + return hostname; + } + + public String getRegion() { + return region; + } + + public Double getIdle() { + return idle; + } + + public Boolean getHappydevop() { + return happydevop; + } + + public Long getUptimeSecs() { + return uptimeSecs; + } + + public Cpu cloneAt(Instant time) { + return new Cpu(time, hostname, region, idle, happydevop, uptimeSecs); + } } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java index 7ba8d58bff..2653fc3afa 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.javadsl; import java.util.List; @@ -30,74 +34,71 @@ public class InfluxDBSourceTest { - private static ActorSystem system; - private static Materializer materializer; - private static InfluxDB influxDB; - - private static Pair setupMaterializer() { - // #init-mat - final ActorSystem system = ActorSystem.create(); - final Materializer materializer = ActorMaterializer.create(system); - // #init-mat - return Pair.create(system, materializer); - } + private static ActorSystem system; + private static Materializer materializer; + private static InfluxDB influxDB; - @BeforeClass - public static void setupDatabase() { - final Pair sysmat = setupMaterializer(); - system = sysmat.first(); - materializer = sysmat.second(); + private static Pair setupMaterializer() { + // #init-mat + final ActorSystem system = ActorSystem.create(); + final Materializer materializer = ActorMaterializer.create(system); + // #init-mat + return Pair.create(system, materializer); + } - influxDB = setupConnection(); - } + @BeforeClass + public static void setupDatabase() { + final Pair sysmat = setupMaterializer(); + system = sysmat.first(); + materializer = sysmat.second(); - @AfterClass - public static void teardown() { - dropDatabase(influxDB); - TestKit.shutdownActorSystem(system); - } + influxDB = setupConnection(); + } - @Before - public void setUp() throws Exception { - populateDatabase(influxDB); - } + @AfterClass + public static void teardown() { + dropDatabase(influxDB); + TestKit.shutdownActorSystem(system); + } - @After - public void cleanUp() { - cleanDatabase(influxDB); - StreamTestKit.assertAllStagesStopped(materializer); - } + @Before + public void setUp() throws Exception { + populateDatabase(influxDB); + } - @Test - public void streamQueryResult() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + @After + public void cleanUp() { + cleanDatabase(influxDB); + StreamTestKit.assertAllStagesStopped(materializer); + } - CompletionStage> rows = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) - .runWith(Sink.seq(),materializer); + @Test + public void streamQueryResult() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); - List cpus = rows.toCompletableFuture().get(); + CompletionStage> rows = + InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + .runWith(Sink.seq(), materializer); - Assert.assertEquals(2,cpus.size()); - } + List cpus = rows.toCompletableFuture().get(); - @Test - public void streamRawQueryResult() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + Assert.assertEquals(2, cpus.size()); + } - CompletionStage> completionStage = InfluxDBSource.create(influxDB, query) - .runWith(Sink.seq(), materializer); + @Test + public void streamRawQueryResult() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); - List queryResults = completionStage.toCompletableFuture().get(); - QueryResult queryResult = queryResults.get(0); + CompletionStage> completionStage = + InfluxDBSource.create(influxDB, query).runWith(Sink.seq(), materializer); - Assert.assertFalse(queryResult.hasError()); + List queryResults = completionStage.toCompletableFuture().get(); + QueryResult queryResult = queryResults.get(0); - final int resultSize = queryResult.getResults() - .get(0).getSeries() - .get(0).getValues() - .size(); + Assert.assertFalse(queryResult.hasError()); - Assert.assertEquals(2, resultSize); - } + final int resultSize = queryResult.getResults().get(0).getSeries().get(0).getValues().size(); + Assert.assertEquals(2, resultSize); + } } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java index 71be8507df..73fec964a1 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.javadsl; import java.util.ArrayList; @@ -37,97 +41,102 @@ public class InfluxDBTest { - private static ActorSystem system; - private static Materializer materializer; - private static InfluxDB influxDB; - - private static Pair setupMaterializer() { - // #init-mat - final ActorSystem system = ActorSystem.create(); - final Materializer materializer = ActorMaterializer.create(system); - // #init-mat - return Pair.create(system, materializer); - } - - @BeforeClass - public static void setupDatabase() { - final Pair sysmat = setupMaterializer(); - system = sysmat.first(); - materializer = sysmat.second(); - - influxDB = setupConnection(); - } - - @AfterClass - public static void teardown() { - dropDatabase(influxDB); - TestKit.shutdownActorSystem(system); - } - - @Before - public void setUp() throws Exception { - populateDatabase(influxDB); - } - - @After - public void cleanUp() { - cleanDatabase(influxDB); - StreamTestKit.assertAllStagesStopped(materializer); - } - - @Test - public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); - CompletionStage completionStage = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(),influxDB, query) - .map(cpu -> { - Cpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); - return new InfluxDBWriteMessage<>(clonedCpu, NotUsed.notUsed()); - }).runWith(InfluxDBSink.typed(Cpu.class, InfluxDBSettings.Default(), influxDB),materializer); - - Assert.assertNotNull(completionStage.toCompletableFuture().get()); - - CompletionStage> sources = InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(),influxDB, query) - .runWith(Sink.seq(), materializer); - - Assert.assertEquals(4, sources.toCompletableFuture().get().size()); - } - - @Test - public void testConsumeAndPublishMeasurements() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); - - CompletionStage completionStage = InfluxDBSource.create(influxDB,query) - .map(queryResult -> points(queryResult)) - .mapConcat(i -> i) - .runWith(InfluxDBSink.create(InfluxDBSettings.Default(),influxDB),materializer); - - Assert.assertNotNull(completionStage.toCompletableFuture().get()); - - List queryResult = InfluxDBSource.create(influxDB, query) - .runWith(Sink.seq(), materializer) - .toCompletableFuture() - .get(); - final int resultSize = queryResult.get(0).getResults() - .get(0).getSeries() - .get(0).getValues() - .size(); - - Assert.assertEquals(4, resultSize); - } - - private List> points(QueryResult queryResult) { - List> points = new ArrayList<>(); - - for(QueryResult.Result result: queryResult.getResults()) { - for(QueryResult.Series serie: result.getSeries()) { - for(List rows: serie.getValues()) { - InfluxDBWriteMessage influxDBWriteMessage = new InfluxDBWriteMessage<>(resultToPoint(serie,rows),NotUsed.notUsed()); - points.add(influxDBWriteMessage); - } - } + private static ActorSystem system; + private static Materializer materializer; + private static InfluxDB influxDB; + + private static Pair setupMaterializer() { + // #init-mat + final ActorSystem system = ActorSystem.create(); + final Materializer materializer = ActorMaterializer.create(system); + // #init-mat + return Pair.create(system, materializer); + } + + @BeforeClass + public static void setupDatabase() { + final Pair sysmat = setupMaterializer(); + system = sysmat.first(); + materializer = sysmat.second(); + + influxDB = setupConnection(); + } + + @AfterClass + public static void teardown() { + dropDatabase(influxDB); + TestKit.shutdownActorSystem(system); + } + + @Before + public void setUp() throws Exception { + populateDatabase(influxDB); + } + + @After + public void cleanUp() { + cleanDatabase(influxDB); + StreamTestKit.assertAllStagesStopped(materializer); + } + + @Test + public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + CompletionStage completionStage = + InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + .map( + cpu -> { + Cpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); + return new InfluxDBWriteMessage<>(clonedCpu, NotUsed.notUsed()); + }) + .runWith( + InfluxDBSink.typed(Cpu.class, InfluxDBSettings.Default(), influxDB), materializer); + + Assert.assertNotNull(completionStage.toCompletableFuture().get()); + + CompletionStage> sources = + InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + .runWith(Sink.seq(), materializer); + + Assert.assertEquals(4, sources.toCompletableFuture().get().size()); + } + + @Test + public void testConsumeAndPublishMeasurements() throws Exception { + Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + + CompletionStage completionStage = + InfluxDBSource.create(influxDB, query) + .map(queryResult -> points(queryResult)) + .mapConcat(i -> i) + .runWith(InfluxDBSink.create(InfluxDBSettings.Default(), influxDB), materializer); + + Assert.assertNotNull(completionStage.toCompletableFuture().get()); + + List queryResult = + InfluxDBSource.create(influxDB, query) + .runWith(Sink.seq(), materializer) + .toCompletableFuture() + .get(); + final int resultSize = + queryResult.get(0).getResults().get(0).getSeries().get(0).getValues().size(); + + Assert.assertEquals(4, resultSize); + } + + private List> points(QueryResult queryResult) { + List> points = new ArrayList<>(); + + for (QueryResult.Result result : queryResult.getResults()) { + for (QueryResult.Series serie : result.getSeries()) { + for (List rows : serie.getValues()) { + InfluxDBWriteMessage influxDBWriteMessage = + new InfluxDBWriteMessage<>(resultToPoint(serie, rows), NotUsed.notUsed()); + points.add(influxDBWriteMessage); } - - return points; + } } + return points; + } } diff --git a/influxdb/src/test/java/docs/javadsl/TestConstants.java b/influxdb/src/test/java/docs/javadsl/TestConstants.java index bcc14028f7..ea744cc1eb 100644 --- a/influxdb/src/test/java/docs/javadsl/TestConstants.java +++ b/influxdb/src/test/java/docs/javadsl/TestConstants.java @@ -1,11 +1,14 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.javadsl; public class TestConstants { - public static final String INFLUXDB_URL = "http://localhost:8086"; - public static final String USERNAME = "root"; - public static final String PASSWORD = "root"; - - public static final String DATABASE_NAME = "influxdb_stream_scala_test"; + public static final String INFLUXDB_URL = "http://localhost:8086"; + public static final String USERNAME = "root"; + public static final String PASSWORD = "root"; + public static final String DATABASE_NAME = "influxdb_stream_scala_test"; } diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index ae3355440b..9a682e141e 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.javadsl; import java.time.Instant; @@ -18,53 +22,54 @@ public class TestUtils { - public static InfluxDB setupConnection() { - final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); - influxDB.setDatabase(DATABASE_NAME); - influxDB.setLogLevel(InfluxDB.LogLevel.FULL); - influxDB.query(new Query("CREATE DATABASE " + DATABASE_NAME, DATABASE_NAME)); - return influxDB; - } + public static InfluxDB setupConnection() { + final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); + influxDB.setDatabase(DATABASE_NAME); + influxDB.setLogLevel(InfluxDB.LogLevel.FULL); + influxDB.query(new Query("CREATE DATABASE " + DATABASE_NAME, DATABASE_NAME)); + return influxDB; + } - public static void populateDatabase(InfluxDB influxDB) { - InfluxDBMapper influxDBMapper = new InfluxDBMapper(influxDB); - Cpu firstCore = new Cpu(Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123l); - influxDBMapper.save(firstCore); - Cpu secondCore = new Cpu(Instant.now().minusSeconds(500),"local_2","eu-west-2",1.4d,true,123l); - influxDBMapper.save(secondCore); - } + public static void populateDatabase(InfluxDB influxDB) { + InfluxDBMapper influxDBMapper = new InfluxDBMapper(influxDB); + Cpu firstCore = + new Cpu(Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123l); + influxDBMapper.save(firstCore); + Cpu secondCore = + new Cpu(Instant.now().minusSeconds(500), "local_2", "eu-west-2", 1.4d, true, 123l); + influxDBMapper.save(secondCore); + } - public static void cleanDatabase(InfluxDB influxDB) { - influxDB.query(new Query("DROP MEASUREMENT cpu", DATABASE_NAME)); - } + public static void cleanDatabase(InfluxDB influxDB) { + influxDB.query(new Query("DROP MEASUREMENT cpu", DATABASE_NAME)); + } - public static void dropDatabase(InfluxDB influxDB) { - influxDB.query(new Query("DROP DATABASE "+DATABASE_NAME)); - } + public static void dropDatabase(InfluxDB influxDB) { + influxDB.query(new Query("DROP DATABASE " + DATABASE_NAME)); + } - public static Point resultToPoint(QueryResult.Series serie, List values) { - Point.Builder builder = Point.measurement(serie.getName()); + public static Point resultToPoint(QueryResult.Series serie, List values) { + Point.Builder builder = Point.measurement(serie.getName()); - for(int i = 0; i < serie.getColumns().size(); i++) { - String column = serie.getColumns().get(i); - Object value = values.get(i); + for (int i = 0; i < serie.getColumns().size(); i++) { + String column = serie.getColumns().get(i); + Object value = values.get(i); - if(column.equals("time")) { - builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS); - } else if(column.equals("host") || column.equals("region")) { - builder.tag(column,value.toString()); - } else if(column.equals("uptimesecs")) { - builder.addField(column,((Double) value).longValue()); - } else { - if(value instanceof Long) builder.addField(column,(Long) value); - else if(value instanceof Double) builder.addField(column, (Double) value); - else if(value instanceof Number) builder.addField(column, (Number) value); - else if(value instanceof String) builder.addField(column, (String) value); - else if(value instanceof Boolean) builder.addField(column, (Boolean) value); - } + if (column.equals("time")) { + builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS); + } else if (column.equals("host") || column.equals("region")) { + builder.tag(column, value.toString()); + } else if (column.equals("uptimesecs")) { + builder.addField(column, ((Double) value).longValue()); + } else { + if (value instanceof Long) builder.addField(column, (Long) value); + else if (value instanceof Double) builder.addField(column, (Double) value); + else if (value instanceof Number) builder.addField(column, (Number) value); + else if (value instanceof String) builder.addField(column, (String) value); + else if (value instanceof Boolean) builder.addField(column, (Boolean) value); } - - return builder.build(); } + return builder.build(); + } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 85f3af6445..39be32f89c 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.scaladsl import akka.actor.ActorSystem @@ -13,34 +17,35 @@ import docs.javadsl.TestUtils._ import docs.javadsl.TestConstants._ import org.influxdb.dto.Query -class InfluxDBSourceSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { +class InfluxDBSourceSpec + extends WordSpec + with MustMatchers + with BeforeAndAfterEach + with BeforeAndAfterAll + with ScalaFutures { implicit val system = ActorSystem() implicit val mat = ActorMaterializer() implicit var influxDB: InfluxDB = _ - override protected def beforeAll(): Unit = { + override protected def beforeAll(): Unit = influxDB = setupConnection() - } - override protected def afterAll(): Unit = { + override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) - } - override def beforeEach(): Unit = { + override def beforeEach(): Unit = populateDatabase(influxDB) - } - override def afterEach() = { + override def afterEach() = cleanDatabase(influxDB) - } "support source" in assertAllStagesStopped { - // #run-typed + // #run-typed val query = new Query("SELECT*FROM cpu", DATABASE_NAME); - val influxDBResult = InfluxDBSource(influxDB ,query).runWith(Sink.seq) + val influxDBResult = InfluxDBSource(influxDB, query).runWith(Sink.seq) val resultToAssert = influxDBResult.futureValue.head val values = resultToAssert.getResults.get(0).getSeries().get(0).getValues diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index 3a3ea4ced0..4476368394 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.scaladsl import java.util.concurrent.TimeUnit @@ -20,7 +24,6 @@ import akka.stream.scaladsl.Sink import scala.collection.JavaConverters._ - class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { implicit val system = ActorSystem() @@ -28,25 +31,21 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi implicit var influxDB: InfluxDB = _ - override protected def beforeAll(): Unit = { + override protected def beforeAll(): Unit = influxDB = setupConnection() - } - override protected def afterAll(): Unit = { + override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) - } - override def beforeEach(): Unit = { + override def beforeEach(): Unit = populateDatabase(influxDB) - } - override def afterEach() = { + override def afterEach() = cleanDatabase(influxDB) - } "support typed source" in assertAllStagesStopped { val query = new Query("SELECT*FROM cpu", DATABASE_NAME); - val measurements = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(),influxDB, query).runWith(Sink.seq) + val measurements = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) measurements.futureValue.map(_.getHostname) mustBe List("local_1", "local_2") } @@ -56,13 +55,15 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "consume and publish measurements using typed" in assertAllStagesStopped { val query = new Query("SELECT*FROM cpu", DATABASE_NAME); - val f1 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query) - .map { - cpu: Cpu => { + val f1 = InfluxDBSource + .typed(classOf[Cpu], InfluxDBSettings(), influxDB, query) + .map { cpu: Cpu => + { val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) InfluxDBWriteMessage(clonedCpu) } - }.runWith(InfluxDBSink.typed(classOf[Cpu], InfluxDBSettings())) + } + .runWith(InfluxDBSink.typed(classOf[Cpu], InfluxDBSettings())) f1.futureValue mustBe Done @@ -74,33 +75,32 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "consume and publish measurements" in assertAllStagesStopped { val query = new Query("SELECT*FROM cpu", DATABASE_NAME); - val f1 = InfluxDBSource(influxDB, query) - .map { - queryReqult => resultToPoints(queryReqult) + .map { queryReqult => + resultToPoints(queryReqult) } .mapConcat(identity) .runWith(InfluxDBSink(InfluxDBSettings())) f1.futureValue mustBe Done - val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(),influxDB, query).runWith(Sink.seq) + val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } - def resultToPoints(queryResult: QueryResult) : List[InfluxDBWriteMessage[Point,NotUsed]] = { + def resultToPoints(queryResult: QueryResult): List[InfluxDBWriteMessage[Point, NotUsed]] = { val points = for { results <- queryResult.getResults.asScala serie <- results.getSeries.asScala values <- serie.getValues.asScala - } yield ( - InfluxDBWriteMessage(resultToPoint(serie, values)) + } yield + ( + InfluxDBWriteMessage(resultToPoint(serie, values)) ) points.toList } - } } From 8ccb0f8a75e6d972eeff9eb91c6588f570d53386 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 6 May 2019 10:17:25 +0100 Subject: [PATCH 21/89] InfluxDB connector disabled MimaPlugin --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 7cc58aac1a..a130470238 100644 --- a/build.sbt +++ b/build.sbt @@ -179,7 +179,7 @@ lazy val hbase = alpakkaProject("hbase", "hbase", Dependencies.HBase, fork in Te lazy val hdfs = alpakkaProject("hdfs", "hdfs", Dependencies.Hdfs, parallelExecution in Test := false) -lazy val influxdb = alpakkaProject("influxdb", "influxdb", Dependencies.InfluxDB) +lazy val influxdb = alpakkaProject("influxdb", "influxdb", Dependencies.InfluxDB).disablePlugins(MimaPlugin) lazy val ironmq = alpakkaProject("ironmq", "ironmq", From 6883fa2de03d2eb79b3ec59fe0453cfe88bae7b1 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 6 May 2019 16:26:20 +0100 Subject: [PATCH 22/89] InfluxDB connector added test to travis --- .travis.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.travis.yml b/.travis.yml index 56dcd6a692..4b082739d1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -73,6 +73,9 @@ jobs: - PRE_CMD="docker-compose up -d hbase" - env: - DIR=hdfs + - env: + - DIR=influxdb + - PRE_CMD="docker-compose up -d influxdb" - env: - DIR=ironmq - PRE_CMD="docker-compose up -d ironauth ironmq" From 25e817177c36d31cb27d7d1aa871aca69be5b5e1 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 6 May 2019 18:38:00 +0100 Subject: [PATCH 23/89] InfluxDB connector added test to travis --- influxdb/src/test/java/docs/javadsl/Cpu.java | 14 --------- .../test/java/docs/javadsl/InfluxDBCpu.java | 30 +++++++++++++++++++ .../java/docs/javadsl/InfluxDBSourceCpu.java | 25 ++++++++++++++++ .../java/docs/javadsl/InfluxDBSourceTest.java | 17 ++++++----- .../test/java/docs/javadsl/InfluxDBTest.java | 18 ++++++----- .../test/java/docs/javadsl/TestConstants.java | 2 -- .../src/test/java/docs/javadsl/TestUtils.java | 30 +++++++++++-------- 7 files changed, 91 insertions(+), 45 deletions(-) create mode 100644 influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java create mode 100644 influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java diff --git a/influxdb/src/test/java/docs/javadsl/Cpu.java b/influxdb/src/test/java/docs/javadsl/Cpu.java index ef09bb4785..2c0e93321d 100644 --- a/influxdb/src/test/java/docs/javadsl/Cpu.java +++ b/influxdb/src/test/java/docs/javadsl/Cpu.java @@ -5,18 +5,8 @@ package docs.javadsl; import java.time.Instant; -import java.util.concurrent.TimeUnit; - import org.influxdb.annotation.Column; -import org.influxdb.annotation.Measurement; - -import static docs.javadsl.TestConstants.DATABASE_NAME; -@Measurement( - name = "cpu", - database = DATABASE_NAME, - retentionPolicy = "autogen", - timeUnit = TimeUnit.MILLISECONDS) public class Cpu { @Column(name = "time") @@ -77,8 +67,4 @@ public Boolean getHappydevop() { public Long getUptimeSecs() { return uptimeSecs; } - - public Cpu cloneAt(Instant time) { - return new Cpu(time, hostname, region, idle, happydevop, uptimeSecs); - } } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java b/influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java new file mode 100644 index 0000000000..47a7c9393a --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package docs.javadsl; + +import java.time.Instant; + +import org.influxdb.annotation.Measurement; + +@Measurement(name = "cpu", database = "InfluxDBTest") +public class InfluxDBCpu extends Cpu { + + public InfluxDBCpu() {} + + public InfluxDBCpu( + Instant time, + String hostname, + String region, + Double idle, + Boolean happydevop, + Long uptimeSecs) { + super(time, hostname, region, idle, happydevop, uptimeSecs); + } + + public InfluxDBCpu cloneAt(Instant time) { + return new InfluxDBCpu( + time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + } +} diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java new file mode 100644 index 0000000000..31df09d95c --- /dev/null +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package docs.javadsl; + +import java.time.Instant; + +import org.influxdb.annotation.Measurement; + +@Measurement(name = "cpu", database = "InfluxDBSourceTest") +public class InfluxDBSourceCpu extends Cpu { + + public InfluxDBSourceCpu() {} + + public InfluxDBSourceCpu( + Instant time, + String hostname, + String region, + Double idle, + Boolean happydevop, + Long uptimeSecs) { + super(time, hostname, region, idle, happydevop, uptimeSecs); + } +} diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java index 2653fc3afa..44c1143dfd 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java @@ -26,7 +26,6 @@ import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; import akka.testkit.javadsl.TestKit; -import static docs.javadsl.TestConstants.DATABASE_NAME; import static docs.javadsl.TestUtils.cleanDatabase; import static docs.javadsl.TestUtils.dropDatabase; import static docs.javadsl.TestUtils.populateDatabase; @@ -38,6 +37,8 @@ public class InfluxDBSourceTest { private static Materializer materializer; private static InfluxDB influxDB; + private static final String DATABASE_NAME = "InfluxDBSourceTest"; + private static Pair setupMaterializer() { // #init-mat final ActorSystem system = ActorSystem.create(); @@ -52,23 +53,23 @@ public static void setupDatabase() { system = sysmat.first(); materializer = sysmat.second(); - influxDB = setupConnection(); + influxDB = setupConnection(DATABASE_NAME); } @AfterClass public static void teardown() { - dropDatabase(influxDB); + dropDatabase(influxDB, DATABASE_NAME); TestKit.shutdownActorSystem(system); } @Before public void setUp() throws Exception { - populateDatabase(influxDB); + populateDatabase(influxDB, InfluxDBSourceCpu.class); } @After public void cleanUp() { - cleanDatabase(influxDB); + cleanDatabase(influxDB, DATABASE_NAME); StreamTestKit.assertAllStagesStopped(materializer); } @@ -76,11 +77,11 @@ public void cleanUp() { public void streamQueryResult() throws Exception { Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); - CompletionStage> rows = - InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + CompletionStage> rows = + InfluxDBSource.typed(InfluxDBSourceCpu.class, InfluxDBSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); - List cpus = rows.toCompletableFuture().get(); + List cpus = rows.toCompletableFuture().get(); Assert.assertEquals(2, cpus.size()); } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java index 73fec964a1..51584a1a46 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -32,7 +32,6 @@ import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; import akka.testkit.javadsl.TestKit; -import static docs.javadsl.TestConstants.DATABASE_NAME; import static docs.javadsl.TestUtils.cleanDatabase; import static docs.javadsl.TestUtils.dropDatabase; import static docs.javadsl.TestUtils.populateDatabase; @@ -45,6 +44,8 @@ public class InfluxDBTest { private static Materializer materializer; private static InfluxDB influxDB; + private static final String DATABASE_NAME = "InfluxDBTest"; + private static Pair setupMaterializer() { // #init-mat final ActorSystem system = ActorSystem.create(); @@ -59,23 +60,23 @@ public static void setupDatabase() { system = sysmat.first(); materializer = sysmat.second(); - influxDB = setupConnection(); + influxDB = setupConnection(DATABASE_NAME); } @AfterClass public static void teardown() { - dropDatabase(influxDB); + dropDatabase(influxDB, DATABASE_NAME); TestKit.shutdownActorSystem(system); } @Before public void setUp() throws Exception { - populateDatabase(influxDB); + populateDatabase(influxDB, InfluxDBCpu.class); } @After public void cleanUp() { - cleanDatabase(influxDB); + cleanDatabase(influxDB, DATABASE_NAME); StreamTestKit.assertAllStagesStopped(materializer); } @@ -83,14 +84,15 @@ public void cleanUp() { public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); CompletionStage completionStage = - InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + InfluxDBSource.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB, query) .map( cpu -> { - Cpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); + InfluxDBCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); return new InfluxDBWriteMessage<>(clonedCpu, NotUsed.notUsed()); }) .runWith( - InfluxDBSink.typed(Cpu.class, InfluxDBSettings.Default(), influxDB), materializer); + InfluxDBSink.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB), + materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); diff --git a/influxdb/src/test/java/docs/javadsl/TestConstants.java b/influxdb/src/test/java/docs/javadsl/TestConstants.java index ea744cc1eb..f4283dca47 100644 --- a/influxdb/src/test/java/docs/javadsl/TestConstants.java +++ b/influxdb/src/test/java/docs/javadsl/TestConstants.java @@ -9,6 +9,4 @@ public class TestConstants { public static final String INFLUXDB_URL = "http://localhost:8086"; public static final String USERNAME = "root"; public static final String PASSWORD = "root"; - - public static final String DATABASE_NAME = "influxdb_stream_scala_test"; } diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index 9a682e141e..a9713450dd 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -4,6 +4,7 @@ package docs.javadsl; +import java.lang.reflect.Constructor; import java.time.Instant; import java.util.List; import java.util.concurrent.TimeUnit; @@ -15,37 +16,40 @@ import org.influxdb.dto.QueryResult; import org.influxdb.impl.InfluxDBMapper; -import static docs.javadsl.TestConstants.DATABASE_NAME; import static docs.javadsl.TestConstants.INFLUXDB_URL; import static docs.javadsl.TestConstants.PASSWORD; import static docs.javadsl.TestConstants.USERNAME; public class TestUtils { - public static InfluxDB setupConnection() { + public static InfluxDB setupConnection(final String databaseName) { final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); - influxDB.setDatabase(DATABASE_NAME); + influxDB.setDatabase(databaseName); influxDB.setLogLevel(InfluxDB.LogLevel.FULL); - influxDB.query(new Query("CREATE DATABASE " + DATABASE_NAME, DATABASE_NAME)); + influxDB.query(new Query("CREATE DATABASE " + databaseName, databaseName)); return influxDB; } - public static void populateDatabase(InfluxDB influxDB) { + public static void populateDatabase(InfluxDB influxDB, Class clazz) throws Exception { InfluxDBMapper influxDBMapper = new InfluxDBMapper(influxDB); - Cpu firstCore = - new Cpu(Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123l); + Constructor cons = + clazz.getConstructor( + Instant.class, String.class, String.class, Double.class, Boolean.class, Long.class); + Object firstCore = + cons.newInstance( + Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123l); influxDBMapper.save(firstCore); - Cpu secondCore = - new Cpu(Instant.now().minusSeconds(500), "local_2", "eu-west-2", 1.4d, true, 123l); + Object secondCore = + cons.newInstance(Instant.now().minusSeconds(500), "local_2", "eu-west-2", 1.4d, true, 123l); influxDBMapper.save(secondCore); } - public static void cleanDatabase(InfluxDB influxDB) { - influxDB.query(new Query("DROP MEASUREMENT cpu", DATABASE_NAME)); + public static void cleanDatabase(final InfluxDB influxDB, final String databaseName) { + influxDB.query(new Query("DROP MEASUREMENT cpu", databaseName)); } - public static void dropDatabase(InfluxDB influxDB) { - influxDB.query(new Query("DROP DATABASE " + DATABASE_NAME)); + public static void dropDatabase(InfluxDB influxDB, final String databaseName) { + influxDB.query(new Query("DROP DATABASE " + databaseName)); } public static Point resultToPoint(QueryResult.Series serie, List values) { From ada72bb5b3decf31fcdc87e5dd51da2a5139bd9a Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 6 May 2019 18:38:54 +0100 Subject: [PATCH 24/89] InfluxDB connector added test to travis --- .../docs/scaladsl/InfluxDBSourceCpu.java | 25 +++++++++++++++ .../docs/scaladsl/InfluxDBSourceSpec.scala | 10 +++--- .../scala/docs/scaladsl/InfluxDBSpec.scala | 31 +++++++++---------- .../scala/docs/scaladsl/InfluxDBSpecCpu.java | 25 +++++++++++++++ 4 files changed, 71 insertions(+), 20 deletions(-) create mode 100644 influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java create mode 100644 influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java new file mode 100644 index 0000000000..5ae1664d2c --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package docs.scaladsl; + +import java.time.Instant; + +import org.influxdb.annotation.Measurement; + +import docs.javadsl.Cpu; + +@Measurement(name = "cpu", database = "InfluxDBSourceSpec") +public class InfluxDBSourceCpu extends Cpu { + public InfluxDBSourceCpu() { + } + + public InfluxDBSourceCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + super(time, hostname, region, idle, happydevop, uptimeSecs); + } + + public InfluxDBSpecCpu cloneAt(Instant time) { + return new InfluxDBSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + } +} diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 39be32f89c..8eb1aa6719 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -24,26 +24,28 @@ class InfluxDBSourceSpec with BeforeAndAfterAll with ScalaFutures { + val databaseName = "InfluxDBSourceSpec" + implicit val system = ActorSystem() implicit val mat = ActorMaterializer() implicit var influxDB: InfluxDB = _ override protected def beforeAll(): Unit = - influxDB = setupConnection() + influxDB = setupConnection(databaseName) override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) override def beforeEach(): Unit = - populateDatabase(influxDB) + populateDatabase(influxDB, classOf[InfluxDBSourceCpu]) override def afterEach() = - cleanDatabase(influxDB) + cleanDatabase(influxDB, databaseName) "support source" in assertAllStagesStopped { // #run-typed - val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + val query = new Query("SELECT*FROM cpu", databaseName); val influxDBResult = InfluxDBSource(influxDB, query).runWith(Sink.seq) val resultToAssert = influxDBResult.futureValue.head diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index 4476368394..d6dd1850b8 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -4,8 +4,6 @@ package docs.scaladsl -import java.util.concurrent.TimeUnit - import akka.actor.ActorSystem import akka.stream.ActorMaterializer import org.influxdb.InfluxDB @@ -13,12 +11,10 @@ import org.influxdb.dto.{Point, Query, QueryResult} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped -import docs.javadsl.TestConstants.DATABASE_NAME import akka.{Done, NotUsed} import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage} import akka.stream.alpakka.influxdb.scaladsl.{InfluxDBSink, InfluxDBSource} import akka.testkit.TestKit -import docs.javadsl.Cpu import docs.javadsl.TestUtils._ import akka.stream.scaladsl.Sink @@ -29,23 +25,26 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi implicit val system = ActorSystem() implicit val mat = ActorMaterializer() + val databaseName = this.getClass.getSimpleName + implicit var influxDB: InfluxDB = _ override protected def beforeAll(): Unit = - influxDB = setupConnection() + influxDB = setupConnection(databaseName) override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) override def beforeEach(): Unit = - populateDatabase(influxDB) + populateDatabase(influxDB, classOf[InfluxDBSpecCpu]) override def afterEach() = - cleanDatabase(influxDB) + cleanDatabase(influxDB, databaseName) "support typed source" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DATABASE_NAME); - val measurements = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + val query = new Query("SELECT*FROM cpu", databaseName); + val measurements = + InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) measurements.futureValue.map(_.getHostname) mustBe List("local_1", "local_2") } @@ -53,27 +52,27 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "InfluxDBFlow" should { "consume and publish measurements using typed" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + val query = new Query("SELECT*FROM cpu", databaseName); val f1 = InfluxDBSource - .typed(classOf[Cpu], InfluxDBSettings(), influxDB, query) - .map { cpu: Cpu => + .typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query) + .map { cpu: InfluxDBSpecCpu => { val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) InfluxDBWriteMessage(clonedCpu) } } - .runWith(InfluxDBSink.typed(classOf[Cpu], InfluxDBSettings())) + .runWith(InfluxDBSink.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings())) f1.futureValue mustBe Done - val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + val f2 = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } "consume and publish measurements" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DATABASE_NAME); + val query = new Query("SELECT*FROM cpu", databaseName); val f1 = InfluxDBSource(influxDB, query) .map { queryReqult => @@ -84,7 +83,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi f1.futureValue mustBe Done - val f2 = InfluxDBSource.typed(classOf[Cpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + val f2 = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java new file mode 100644 index 0000000000..3e0ac57e40 --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java @@ -0,0 +1,25 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package docs.scaladsl; + +import java.time.Instant; + +import org.influxdb.annotation.Measurement; + +import docs.javadsl.Cpu; + +@Measurement(name = "cpu", database = "InfluxDBSpec") +public class InfluxDBSpecCpu extends Cpu { + public InfluxDBSpecCpu() { + } + + public InfluxDBSpecCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + super(time, hostname, region, idle, happydevop, uptimeSecs); + } + + public InfluxDBSpecCpu cloneAt(Instant time) { + return new InfluxDBSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + } +} From b165f258d72a50a193b0799efa8d70ebbd9bf71f Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 6 May 2019 19:02:05 +0100 Subject: [PATCH 25/89] Removed println --- .../akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 06baa4e380..51983c6144 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -61,7 +61,6 @@ private[influxdb] class InfluxDBFlowStage[T, C]( messages.foreach { case InfluxDBWriteMessage(point: Point, _) => { influxDB.write(point) - println(point.toString) } case InfluxDBWriteMessage(others: AnyRef, _) => failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) From d7f10b8193672bb5df1c12717d835274886a0c22 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 21:42:10 +0100 Subject: [PATCH 26/89] Changed typos on spec for InfluxDB connector --- .../src/test/scala/docs/scaladsl/InfluxDBSpec.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index d6dd1850b8..af042d1122 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -75,10 +75,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi val query = new Query("SELECT*FROM cpu", databaseName); val f1 = InfluxDBSource(influxDB, query) - .map { queryReqult => - resultToPoints(queryReqult) - } - .mapConcat(identity) + .mapConcat(resultToPoints) .runWith(InfluxDBSink(InfluxDBSettings())) f1.futureValue mustBe Done @@ -91,11 +88,11 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi def resultToPoints(queryResult: QueryResult): List[InfluxDBWriteMessage[Point, NotUsed]] = { val points = for { results <- queryResult.getResults.asScala - serie <- results.getSeries.asScala - values <- serie.getValues.asScala + series <- results.getSeries.asScala + values <- series.getValues.asScala } yield ( - InfluxDBWriteMessage(resultToPoint(serie, values)) + InfluxDBWriteMessage(resultToPoint(series, values)) ) points.toList } From 4ca0611369b90ceed8f1b45d37c002f73f6239a1 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 21:58:36 +0100 Subject: [PATCH 27/89] Changed to scala constants for InfluxDB connector --- .../scala/docs/scaladsl/InfluxDBSourceSpec.scala | 8 ++++---- .../src/test/scala/docs/scaladsl/InfluxDBSpec.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 8eb1aa6719..67aa478632 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -24,7 +24,7 @@ class InfluxDBSourceSpec with BeforeAndAfterAll with ScalaFutures { - val databaseName = "InfluxDBSourceSpec" + val DatabaseName = "InfluxDBSourceSpec" implicit val system = ActorSystem() implicit val mat = ActorMaterializer() @@ -32,7 +32,7 @@ class InfluxDBSourceSpec implicit var influxDB: InfluxDB = _ override protected def beforeAll(): Unit = - influxDB = setupConnection(databaseName) + influxDB = setupConnection(DatabaseName) override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) @@ -41,11 +41,11 @@ class InfluxDBSourceSpec populateDatabase(influxDB, classOf[InfluxDBSourceCpu]) override def afterEach() = - cleanDatabase(influxDB, databaseName) + cleanDatabase(influxDB, DatabaseName) "support source" in assertAllStagesStopped { // #run-typed - val query = new Query("SELECT*FROM cpu", databaseName); + val query = new Query("SELECT*FROM cpu", DatabaseName); val influxDBResult = InfluxDBSource(influxDB, query).runWith(Sink.seq) val resultToAssert = influxDBResult.futureValue.head diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index af042d1122..0399c04998 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -25,12 +25,12 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi implicit val system = ActorSystem() implicit val mat = ActorMaterializer() - val databaseName = this.getClass.getSimpleName + val DatabaseName = this.getClass.getSimpleName implicit var influxDB: InfluxDB = _ override protected def beforeAll(): Unit = - influxDB = setupConnection(databaseName) + influxDB = setupConnection(DatabaseName) override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) @@ -39,10 +39,10 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi populateDatabase(influxDB, classOf[InfluxDBSpecCpu]) override def afterEach() = - cleanDatabase(influxDB, databaseName) + cleanDatabase(influxDB, DatabaseName) "support typed source" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", databaseName); + val query = new Query("SELECT*FROM cpu", DatabaseName); val measurements = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) @@ -52,7 +52,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "InfluxDBFlow" should { "consume and publish measurements using typed" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", databaseName); + val query = new Query("SELECT*FROM cpu", DatabaseName); val f1 = InfluxDBSource .typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query) @@ -72,7 +72,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi } "consume and publish measurements" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", databaseName); + val query = new Query("SELECT*FROM cpu", DatabaseName); val f1 = InfluxDBSource(influxDB, query) .mapConcat(resultToPoints) From e3560557ee604c3a1c3fbd5b51ceec0bf7ee72d2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 21:59:52 +0100 Subject: [PATCH 28/89] Add final to constants for InfluxDB connector --- influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala | 2 +- influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 67aa478632..13da5529d5 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -24,7 +24,7 @@ class InfluxDBSourceSpec with BeforeAndAfterAll with ScalaFutures { - val DatabaseName = "InfluxDBSourceSpec" + final val DatabaseName = "InfluxDBSourceSpec" implicit val system = ActorSystem() implicit val mat = ActorMaterializer() diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index 0399c04998..6761d9b024 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -25,7 +25,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi implicit val system = ActorSystem() implicit val mat = ActorMaterializer() - val DatabaseName = this.getClass.getSimpleName + final val DatabaseName = this.getClass.getSimpleName implicit var influxDB: InfluxDB = _ From 66988af40bf401dc2d40019a538665bd82c90da2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:02:01 +0100 Subject: [PATCH 29/89] Fixed typo for InfluxDB connector --- influxdb/src/test/java/docs/javadsl/TestUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index a9713450dd..601eae864d 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -52,11 +52,11 @@ public static void dropDatabase(InfluxDB influxDB, final String databaseName) { influxDB.query(new Query("DROP DATABASE " + databaseName)); } - public static Point resultToPoint(QueryResult.Series serie, List values) { - Point.Builder builder = Point.measurement(serie.getName()); + public static Point resultToPoint(QueryResult.Series series, List values) { + Point.Builder builder = Point.measurement(series.getName()); - for (int i = 0; i < serie.getColumns().size(); i++) { - String column = serie.getColumns().get(i); + for (int i = 0; i < series.getColumns().size(); i++) { + String column = series.getColumns().get(i); Object value = values.get(i); if (column.equals("time")) { From 4f039ca46b315081280ce2a4ae6c7e59cb66522e Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:04:18 +0100 Subject: [PATCH 30/89] Fixed typo for InfluxDB connector --- influxdb/src/test/java/docs/javadsl/InfluxDBTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java index 51584a1a46..6b862a678c 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -130,10 +130,10 @@ private List> points(QueryResult queryResul List> points = new ArrayList<>(); for (QueryResult.Result result : queryResult.getResults()) { - for (QueryResult.Series serie : result.getSeries()) { - for (List rows : serie.getValues()) { + for (QueryResult.Series series : result.getSeries()) { + for (List rows : series.getValues()) { InfluxDBWriteMessage influxDBWriteMessage = - new InfluxDBWriteMessage<>(resultToPoint(serie, rows), NotUsed.notUsed()); + new InfluxDBWriteMessage<>(resultToPoint(series, rows), NotUsed.notUsed()); points.add(influxDBWriteMessage); } } From 3b5bbda0d69faedc27ea66936418b219228a03eb Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:08:28 +0100 Subject: [PATCH 31/89] Added spaces on select statements for InfluxDB connector --- influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java | 4 ++-- influxdb/src/test/java/docs/javadsl/InfluxDBTest.java | 4 ++-- .../src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala | 2 +- influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala | 6 +++--- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java index 44c1143dfd..69d501e50c 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java @@ -75,7 +75,7 @@ public void cleanUp() { @Test public void streamQueryResult() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage> rows = InfluxDBSource.typed(InfluxDBSourceCpu.class, InfluxDBSettings.Default(), influxDB, query) @@ -88,7 +88,7 @@ public void streamQueryResult() throws Exception { @Test public void streamRawQueryResult() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage> completionStage = InfluxDBSource.create(influxDB, query).runWith(Sink.seq(), materializer); diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java index 6b862a678c..023b4239bf 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -82,7 +82,7 @@ public void cleanUp() { @Test public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = InfluxDBSource.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB, query) .map( @@ -105,7 +105,7 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { @Test public void testConsumeAndPublishMeasurements() throws Exception { - Query query = new Query("SELECT*FROM cpu", DATABASE_NAME); + Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = InfluxDBSource.create(influxDB, query) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 13da5529d5..311ba48772 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -45,7 +45,7 @@ class InfluxDBSourceSpec "support source" in assertAllStagesStopped { // #run-typed - val query = new Query("SELECT*FROM cpu", DatabaseName); + val query = new Query("SELECT * FROM cpu", DatabaseName); val influxDBResult = InfluxDBSource(influxDB, query).runWith(Sink.seq) val resultToAssert = influxDBResult.futureValue.head diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala index 6761d9b024..962a7d2938 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala @@ -42,7 +42,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi cleanDatabase(influxDB, DatabaseName) "support typed source" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DatabaseName); + val query = new Query("SELECT * FROM cpu", DatabaseName); val measurements = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) @@ -52,7 +52,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "InfluxDBFlow" should { "consume and publish measurements using typed" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DatabaseName); + val query = new Query("SELECT * FROM cpu", DatabaseName); val f1 = InfluxDBSource .typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query) @@ -72,7 +72,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi } "consume and publish measurements" in assertAllStagesStopped { - val query = new Query("SELECT*FROM cpu", DatabaseName); + val query = new Query("SELECT * FROM cpu", DatabaseName); val f1 = InfluxDBSource(influxDB, query) .mapConcat(resultToPoints) From 58209e2f39bf372fd512522253c4fb9a88b18022 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:16:23 +0100 Subject: [PATCH 32/89] Changed to hostname for InfluxDB connector --- influxdb/src/test/java/docs/javadsl/Cpu.java | 2 +- influxdb/src/test/java/docs/javadsl/TestUtils.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/influxdb/src/test/java/docs/javadsl/Cpu.java b/influxdb/src/test/java/docs/javadsl/Cpu.java index 2c0e93321d..f36ff63319 100644 --- a/influxdb/src/test/java/docs/javadsl/Cpu.java +++ b/influxdb/src/test/java/docs/javadsl/Cpu.java @@ -12,7 +12,7 @@ public class Cpu { @Column(name = "time") private Instant time; - @Column(name = "host", tag = true) + @Column(name = "hostname", tag = true) private String hostname; @Column(name = "region", tag = true) diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index 601eae864d..38de4db8d0 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -61,7 +61,7 @@ public static Point resultToPoint(QueryResult.Series series, List values if (column.equals("time")) { builder.time(System.currentTimeMillis(), TimeUnit.MILLISECONDS); - } else if (column.equals("host") || column.equals("region")) { + } else if (column.equals("hostname") || column.equals("region")) { builder.tag(column, value.toString()); } else if (column.equals("uptimesecs")) { builder.addField(column, ((Double) value).longValue()); From 14a4cfe669d3ed2ced5a8efea19ce28f44a354f6 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:22:00 +0100 Subject: [PATCH 33/89] Switched settings class to case for InfluxDB connector --- .../scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala index 5d9db02cc2..f468e00879 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -15,7 +15,7 @@ object InfluxDBSettings { } -final class InfluxDBSettings private ( +final case class InfluxDBSettings private ( val batchSize: Int, val precision: TimeUnit ) { From 501d70838d8141a0a7a6aaeba444bd800797a43a Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:39:17 +0100 Subject: [PATCH 34/89] Fixed reference spacing for InfluxDB connector --- influxdb/src/main/resources/reference.conf | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/influxdb/src/main/resources/reference.conf b/influxdb/src/main/resources/reference.conf index ff71f4b780..659b42b8a1 100644 --- a/influxdb/src/main/resources/reference.conf +++ b/influxdb/src/main/resources/reference.conf @@ -4,6 +4,15 @@ akka.stream.alpakka.influxdb{ batchSize = 10 # The InfluxDB write precision - precision = MILLISECONDS + precision = MILLISECONDS + + url = "" + + # credentials { + # username = "root" + # password = "root" + # } + + batchedWrites = true } From eabe031bb7154d0bfc8482ff1288aa1bb91d3b32 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:50:25 +0100 Subject: [PATCH 35/89] Changed option to Some/None for InfluxDB connector --- .../influxdb/impl/InfluxDBSourceStage.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 3bf06d2d84..70cc6fce87 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -108,21 +108,22 @@ private[influxdb] final class InfluxDBSourceRawLogic(query: Query, setHandler(outlet, this) - var dataRetrieved: Option[QueryResult] = Option.empty + var dataRetrieved: Option[QueryResult] = None override def preStart(): Unit = { val queryResult = influxDB.query(query) if (!queryResult.hasError) { - dataRetrieved = Option(queryResult) + dataRetrieved = Some(queryResult) } } override def onPull(): Unit = - if (dataRetrieved.isEmpty) { - completeStage() - } else { - emit(outlet, dataRetrieved.get) - dataRetrieved = Option.empty + dataRetrieved match { + case None => completeStage() + case Some(queryResult) => { + emit(outlet, queryResult) + dataRetrieved = None + } } } From 9ba52633201c5aeb938ee321ad10bf8df0ffc9b5 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 22:56:27 +0100 Subject: [PATCH 36/89] Changed to (_ :+ _) from (seq, wm) => seq :+ wm on InfluxDB connector --- .../stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index 3054745406..7ee3143d24 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -20,7 +20,7 @@ object InfluxDBFlow { implicit influxDB: InfluxDB ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) .via( new impl.InfluxDBFlowStage[T, NotUsed]( Option.empty, @@ -33,7 +33,7 @@ object InfluxDBFlow { implicit influxDB: InfluxDB ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) .via( new impl.InfluxDBFlowStage[T, NotUsed]( Option(clazz), @@ -46,7 +46,7 @@ object InfluxDBFlow { implicit influxDB: InfluxDB ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] - .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) .via( new impl.InfluxDBFlowStage[T, C]( Option.empty, @@ -59,7 +59,7 @@ object InfluxDBFlow { implicit influxDB: InfluxDB ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] - .batch(settings.batchSize, immutable.Seq(_)) { case (seq, wm) => seq :+ wm } + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) .via( new impl.InfluxDBFlowStage[T, C]( Option(clazz), From 54c09c88513bc92f38177906947ec8675b629999 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 23:05:17 +0100 Subject: [PATCH 37/89] Added imports to clean flow for InfluxDB connector --- .../influxdb/javadsl/InfluxDBFlow.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala index 3d5d79920d..9849cf07df 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala @@ -7,46 +7,48 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} import org.influxdb.InfluxDB +import akka.stream.javadsl.Flow +import akka.stream.alpakka.influxdb.scaladsl object InfluxDBFlow { def create[T]( settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.create(settings)(influxDB).asJava + ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + scaladsl.InfluxDBFlow.create(settings)(influxDB).asJava def typed[T]( clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typed(clazz, settings)(influxDB).asJava + ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = + scaladsl.InfluxDBFlow.typed(clazz, settings)(influxDB).asJava def createWithPassThrough[T, C]( settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithPassThrough(settings)(influxDB).asJava + ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + scaladsl.InfluxDBFlow.createWithPassThrough(settings)(influxDB).asJava def typedWithPassThrough[T, C]( clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava + ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = + scaladsl.InfluxDBFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava def createWithContext[T, C]( settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.createWithContext(settings)(influxDB).asJava + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + scaladsl.InfluxDBFlow.createWithContext(settings)(influxDB).asJava def typedWithContext[T, C]( clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow.typedWithContext(clazz, settings)(influxDB).asJava + ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = + scaladsl.InfluxDBFlow.typedWithContext(clazz, settings)(influxDB).asJava } From 308a64d5845a756bf7384e9e079dfb2872aafc8f Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 23:12:55 +0100 Subject: [PATCH 38/89] Changed Option.empty() to None and used match cases for InfluxDB connector --- .../influxdb/impl/InfluxDBFlowStage.scala | 2 +- .../influxdb/impl/InfluxDBSourceStage.scala | 31 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 51983c6144..9465061985 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -46,7 +46,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( influxDB.enableBatch(BatchOptions.DEFAULTS) write(messages) - val writtenMessages = messages.map(m => new InfluxDBWriteResult(m, Option.empty)) + val writtenMessages = messages.map(m => new InfluxDBWriteResult(m, None)) influxDB.close() push(out, writtenMessages) } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 70cc6fce87..8fed50e2e5 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -49,7 +49,7 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], setHandler(outlet, this) - var dataRetrieved: Option[QueryResult] = Option.empty + var dataRetrieved: Option[QueryResult] = None var resultMapperHelper: InfluxDBResultMapperHelper = _ override def preStart(): Unit = { @@ -58,26 +58,25 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], val queryResult = influxDB.query(query) if (!queryResult.hasError) { - dataRetrieved = Option(queryResult) + dataRetrieved = Some(queryResult) } } override def onPull(): Unit = - if (dataRetrieved.isEmpty) - completeStage() - else { - val queryResult = dataRetrieved.get - for { - result <- queryResult.getResults.asScala - series <- result.getSeries.asScala - }( - emitMultiple(outlet, - resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) - ) - - dataRetrieved = Option.empty - } + dataRetrieved match { + case None => completeStage() + case Some(queryResult) => { + for { + result <- queryResult.getResults.asScala + series <- result.getSeries.asScala + }( + emitMultiple(outlet, + resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) + ) + dataRetrieved = None + } + } } /** From a6b90705eef97dd68c07356009c61ba46afbb94c Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 23:19:37 +0100 Subject: [PATCH 39/89] Changed to one liners --- .../influxdb/impl/InfluxDBSourceStage.scala | 2 -- .../influxdb/scaladsl/InfluxDBFlow.scala | 28 +++---------------- .../influxdb/scaladsl/InfluxDBSource.scala | 7 +---- 3 files changed, 5 insertions(+), 32 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 8fed50e2e5..20b4e58ad4 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -4,8 +4,6 @@ package akka.stream.alpakka.influxdb.impl -import java.util.concurrent.TimeUnit - import akka.annotation.InternalApi import akka.stream.alpakka.influxdb.InfluxDBSettings import akka.stream.{Attributes, Outlet, SourceShape} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index 7ee3143d24..13e812dbcd 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -21,12 +21,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via( - new impl.InfluxDBFlowStage[T, NotUsed]( - Option.empty, - influxDB - ) - ) + .via(new impl.InfluxDBFlowStage[T, NotUsed](Option.empty, influxDB)) .mapConcat(identity) def typed[T](clazz: Class[T], settings: InfluxDBSettings)( @@ -34,12 +29,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via( - new impl.InfluxDBFlowStage[T, NotUsed]( - Option(clazz), - influxDB - ) - ) + .via(new impl.InfluxDBFlowStage[T, NotUsed](Option(clazz), influxDB)) .mapConcat(identity) def createWithPassThrough[T, C](settings: InfluxDBSettings)( @@ -47,12 +37,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via( - new impl.InfluxDBFlowStage[T, C]( - Option.empty, - influxDB - ) - ) + .via(new impl.InfluxDBFlowStage[T, C](Option.empty, influxDB)) .mapConcat(identity) def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings)( @@ -60,12 +45,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via( - new impl.InfluxDBFlowStage[T, C]( - Option(clazz), - influxDB - ) - ) + .via(new impl.InfluxDBFlowStage[T, C](Option(clazz), influxDB)) .mapConcat(identity) def createWithContext[T, C](settings: InfluxDBSettings)( diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala index 18e61c28d7..4b6852aed6 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -20,12 +20,7 @@ object InfluxDBSource { * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDBRawSourceStage]] from a given statement. */ def apply(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = - Source.fromGraph( - new InfluxDBRawSourceStage( - query, - influxDB - ) - ) + Source.fromGraph(new InfluxDBRawSourceStage(query, influxDB)) /** * Read elements of `T` from `className` or by `query`. From c50fd1926aa449be4cce56b1e1cf025c84bc4fc7 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 23:20:58 +0100 Subject: [PATCH 40/89] Changed to None for InfluxDB connector --- .../akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index 13e812dbcd..60afb49f22 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -21,7 +21,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, NotUsed](Option.empty, influxDB)) + .via(new impl.InfluxDBFlowStage[T, NotUsed](None, influxDB)) .mapConcat(identity) def typed[T](clazz: Class[T], settings: InfluxDBSettings)( @@ -37,7 +37,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, C](Option.empty, influxDB)) + .via(new impl.InfluxDBFlowStage[T, C](None, influxDB)) .mapConcat(identity) def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings)( From 12cf0670da759e49ab2d98cd6fa5b6df2ce477bf Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Wed, 8 May 2019 23:25:07 +0100 Subject: [PATCH 41/89] Changed from option to some for InfluxDB connector --- .../akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala index 60afb49f22..1a8004edd1 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala @@ -29,7 +29,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDBWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, NotUsed](Option(clazz), influxDB)) + .via(new impl.InfluxDBFlowStage[T, NotUsed](Some(clazz), influxDB)) .mapConcat(identity) def createWithPassThrough[T, C](settings: InfluxDBSettings)( @@ -45,7 +45,7 @@ object InfluxDBFlow { ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = Flow[InfluxDBWriteMessage[T, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, C](Option(clazz), influxDB)) + .via(new impl.InfluxDBFlowStage[T, C](Some(clazz), influxDB)) .mapConcat(identity) def createWithContext[T, C](settings: InfluxDBSettings)( From d5f984bbf411443041f504acbbd79861232c8f44 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 10 May 2019 21:52:28 +0100 Subject: [PATCH 42/89] Changed on alpakka.influxdb on reference for InfluxDB connector --- influxdb/src/main/resources/reference.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/resources/reference.conf b/influxdb/src/main/resources/reference.conf index 659b42b8a1..a8f9ff429b 100644 --- a/influxdb/src/main/resources/reference.conf +++ b/influxdb/src/main/resources/reference.conf @@ -1,4 +1,4 @@ -akka.stream.alpakka.influxdb{ +alpakka.influxdb{ #batch write size batchSize = 10 From a12867c9bededf320cf211fd3d55c1e4bf0042aa Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sat, 11 May 2019 17:01:17 +0100 Subject: [PATCH 43/89] Added information on database and retention policy on InfluxDBWriteMessage for InfluxDB connector --- .../influxdb/InfluxDBWriteMessage.scala | 28 +++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala index dbec946d90..c5cbe5860a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala @@ -9,7 +9,7 @@ import akka.NotUsed object InfluxDBWriteMessage { // Apply method to use when not using passThrough def apply[T](point: T): InfluxDBWriteMessage[T, NotUsed] = - InfluxDBWriteMessage(point, NotUsed) + InfluxDBWriteMessage(point = point, passThrough = NotUsed) // Java-api - without passThrough def create[T](point: T): InfluxDBWriteMessage[T, NotUsed] = @@ -20,6 +20,30 @@ object InfluxDBWriteMessage { InfluxDBWriteMessage(point, passThrough) } -final case class InfluxDBWriteMessage[T, C](point: T, passThrough: C) +final case class InfluxDBWriteMessage[T, C](point: T, + passThrough: C, + databaseName: Option[String] = None, + retentionPolicy: Option[String] = None) { + + def withPoint(point: T): InfluxDBWriteMessage[T, C] = + copy(point = point) + + def withDatabaseName(databaseName: String): InfluxDBWriteMessage[T, C] = + copy(databaseName = Some(databaseName)) + + def withRetentionPolicy(retentionPolicy: String): InfluxDBWriteMessage[T, C] = + copy(retentionPolicy = Some(retentionPolicy)) + + private def copy( + point: T = point, + passThrough: C = passThrough, + databaseName: Option[String] = databaseName, + retentionPolicy: Option[String] = retentionPolicy + ): InfluxDBWriteMessage[T, C] = + new InfluxDBWriteMessage(point = point, + passThrough = passThrough, + databaseName = databaseName, + retentionPolicy = retentionPolicy) +} final case class InfluxDBWriteResult[T, C](writeMessage: InfluxDBWriteMessage[T, C], error: Option[String]) From a46492761e4f3067dd01e193ffcb6ef48d4f7e54 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sat, 11 May 2019 17:03:25 +0100 Subject: [PATCH 44/89] Added result mapper utility for InfluxDB connector --- .../impl/InfluxDBResultMapperHelper.java | 99 ++++++++++++++++++- 1 file changed, 95 insertions(+), 4 deletions(-) diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java index 36bd1eb60d..be8472b14b 100644 --- a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java +++ b/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java @@ -5,13 +5,15 @@ package org.influxdb.impl; import java.lang.reflect.Field; -import java.util.ArrayList; +import java.time.Instant; import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import org.influxdb.InfluxDBMapperException; +import org.influxdb.annotation.Column; +import org.influxdb.dto.Point; import org.influxdb.dto.QueryResult; public class InfluxDBResultMapperHelper { @@ -22,12 +24,101 @@ public void cacheClassFields(final Class clazz) { influxDBResultMapper.cacheMeasurementClass(clazz); } + public String databaseName(T model) { + Class clazz = model.getClass(); + return influxDBResultMapper.getDatabaseName(clazz); + } + + public String retentionPolicy(T model) { + Class clazz = model.getClass(); + return influxDBResultMapper.getRetentionPolicy(clazz); + } + + public Point convertModelToPoint(T model) { + influxDBResultMapper.throwExceptionIfMissingAnnotation(model.getClass()); + influxDBResultMapper.cacheMeasurementClass(model.getClass()); + + ConcurrentMap colNameAndFieldMap = + influxDBResultMapper.getColNameAndFieldMap(model.getClass()); + + try { + Class modelType = model.getClass(); + String measurement = influxDBResultMapper.getMeasurementName(modelType); + TimeUnit timeUnit = influxDBResultMapper.getTimeUnit(modelType); + long time = timeUnit.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS); + Point.Builder pointBuilder = Point.measurement(measurement).time(time, timeUnit); + + for (String key : colNameAndFieldMap.keySet()) { + Field field = colNameAndFieldMap.get(key); + Column column = field.getAnnotation(Column.class); + String columnName = column.name(); + Class fieldType = field.getType(); + + if (!field.isAccessible()) { + field.setAccessible(true); + } + + Object value = field.get(model); + + if (column.tag()) { + /** Tags are strings either way. */ + pointBuilder.tag(columnName, value.toString()); + } else if ("time".equals(columnName)) { + if (value != null) { + setTime(pointBuilder, fieldType, timeUnit, value); + } + } else { + setField(pointBuilder, fieldType, columnName, value); + } + } + + return pointBuilder.build(); + } catch (IllegalAccessException e) { + throw new InfluxDBMapperException(e); + } + } + + private void setTime( + final Point.Builder pointBuilder, + final Class fieldType, + final TimeUnit timeUnit, + final Object value) { + if (Instant.class.isAssignableFrom(fieldType)) { + Instant instant = (Instant) value; + long time = timeUnit.convert(instant.toEpochMilli(), TimeUnit.MILLISECONDS); + pointBuilder.time(time, timeUnit); + } else { + throw new InfluxDBMapperException( + "Unsupported type " + fieldType + " for time: should be of Instant type"); + } + } + + private void setField( + final Point.Builder pointBuilder, + final Class fieldType, + final String columnName, + final Object value) { + if (boolean.class.isAssignableFrom(fieldType) || Boolean.class.isAssignableFrom(fieldType)) { + pointBuilder.addField(columnName, (boolean) value); + } else if (long.class.isAssignableFrom(fieldType) || Long.class.isAssignableFrom(fieldType)) { + pointBuilder.addField(columnName, (long) value); + } else if (double.class.isAssignableFrom(fieldType) + || Double.class.isAssignableFrom(fieldType)) { + pointBuilder.addField(columnName, (double) value); + } else if (int.class.isAssignableFrom(fieldType) || Integer.class.isAssignableFrom(fieldType)) { + pointBuilder.addField(columnName, (int) value); + } else if (String.class.isAssignableFrom(fieldType)) { + pointBuilder.addField(columnName, (String) value); + } else { + throw new InfluxDBMapperException( + "Unsupported type " + fieldType + " for column " + columnName); + } + } + public List parseSeriesAs( final Class clazz, final QueryResult.Series series, final TimeUnit precision) { influxDBResultMapper.cacheMeasurementClass(clazz); - return series - .getValues() - .stream() + return series.getValues().stream() .map(v -> parseRowAs(clazz, series.getColumns(), v, precision)) .collect(Collectors.toList()); } From 162b89766dfb5854de08b9136a3ca42071c1b6b4 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sat, 11 May 2019 17:05:49 +0100 Subject: [PATCH 45/89] Changed flow to avoid batching for InfluxDB connector --- .../influxdb/impl/InfluxDBFlowStage.scala | 74 +++++++++++++++---- .../influxdb/impl/InfluxDBSourceStage.scala | 5 +- .../test/java/docs/javadsl/InfluxDBTest.java | 4 +- .../docs/scaladsl/InfluxDBSourceSpec.scala | 1 - 4 files changed, 62 insertions(+), 22 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 9465061985..be0278a834 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -8,11 +8,11 @@ import akka.stream.{Attributes, FlowShape, Inlet, Outlet} import akka.stream.alpakka.influxdb.{InfluxDBWriteMessage, InfluxDBWriteResult} import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} import org.influxdb.InfluxDB -import org.influxdb.impl.InfluxDBMapper +import org.influxdb.impl.InfluxDBResultMapperHelper import scala.collection.immutable import org.influxdb.BatchOptions -import org.influxdb.dto.Point +import org.influxdb.dto.{BatchPoints, Point} /** * INTERNAL API @@ -57,29 +57,73 @@ private[influxdb] class InfluxDBFlowStage[T, C]( final class InfluxDBRecordLogic extends InfluxDBLogic { + private var mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper + override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = - messages.foreach { - case InfluxDBWriteMessage(point: Point, _) => { - influxDB.write(point) + messages + .filter { + case InfluxDBWriteMessage(point: Point, _, _, _) => { + true + } + case InfluxDBWriteMessage(others: AnyRef, _, _, _) => { + failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) + false + } } - case InfluxDBWriteMessage(others: AnyRef, _) => - failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) - } + .groupBy(im => (im.databaseName, im.retentionPolicy)) + .map(wm => toBatchPoints(wm._1._1, wm._1._2, wm._2)) + .foreach(influxDB.write) } final class InfluxDBMapperRecordLogic extends InfluxDBLogic { - protected var influxDBMapper: InfluxDBMapper = _ - - override def preStart(): Unit = - influxDBMapper = new InfluxDBMapper(influxDB) + private var mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = - messages.foreach { - case InfluxDBWriteMessage(typeMetric: Any, _) => - influxDBMapper.save(typeMetric) + messages + .groupBy(im => { + ( + im.databaseName match { + case Some(databaseName) => Some(databaseName) + case None => Some(mapperHelper.databaseName(im.point)) + }, + im.retentionPolicy match { + case Some(databaseName) => Some(databaseName) + case None => Some(mapperHelper.retentionPolicy(im.point)) + } + ) + }) + .map( + wm => + toBatchPoints(wm._1._1, + wm._1._2, + wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) + ) + .foreach(influxDB.write) + } + + private def toBatchPoints(databaseName: Option[String], + retentionPolicy: Option[String], + seq: Seq[InfluxDBWriteMessage[T, C]]) = { + + val builder = databaseName match { + case Some(databaseName) => BatchPoints.database(databaseName) + case None => BatchPoints.builder() + } + + if (retentionPolicy.isDefined) builder.retentionPolicy(retentionPolicy.get) + + def convert(messages: Seq[InfluxDBWriteMessage[T, C]]): BatchPoints = + messages match { + case head :: tail => { + builder.point(head.point.asInstanceOf[Point]) + convert(tail) + } + case Nil => builder.build() } + + convert(seq) } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala index 20b4e58ad4..ccf27a0016 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala @@ -67,10 +67,7 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], for { result <- queryResult.getResults.asScala series <- result.getSeries.asScala - }( - emitMultiple(outlet, - resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) - ) + } emitMultiple(outlet, resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) dataRetrieved = None } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java index 023b4239bf..8ea77630d7 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java @@ -88,7 +88,7 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { .map( cpu -> { InfluxDBCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); - return new InfluxDBWriteMessage<>(clonedCpu, NotUsed.notUsed()); + return InfluxDBWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) .runWith( InfluxDBSink.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB), @@ -133,7 +133,7 @@ private List> points(QueryResult queryResul for (QueryResult.Series series : result.getSeries()) { for (List rows : series.getValues()) { InfluxDBWriteMessage influxDBWriteMessage = - new InfluxDBWriteMessage<>(resultToPoint(series, rows), NotUsed.notUsed()); + InfluxDBWriteMessage.create(resultToPoint(series, rows), NotUsed.notUsed()); points.add(influxDBWriteMessage); } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala index 311ba48772..58d3399c85 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala @@ -14,7 +14,6 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordS import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import docs.javadsl.TestUtils._ -import docs.javadsl.TestConstants._ import org.influxdb.dto.Query class InfluxDBSourceSpec From dc6197256aed3e9f368ca364b32fe20885425ef2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sat, 11 May 2019 19:35:48 +0100 Subject: [PATCH 46/89] Set IODispatcher on InfluxDBFlowStage for InfluxDB connector --- .../stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index be0278a834..acd4a65c05 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -4,7 +4,7 @@ package akka.stream.alpakka.influxdb.impl -import akka.stream.{Attributes, FlowShape, Inlet, Outlet} +import akka.stream._ import akka.stream.alpakka.influxdb.{InfluxDBWriteMessage, InfluxDBWriteResult} import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} import org.influxdb.InfluxDB @@ -26,6 +26,9 @@ private[influxdb] class InfluxDBFlowStage[T, C]( override val shape = FlowShape(in, out) + override protected def initialAttributes: Attributes = + super.initialAttributes and Attributes(ActorAttributes.IODispatcher) + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = clazz match { case Some(c) => new InfluxDBMapperRecordLogic From 395f373a3010e5abe2d23d62324f49dde820a02c Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sat, 11 May 2019 20:33:13 +0100 Subject: [PATCH 47/89] Chnaged to val for InfluxDB connector --- .../akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index acd4a65c05..8ed3c20dfb 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -60,8 +60,6 @@ private[influxdb] class InfluxDBFlowStage[T, C]( final class InfluxDBRecordLogic extends InfluxDBLogic { - private var mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper - override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages .filter { @@ -81,7 +79,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( final class InfluxDBMapperRecordLogic extends InfluxDBLogic { - private var mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper + private val mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages From 4000c84d822a32e529239e88388b90a110c2d4f2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 14 May 2019 10:46:06 +0100 Subject: [PATCH 48/89] Removed unecessary variables InfluxDB connector --- .../influxdb/impl/InfluxDBFlowStage.scala | 43 +++++------ .../test/scala/docs/scaladsl/FlowSpec.scala | 74 +++++++++++++++++++ 2 files changed, 96 insertions(+), 21 deletions(-) create mode 100644 influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 8ed3c20dfb..10f8bf839b 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -63,11 +63,11 @@ private[influxdb] class InfluxDBFlowStage[T, C]( override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages .filter { - case InfluxDBWriteMessage(point: Point, _, _, _) => { + case InfluxDBWriteMessage( _: Point, _, _, _) => { true } - case InfluxDBWriteMessage(others: AnyRef, _, _, _) => { - failStage(new RuntimeException(s"unexpected type Point or annotated with Measurement required")) + case InfluxDBWriteMessage( _: AnyRef, _, _, _) => { + failStage(new RuntimeException(s"unexpected type Point required")) false } } @@ -83,25 +83,26 @@ private[influxdb] class InfluxDBFlowStage[T, C]( override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages - .groupBy(im => { - ( - im.databaseName match { - case Some(databaseName) => Some(databaseName) - case None => Some(mapperHelper.databaseName(im.point)) - }, - im.retentionPolicy match { - case Some(databaseName) => Some(databaseName) - case None => Some(mapperHelper.retentionPolicy(im.point)) - } - ) - }) - .map( - wm => - toBatchPoints(wm._1._1, - wm._1._2, - wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) - ) + .groupBy(groupByDbRp) + .map(convertToBatchPoints) .foreach(influxDB.write) + + def groupByDbRp(im: InfluxDBWriteMessage[T, C]) = + ( + im.databaseName match { + case Some(databaseName) => Some(databaseName) + case None => Some(mapperHelper.databaseName(im.point)) + }, + im.retentionPolicy match { + case Some(databaseName) => Some(databaseName) + case None => Some(mapperHelper.retentionPolicy(im.point)) + } + ) + + def convertToBatchPoints(wm: ((Some[String], Some[String]), immutable.Seq[InfluxDBWriteMessage[T, C]])) = + toBatchPoints(wm._1._1, + wm._1._2, + wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) } private def toBatchPoints(databaseName: Option[String], diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala new file mode 100644 index 0000000000..ef92fdf279 --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -0,0 +1,74 @@ +package docs.scaladsl + +import java.util.concurrent.TimeUnit + +import akka.actor.ActorSystem +import akka.stream.ActorMaterializer +import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} +import akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow +import akka.stream.scaladsl.{Sink, Source} +import akka.testkit.TestKit +import org.influxdb.InfluxDB +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} +import org.scalatest.concurrent.ScalaFutures +import docs.javadsl.TestUtils._ +import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped +import org.influxdb.dto.Point + +private final case class InvalidModel(description: String) { +} + +class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { + + implicit val system = ActorSystem() + implicit val mat = ActorMaterializer() + + final val DatabaseName = this.getClass.getSimpleName + + implicit var influxDB: InfluxDB = _ + + override protected def beforeAll(): Unit = + influxDB = setupConnection(DatabaseName) + + override protected def afterAll(): Unit = { + dropDatabase(influxDB, DatabaseName) + TestKit.shutdownActorSystem(system) + } + + + "invalid model" in assertAllStagesStopped { + val result = Source(List( + InfluxDBWriteMessage(InvalidModel("Invalid measurement one")), + InfluxDBWriteMessage(InvalidModel("Invalid measurement two"))) + ) + .via(InfluxDBFlow.create[InvalidModel](InfluxDBSettings())) + .recover { + case _: RuntimeException => InfluxDBWriteResult( null, Some("error occurred")) + } + .runWith(Sink.seq) + .futureValue + + result mustBe Seq(InfluxDBWriteResult( null, Some("error occurred"))) + } + + "mixed model" in assertAllStagesStopped { + + val point = Point.measurement("disk") + .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) + .addField("used", 80L) + .addField("free", 1L) + .build() + + + val validMessage = InfluxDBWriteMessage(point) + .withDatabaseName(DatabaseName) + + val result = Source(List( + validMessage + )).via(InfluxDBFlow.create[Point](InfluxDBSettings())) + .runWith(Sink.seq) + .futureValue + + result(0).error mustBe None + } +} From eaf6060c1c3e09be148209112cdb0d3ad285b994 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 14 May 2019 11:10:03 +0100 Subject: [PATCH 49/89] Applied formatting on InfluxDB connector --- .../influxdb/impl/InfluxDBFlowStage.scala | 4 +-- .../test/scala/docs/scaladsl/FlowSpec.scala | 33 ++++++++++--------- 2 files changed, 20 insertions(+), 17 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 10f8bf839b..6697beaa69 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -63,10 +63,10 @@ private[influxdb] class InfluxDBFlowStage[T, C]( override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = messages .filter { - case InfluxDBWriteMessage( _: Point, _, _, _) => { + case InfluxDBWriteMessage(_: Point, _, _, _) => { true } - case InfluxDBWriteMessage( _: AnyRef, _, _, _) => { + case InfluxDBWriteMessage(_: AnyRef, _, _, _) => { failStage(new RuntimeException(s"unexpected type Point required")) false } diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index ef92fdf279..8f6f1909aa 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -1,3 +1,7 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + package docs.scaladsl import java.util.concurrent.TimeUnit @@ -15,8 +19,7 @@ import docs.javadsl.TestUtils._ import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import org.influxdb.dto.Point -private final case class InvalidModel(description: String) { -} +private final case class InvalidModel(description: String) {} class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { @@ -35,37 +38,37 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B TestKit.shutdownActorSystem(system) } - "invalid model" in assertAllStagesStopped { - val result = Source(List( - InfluxDBWriteMessage(InvalidModel("Invalid measurement one")), - InfluxDBWriteMessage(InvalidModel("Invalid measurement two"))) - ) - .via(InfluxDBFlow.create[InvalidModel](InfluxDBSettings())) + val result = Source( + List(InfluxDBWriteMessage(InvalidModel("Invalid measurement one")), + InfluxDBWriteMessage(InvalidModel("Invalid measurement two"))) + ).via(InfluxDBFlow.create[InvalidModel](InfluxDBSettings())) .recover { - case _: RuntimeException => InfluxDBWriteResult( null, Some("error occurred")) + case _: RuntimeException => InfluxDBWriteResult(null, Some("error occurred")) } .runWith(Sink.seq) .futureValue - result mustBe Seq(InfluxDBWriteResult( null, Some("error occurred"))) + result mustBe Seq(InfluxDBWriteResult(null, Some("error occurred"))) } "mixed model" in assertAllStagesStopped { - val point = Point.measurement("disk") + val point = Point + .measurement("disk") .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) .addField("used", 80L) .addField("free", 1L) .build() - val validMessage = InfluxDBWriteMessage(point) .withDatabaseName(DatabaseName) - val result = Source(List( - validMessage - )).via(InfluxDBFlow.create[Point](InfluxDBSettings())) + val result = Source( + List( + validMessage + ) + ).via(InfluxDBFlow.create[Point](InfluxDBSettings())) .runWith(Sink.seq) .futureValue From 17019016df6be6149437e6f52a48f7bef1a86ee9 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 17 May 2019 16:56:37 +0300 Subject: [PATCH 50/89] Update influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala Co-Authored-By: Enno <458526+ennru@users.noreply.github.com> --- .../akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 6697beaa69..003ed4dc1b 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -90,7 +90,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( def groupByDbRp(im: InfluxDBWriteMessage[T, C]) = ( im.databaseName match { - case Some(databaseName) => Some(databaseName) + case dbn: Some(databaseName) => dbn case None => Some(mapperHelper.databaseName(im.point)) }, im.retentionPolicy match { From a150cc4129be96e97027a5b67ea4a1b00533ad13 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 17 May 2019 20:28:23 +0100 Subject: [PATCH 51/89] Removed reference on InfluxDB connector avoid new object creation on case on InlfuxDb connector --- influxdb/src/main/resources/reference.conf | 18 ------------------ .../influxdb/impl/InfluxDBFlowStage.scala | 4 ++-- 2 files changed, 2 insertions(+), 20 deletions(-) delete mode 100644 influxdb/src/main/resources/reference.conf diff --git a/influxdb/src/main/resources/reference.conf b/influxdb/src/main/resources/reference.conf deleted file mode 100644 index a8f9ff429b..0000000000 --- a/influxdb/src/main/resources/reference.conf +++ /dev/null @@ -1,18 +0,0 @@ -alpakka.influxdb{ - - #batch write size - batchSize = 10 - - # The InfluxDB write precision - precision = MILLISECONDS - - url = "" - - # credentials { - # username = "root" - # password = "root" - # } - - batchedWrites = true - -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala index 003ed4dc1b..aae146b3e2 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala @@ -90,11 +90,11 @@ private[influxdb] class InfluxDBFlowStage[T, C]( def groupByDbRp(im: InfluxDBWriteMessage[T, C]) = ( im.databaseName match { - case dbn: Some(databaseName) => dbn + case dbn: Some[String] => dbn case None => Some(mapperHelper.databaseName(im.point)) }, im.retentionPolicy match { - case Some(databaseName) => Some(databaseName) + case dbn: Some[String] => dbn case None => Some(mapperHelper.retentionPolicy(im.point)) } ) From b50c5c4f38992dc67ec64927ccb91dd0b5bddec4 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 17 May 2019 20:50:07 +0100 Subject: [PATCH 52/89] Removed case class on settings for InfluxDB connector --- .../scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala index f468e00879..5d9db02cc2 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala @@ -15,7 +15,7 @@ object InfluxDBSettings { } -final case class InfluxDBSettings private ( +final class InfluxDBSettings private ( val batchSize: Int, val precision: TimeUnit ) { From 953738eaaa4625897281bdf69bd990f11d93f2d9 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 00:14:45 +0100 Subject: [PATCH 53/89] Changed to InfluxDb for InfluxDb connector --- ...r.java => InfluxDbResultMapperHelper.java} | 2 +- ...BSettings.scala => InfluxDbSettings.scala} | 12 +-- ...ssage.scala => InfluxDbWriteMessage.scala} | 26 +++---- ...lowStage.scala => InfluxDbFlowStage.scala} | 44 +++++------ ...eStage.scala => InfluxDbSourceStage.scala} | 28 +++---- .../influxdb/javadsl/InfluxDBFlow.scala | 54 ------------- .../influxdb/javadsl/InfluxDbFlow.scala | 54 +++++++++++++ ...{InfluxDBSink.scala => InfluxDbSink.scala} | 20 ++--- ...luxDBSource.scala => InfluxDbSource.scala} | 16 ++-- .../influxdb/scaladsl/InfluxDBFlow.scala | 77 ------------------- .../influxdb/scaladsl/InfluxDBSource.scala | 14 ++-- .../influxdb/scaladsl/InfluxDbFlow.scala | 77 +++++++++++++++++++ ...{InfluxDBSink.scala => InfluxDbSink.scala} | 16 ++-- .../{InfluxDBCpu.java => InfluxDbCpu.java} | 12 +-- ...BSourceCpu.java => InfluxDbSourceCpu.java} | 8 +- ...ourceTest.java => InfluxDbSourceTest.java} | 18 ++--- .../{InfluxDBTest.java => InfluxDbTest.java} | 38 ++++----- .../test/scala/docs/scaladsl/FlowSpec.scala | 18 ++--- ...BSourceCpu.java => InfluxDbSourceCpu.java} | 12 +-- ...rceSpec.scala => InfluxDbSourceSpec.scala} | 10 +-- ...{InfluxDBSpec.scala => InfluxDbSpec.scala} | 34 ++++---- ...luxDBSpecCpu.java => InfluxDbSpecCpu.java} | 12 +-- 22 files changed, 301 insertions(+), 301 deletions(-) rename influxdb/src/main/java/org/influxdb/impl/{InfluxDBResultMapperHelper.java => InfluxDbResultMapperHelper.java} (99%) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/{InfluxDBSettings.scala => InfluxDbSettings.scala} (60%) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/{InfluxDBWriteMessage.scala => InfluxDbWriteMessage.scala} (59%) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/{InfluxDBFlowStage.scala => InfluxDbFlowStage.scala} (68%) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/{InfluxDBSourceStage.scala => InfluxDbSourceStage.scala} (80%) delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/{InfluxDBSink.scala => InfluxDbSink.scala} (57%) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/{InfluxDBSource.scala => InfluxDbSource.scala} (56%) delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/{InfluxDBSink.scala => InfluxDbSink.scala} (57%) rename influxdb/src/test/java/docs/javadsl/{InfluxDBCpu.java => InfluxDbCpu.java} (68%) rename influxdb/src/test/java/docs/javadsl/{InfluxDBSourceCpu.java => InfluxDbSourceCpu.java} (69%) rename influxdb/src/test/java/docs/javadsl/{InfluxDBSourceTest.java => InfluxDbSourceTest.java} (83%) rename influxdb/src/test/java/docs/javadsl/{InfluxDBTest.java => InfluxDbTest.java} (77%) rename influxdb/src/test/scala/docs/scaladsl/{InfluxDBSourceCpu.java => InfluxDbSourceCpu.java} (56%) rename influxdb/src/test/scala/docs/scaladsl/{InfluxDBSourceSpec.scala => InfluxDbSourceSpec.scala} (84%) rename influxdb/src/test/scala/docs/scaladsl/{InfluxDBSpec.scala => InfluxDbSpec.scala} (67%) rename influxdb/src/test/scala/docs/scaladsl/{InfluxDBSpecCpu.java => InfluxDbSpecCpu.java} (56%) diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java similarity index 99% rename from influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java rename to influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java index be8472b14b..00a1310487 100644 --- a/influxdb/src/main/java/org/influxdb/impl/InfluxDBResultMapperHelper.java +++ b/influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java @@ -16,7 +16,7 @@ import org.influxdb.dto.Point; import org.influxdb.dto.QueryResult; -public class InfluxDBResultMapperHelper { +public class InfluxDbResultMapperHelper { private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala similarity index 60% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala index 5d9db02cc2..03de317f34 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala @@ -8,24 +8,24 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.TimeUnit -object InfluxDBSettings { - val Default = new InfluxDBSettings(batchSize = 10, TimeUnit.MILLISECONDS) +object InfluxDbSettings { + val Default = new InfluxDbSettings(batchSize = 10, TimeUnit.MILLISECONDS) - def apply(): InfluxDBSettings = Default + def apply(): InfluxDbSettings = Default } -final class InfluxDBSettings private ( +final class InfluxDbSettings private ( val batchSize: Int, val precision: TimeUnit ) { - def withBatchSize(value: Int): InfluxDBSettings = copy(batchSize = value) + def withBatchSize(value: Int): InfluxDbSettings = copy(batchSize = value) private def copy( batchSize: Int = batchSize, precision: TimeUnit = precision - ): InfluxDBSettings = new InfluxDBSettings( + ): InfluxDbSettings = new InfluxDbSettings( batchSize = batchSize, precision = precision ) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala similarity index 59% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala index c5cbe5860a..2b557aa70b 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDBWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala @@ -6,32 +6,32 @@ package akka.stream.alpakka.influxdb import akka.NotUsed -object InfluxDBWriteMessage { +object InfluxDbWriteMessage { // Apply method to use when not using passThrough - def apply[T](point: T): InfluxDBWriteMessage[T, NotUsed] = - InfluxDBWriteMessage(point = point, passThrough = NotUsed) + def apply[T](point: T): InfluxDbWriteMessage[T, NotUsed] = + InfluxDbWriteMessage(point = point, passThrough = NotUsed) // Java-api - without passThrough - def create[T](point: T): InfluxDBWriteMessage[T, NotUsed] = - InfluxDBWriteMessage(point, NotUsed) + def create[T](point: T): InfluxDbWriteMessage[T, NotUsed] = + InfluxDbWriteMessage(point, NotUsed) // Java-api - with passThrough def create[T, C](point: T, passThrough: C) = - InfluxDBWriteMessage(point, passThrough) + InfluxDbWriteMessage(point, passThrough) } -final case class InfluxDBWriteMessage[T, C](point: T, +final case class InfluxDbWriteMessage[T, C](point: T, passThrough: C, databaseName: Option[String] = None, retentionPolicy: Option[String] = None) { - def withPoint(point: T): InfluxDBWriteMessage[T, C] = + def withPoint(point: T): InfluxDbWriteMessage[T, C] = copy(point = point) - def withDatabaseName(databaseName: String): InfluxDBWriteMessage[T, C] = + def withDatabaseName(databaseName: String): InfluxDbWriteMessage[T, C] = copy(databaseName = Some(databaseName)) - def withRetentionPolicy(retentionPolicy: String): InfluxDBWriteMessage[T, C] = + def withRetentionPolicy(retentionPolicy: String): InfluxDbWriteMessage[T, C] = copy(retentionPolicy = Some(retentionPolicy)) private def copy( @@ -39,11 +39,11 @@ final case class InfluxDBWriteMessage[T, C](point: T, passThrough: C = passThrough, databaseName: Option[String] = databaseName, retentionPolicy: Option[String] = retentionPolicy - ): InfluxDBWriteMessage[T, C] = - new InfluxDBWriteMessage(point = point, + ): InfluxDbWriteMessage[T, C] = + new InfluxDbWriteMessage(point = point, passThrough = passThrough, databaseName = databaseName, retentionPolicy = retentionPolicy) } -final case class InfluxDBWriteResult[T, C](writeMessage: InfluxDBWriteMessage[T, C], error: Option[String]) +final case class InfluxDbWriteResult[T, C](writeMessage: InfluxDbWriteMessage[T, C], error: Option[String]) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala similarity index 68% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index aae146b3e2..b84a59de5a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -5,24 +5,24 @@ package akka.stream.alpakka.influxdb.impl import akka.stream._ -import akka.stream.alpakka.influxdb.{InfluxDBWriteMessage, InfluxDBWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} import org.influxdb.InfluxDB -import org.influxdb.impl.InfluxDBResultMapperHelper import scala.collection.immutable import org.influxdb.BatchOptions import org.influxdb.dto.{BatchPoints, Point} +import org.influxdb.impl.InfluxDbResultMapperHelper /** * INTERNAL API */ -private[influxdb] class InfluxDBFlowStage[T, C]( +private[influxdb] class InfluxDbFlowStage[T, C]( clazz: Option[Class[T]], influxDB: InfluxDB -) extends GraphStage[FlowShape[immutable.Seq[InfluxDBWriteMessage[T, C]], immutable.Seq[InfluxDBWriteResult[T, C]]]] { - private val in = Inlet[immutable.Seq[InfluxDBWriteMessage[T, C]]]("in") - private val out = Outlet[immutable.Seq[InfluxDBWriteResult[T, C]]]("out") +) extends GraphStage[FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]]] { + private val in = Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]]("in") + private val out = Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]]("out") override val shape = FlowShape(in, out) @@ -31,13 +31,13 @@ private[influxdb] class InfluxDBFlowStage[T, C]( override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = clazz match { - case Some(c) => new InfluxDBMapperRecordLogic - case None => new InfluxDBRecordLogic + case Some(c) => new InfluxDbMapperRecordLogic + case None => new InfluxDbRecordLogic } - sealed abstract class InfluxDBLogic extends GraphStageLogic(shape) with InHandler with OutHandler { + sealed abstract class InfluxDbLogic extends GraphStageLogic(shape) with InHandler with OutHandler { - protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit + protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit setHandlers(in, out, this) @@ -49,7 +49,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( influxDB.enableBatch(BatchOptions.DEFAULTS) write(messages) - val writtenMessages = messages.map(m => new InfluxDBWriteResult(m, None)) + val writtenMessages = messages.map(m => new InfluxDbWriteResult(m, None)) influxDB.close() push(out, writtenMessages) } @@ -58,15 +58,15 @@ private[influxdb] class InfluxDBFlowStage[T, C]( } } - final class InfluxDBRecordLogic extends InfluxDBLogic { + final class InfluxDbRecordLogic extends InfluxDbLogic { - override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = + override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = messages .filter { - case InfluxDBWriteMessage(_: Point, _, _, _) => { + case InfluxDbWriteMessage(_: Point, _, _, _) => { true } - case InfluxDBWriteMessage(_: AnyRef, _, _, _) => { + case InfluxDbWriteMessage(_: AnyRef, _, _, _) => { failStage(new RuntimeException(s"unexpected type Point required")) false } @@ -77,17 +77,17 @@ private[influxdb] class InfluxDBFlowStage[T, C]( } - final class InfluxDBMapperRecordLogic extends InfluxDBLogic { + final class InfluxDbMapperRecordLogic extends InfluxDbLogic { - private val mapperHelper: InfluxDBResultMapperHelper = new InfluxDBResultMapperHelper + private val mapperHelper: InfluxDbResultMapperHelper = new InfluxDbResultMapperHelper - override protected def write(messages: immutable.Seq[InfluxDBWriteMessage[T, C]]): Unit = + override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = messages .groupBy(groupByDbRp) .map(convertToBatchPoints) .foreach(influxDB.write) - def groupByDbRp(im: InfluxDBWriteMessage[T, C]) = + def groupByDbRp(im: InfluxDbWriteMessage[T, C]) = ( im.databaseName match { case dbn: Some[String] => dbn @@ -99,7 +99,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( } ) - def convertToBatchPoints(wm: ((Some[String], Some[String]), immutable.Seq[InfluxDBWriteMessage[T, C]])) = + def convertToBatchPoints(wm: ((Some[String], Some[String]), immutable.Seq[InfluxDbWriteMessage[T, C]])) = toBatchPoints(wm._1._1, wm._1._2, wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) @@ -107,7 +107,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( private def toBatchPoints(databaseName: Option[String], retentionPolicy: Option[String], - seq: Seq[InfluxDBWriteMessage[T, C]]) = { + seq: Seq[InfluxDbWriteMessage[T, C]]) = { val builder = databaseName match { case Some(databaseName) => BatchPoints.database(databaseName) @@ -116,7 +116,7 @@ private[influxdb] class InfluxDBFlowStage[T, C]( if (retentionPolicy.isDefined) builder.retentionPolicy(retentionPolicy.get) - def convert(messages: Seq[InfluxDBWriteMessage[T, C]]): BatchPoints = + def convert(messages: Seq[InfluxDbWriteMessage[T, C]]): BatchPoints = messages match { case head :: tail => { builder.point(head.point.asInstanceOf[Point]) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala similarity index 80% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index ccf27a0016..e4ee510ae0 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDBSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -5,12 +5,12 @@ package akka.stream.alpakka.influxdb.impl import akka.annotation.InternalApi -import akka.stream.alpakka.influxdb.InfluxDBSettings +import akka.stream.alpakka.influxdb.InfluxDbSettings import akka.stream.{Attributes, Outlet, SourceShape} import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} -import org.influxdb.impl.InfluxDBResultMapperHelper +import org.influxdb.impl.InfluxDbResultMapperHelper import scala.collection.JavaConverters._ @@ -18,17 +18,17 @@ import scala.collection.JavaConverters._ * INTERNAL API */ @InternalApi -private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], - settings: InfluxDBSettings, +private[influxdb] final class InfluxDbSourceStage[T](clazz: Class[T], + settings: InfluxDbSettings, influxDB: InfluxDB, query: Query) extends GraphStage[SourceShape[T]] { - val out: Outlet[T] = Outlet("InfluxDB.out") + val out: Outlet[T] = Outlet("InfluxDb.out") override val shape = SourceShape(out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new InfluxDBSourceLogic[T](clazz, settings, influxDB, query, out, shape) + new InfluxDbSourceLogic[T](clazz, settings, influxDB, query, out, shape) } @@ -36,8 +36,8 @@ private[influxdb] final class InfluxDBSourceStage[T](clazz: Class[T], * INTERNAL API */ @InternalApi -private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], - settings: InfluxDBSettings, +private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], + settings: InfluxDbSettings, influxDB: InfluxDB, query: Query, outlet: Outlet[T], @@ -48,10 +48,10 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], setHandler(outlet, this) var dataRetrieved: Option[QueryResult] = None - var resultMapperHelper: InfluxDBResultMapperHelper = _ + var resultMapperHelper: InfluxDbResultMapperHelper = _ override def preStart(): Unit = { - resultMapperHelper = new InfluxDBResultMapperHelper + resultMapperHelper = new InfluxDbResultMapperHelper resultMapperHelper.cacheClassFields(clazz) val queryResult = influxDB.query(query) @@ -78,14 +78,14 @@ private[influxdb] final class InfluxDBSourceLogic[T](clazz: Class[T], * INTERNAL API */ @InternalApi -private[influxdb] final class InfluxDBRawSourceStage(query: Query, influxDB: InfluxDB) +private[influxdb] final class InfluxDbRawSourceStage(query: Query, influxDB: InfluxDB) extends GraphStage[SourceShape[QueryResult]] { - val out: Outlet[QueryResult] = Outlet("InfluxDB.out") + val out: Outlet[QueryResult] = Outlet("InfluxDb.out") override val shape = SourceShape(out) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - new InfluxDBSourceRawLogic(query, influxDB, out, shape) + new InfluxDbSourceRawLogic(query, influxDB, out, shape) } @@ -93,7 +93,7 @@ private[influxdb] final class InfluxDBRawSourceStage(query: Query, influxDB: Inf * INTERNAL API */ @InternalApi -private[influxdb] final class InfluxDBSourceRawLogic(query: Query, +private[influxdb] final class InfluxDbSourceRawLogic(query: Query, influxDB: InfluxDB, outlet: Outlet[QueryResult], shape: SourceShape[QueryResult]) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala deleted file mode 100644 index 9849cf07df..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBFlow.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright (C) 2016-2019 Lightbend Inc. - */ - -package akka.stream.alpakka.influxdb.javadsl - -import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} -import org.influxdb.InfluxDB -import akka.stream.javadsl.Flow -import akka.stream.alpakka.influxdb.scaladsl - -object InfluxDBFlow { - - def create[T]( - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - scaladsl.InfluxDBFlow.create(settings)(influxDB).asJava - - def typed[T]( - clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - scaladsl.InfluxDBFlow.typed(clazz, settings)(influxDB).asJava - - def createWithPassThrough[T, C]( - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - scaladsl.InfluxDBFlow.createWithPassThrough(settings)(influxDB).asJava - - def typedWithPassThrough[T, C]( - clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - scaladsl.InfluxDBFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava - - def createWithContext[T, C]( - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - scaladsl.InfluxDBFlow.createWithContext(settings)(influxDB).asJava - - def typedWithContext[T, C]( - clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - scaladsl.InfluxDBFlow.typedWithContext(clazz, settings)(influxDB).asJava - -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala new file mode 100644 index 0000000000..61e69097df --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package akka.stream.alpakka.influxdb.javadsl + +import akka.NotUsed +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import org.influxdb.InfluxDB +import akka.stream.javadsl.Flow +import akka.stream.alpakka.influxdb.scaladsl + +object InfluxDbFlow { + + def create[T]( + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = + scaladsl.InfluxDbFlow.create(settings)(influxDB).asJava + + def typed[T]( + clazz: Class[T], + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = + scaladsl.InfluxDbFlow.typed(clazz, settings)(influxDB).asJava + + def createWithPassThrough[T, C]( + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = + scaladsl.InfluxDbFlow.createWithPassThrough(settings)(influxDB).asJava + + def typedWithPassThrough[T, C]( + clazz: Class[T], + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = + scaladsl.InfluxDbFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava + + def createWithContext[T, C]( + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = + scaladsl.InfluxDbFlow.createWithContext(settings)(influxDB).asJava + + def typedWithContext[T, C]( + clazz: Class[T], + settings: InfluxDbSettings, + influxDB: InfluxDB + ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = + scaladsl.InfluxDbFlow.typedWithContext(clazz, settings)(influxDB).asJava + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala similarity index 57% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala index 9c8db5cfa8..a564013010 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala @@ -7,7 +7,7 @@ package akka.stream.alpakka.influxdb.javadsl import java.util.concurrent.CompletionStage import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.javadsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -15,21 +15,21 @@ import org.influxdb.dto.Point /** * Java API. */ -object InfluxDBSink { +object InfluxDbSink { def create( - settings: InfluxDBSettings, + settings: InfluxDbSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Sink[InfluxDBWriteMessage[Point, NotUsed], CompletionStage[Done]] = - InfluxDBFlow + ): akka.stream.javadsl.Sink[InfluxDbWriteMessage[Point, NotUsed], CompletionStage[Done]] = + InfluxDbFlow .create(settings, influxDB) - .toMat(Sink.ignore[InfluxDBWriteResult[Point, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + .toMat(Sink.ignore[InfluxDbWriteResult[Point, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) def typed[T](clazz: Class[T], - settings: InfluxDBSettings, - influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDBWriteMessage[T, NotUsed], CompletionStage[Done]] = - InfluxDBFlow + settings: InfluxDbSettings, + influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDbWriteMessage[T, NotUsed], CompletionStage[Done]] = + InfluxDbFlow .typed(clazz, settings, influxDB) - .toMat(Sink.ignore[InfluxDBWriteResult[T, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + .toMat(Sink.ignore[InfluxDbWriteResult[T, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala similarity index 56% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala index 9145e33c78..c549b814d7 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala @@ -5,29 +5,29 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.InfluxDBSettings +import akka.stream.alpakka.influxdb.InfluxDbSettings import akka.stream.javadsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} -import akka.stream.alpakka.influxdb.impl.{InfluxDBRawSourceStage, InfluxDBSourceStage} +import akka.stream.alpakka.influxdb.impl.{InfluxDbRawSourceStage, InfluxDbSourceStage} /** * Java API to create InfluxDB sources. */ -object InfluxDBSource { +object InfluxDbSource { /** - * Java API: creates an [[InfluxDBRawSourceStage]] from a given statement. + * Java API: creates an [[InfluxDbRawSourceStage]] from a given statement. */ def create(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = - Source.fromGraph(new InfluxDBRawSourceStage(query, influxDB)) + Source.fromGraph(new InfluxDbRawSourceStage(query, influxDB)) /** - * Java API: creates an [[InfluxDBSourceStage]] of elements of `T` from `query`. + * Java API: creates an [[InfluxDbSourceStage]] of elements of `T` from `query`. */ - def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDbSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( - new InfluxDBSourceStage[T]( + new InfluxDbSourceStage[T]( clazz, settings, influxDB, diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala deleted file mode 100644 index 1a8004edd1..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBFlow.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright (C) 2016-2019 Lightbend Inc. - */ - -package akka.stream.alpakka.influxdb.scaladsl - -import akka.NotUsed -import akka.stream.alpakka.influxdb.{impl, InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} -import akka.stream.scaladsl.Flow -import org.influxdb.InfluxDB - -import scala.collection.immutable - -/** - * Scala API to create InfluxDB flows. - */ -object InfluxDBFlow { - - def create[T](settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - Flow[InfluxDBWriteMessage[T, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, NotUsed](None, influxDB)) - .mapConcat(identity) - - def typed[T](clazz: Class[T], settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, NotUsed], InfluxDBWriteResult[T, NotUsed], NotUsed] = - Flow[InfluxDBWriteMessage[T, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, NotUsed](Some(clazz), influxDB)) - .mapConcat(identity) - - def createWithPassThrough[T, C](settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - Flow[InfluxDBWriteMessage[T, C]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, C](None, influxDB)) - .mapConcat(identity) - - def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[InfluxDBWriteMessage[T, C], InfluxDBWriteResult[T, C], NotUsed] = - Flow[InfluxDBWriteMessage[T, C]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDBFlowStage[T, C](Some(clazz), influxDB)) - .mapConcat(identity) - - def createWithContext[T, C](settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - Flow[(InfluxDBWriteMessage[T, NotUsed], C)] - .map { - case (wm, pt) => - InfluxDBWriteMessage(wm.point, pt) - } - .via(createWithPassThrough(settings)) - .map { wr => - (wr, wr.writeMessage.passThrough) - } - - def typedWithContext[T, C](clazz: Class[T], settings: InfluxDBSettings)( - implicit influxDB: InfluxDB - ): Flow[(InfluxDBWriteMessage[T, NotUsed], C), (InfluxDBWriteResult[T, C], C), NotUsed] = - Flow[(InfluxDBWriteMessage[T, NotUsed], C)] - .map { - case (wm, pt) => - InfluxDBWriteMessage(wm.point, pt) - } - .via(typedWithPassThrough(clazz, settings)) - .map { wr => - (wr, wr.writeMessage.passThrough) - } - -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala index 4b6852aed6..607e846f88 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala @@ -5,8 +5,8 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.InfluxDBSettings -import akka.stream.alpakka.influxdb.impl.{InfluxDBRawSourceStage, InfluxDBSourceStage} +import akka.stream.alpakka.influxdb.InfluxDbSettings +import akka.stream.alpakka.influxdb.impl.{InfluxDbRawSourceStage, InfluxDbSourceStage} import akka.stream.scaladsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} @@ -14,20 +14,20 @@ import org.influxdb.dto.{Query, QueryResult} /** * Scala API. */ -object InfluxDBSource { +object InfluxDbSource { /** - * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDBRawSourceStage]] from a given statement. + * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDbRawSourceStage]] from a given statement. */ def apply(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = - Source.fromGraph(new InfluxDBRawSourceStage(query, influxDB)) + Source.fromGraph(new InfluxDbRawSourceStage(query, influxDB)) /** * Read elements of `T` from `className` or by `query`. */ - def typed[T](clazz: Class[T], settings: InfluxDBSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDbSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( - new InfluxDBSourceStage[T]( + new InfluxDbSourceStage[T]( clazz, settings, influxDB, diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala new file mode 100644 index 0000000000..ccfa261fa9 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package akka.stream.alpakka.influxdb.scaladsl + +import akka.NotUsed +import akka.stream.alpakka.influxdb.{impl, InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.scaladsl.Flow +import org.influxdb.InfluxDB + +import scala.collection.immutable + +/** + * Scala API to create InfluxDB flows. + */ +object InfluxDbFlow { + + def create[T](settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = + Flow[InfluxDbWriteMessage[T, NotUsed]] + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) + .via(new impl.InfluxDbFlowStage[T, NotUsed](None, influxDB)) + .mapConcat(identity) + + def typed[T](clazz: Class[T], settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = + Flow[InfluxDbWriteMessage[T, NotUsed]] + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) + .via(new impl.InfluxDbFlowStage[T, NotUsed](Some(clazz), influxDB)) + .mapConcat(identity) + + def createWithPassThrough[T, C](settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = + Flow[InfluxDbWriteMessage[T, C]] + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) + .via(new impl.InfluxDbFlowStage[T, C](None, influxDB)) + .mapConcat(identity) + + def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = + Flow[InfluxDbWriteMessage[T, C]] + .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) + .via(new impl.InfluxDbFlowStage[T, C](Some(clazz), influxDB)) + .mapConcat(identity) + + def createWithContext[T, C](settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = + Flow[(InfluxDbWriteMessage[T, NotUsed], C)] + .map { + case (wm, pt) => + InfluxDbWriteMessage(wm.point, pt) + } + .via(createWithPassThrough(settings)) + .map { wr => + (wr, wr.writeMessage.passThrough) + } + + def typedWithContext[T, C](clazz: Class[T], settings: InfluxDbSettings)( + implicit influxDB: InfluxDB + ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = + Flow[(InfluxDbWriteMessage[T, NotUsed], C)] + .map { + case (wm, pt) => + InfluxDbWriteMessage(wm.point, pt) + } + .via(typedWithPassThrough(clazz, settings)) + .map { wr => + (wr, wr.writeMessage.passThrough) + } + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala similarity index 57% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index faf9c9b41a..f1664e5ce6 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -5,25 +5,25 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage} +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage} import akka.stream.scaladsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point import scala.concurrent.Future -object InfluxDBSink { +object InfluxDbSink { def apply( - settings: InfluxDBSettings - )(implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[Point, NotUsed], Future[Done]] = - InfluxDBFlow.create[Point](settings).toMat(Sink.ignore)(Keep.right) + settings: InfluxDbSettings + )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[Point, NotUsed], Future[Done]] = + InfluxDbFlow.create[Point](settings).toMat(Sink.ignore)(Keep.right) def typed[T]( clazz: Class[T], - settings: InfluxDBSettings - )(implicit influxDB: InfluxDB): Sink[InfluxDBWriteMessage[T, NotUsed], Future[Done]] = - InfluxDBFlow + settings: InfluxDbSettings + )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[T, NotUsed], Future[Done]] = + InfluxDbFlow .typed(clazz, settings) .toMat(Sink.ignore)(Keep.right) diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java b/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java similarity index 68% rename from influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java rename to influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java index 47a7c9393a..2b11154a37 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBCpu.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java @@ -8,12 +8,12 @@ import org.influxdb.annotation.Measurement; -@Measurement(name = "cpu", database = "InfluxDBTest") -public class InfluxDBCpu extends Cpu { +@Measurement(name = "cpu", database = "InfluxDbTest") +public class InfluxDbCpu extends Cpu { - public InfluxDBCpu() {} + public InfluxDbCpu() {} - public InfluxDBCpu( + public InfluxDbCpu( Instant time, String hostname, String region, @@ -23,8 +23,8 @@ public InfluxDBCpu( super(time, hostname, region, idle, happydevop, uptimeSecs); } - public InfluxDBCpu cloneAt(Instant time) { - return new InfluxDBCpu( + public InfluxDbCpu cloneAt(Instant time) { + return new InfluxDbCpu( time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); } } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceCpu.java similarity index 69% rename from influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java rename to influxdb/src/test/java/docs/javadsl/InfluxDbSourceCpu.java index 31df09d95c..5583c1b110 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceCpu.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceCpu.java @@ -8,12 +8,12 @@ import org.influxdb.annotation.Measurement; -@Measurement(name = "cpu", database = "InfluxDBSourceTest") -public class InfluxDBSourceCpu extends Cpu { +@Measurement(name = "cpu", database = "InfluxDbSourceTest") +public class InfluxDbSourceCpu extends Cpu { - public InfluxDBSourceCpu() {} + public InfluxDbSourceCpu() {} - public InfluxDBSourceCpu( + public InfluxDbSourceCpu( Instant time, String hostname, String region, diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java similarity index 83% rename from influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java rename to influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java index 69d501e50c..56132a1d5c 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java @@ -21,8 +21,8 @@ import akka.japi.Pair; import akka.stream.ActorMaterializer; import akka.stream.Materializer; -import akka.stream.alpakka.influxdb.InfluxDBSettings; -import akka.stream.alpakka.influxdb.javadsl.InfluxDBSource; +import akka.stream.alpakka.influxdb.InfluxDbSettings; +import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; import akka.testkit.javadsl.TestKit; @@ -31,13 +31,13 @@ import static docs.javadsl.TestUtils.populateDatabase; import static docs.javadsl.TestUtils.setupConnection; -public class InfluxDBSourceTest { +public class InfluxDbSourceTest { private static ActorSystem system; private static Materializer materializer; private static InfluxDB influxDB; - private static final String DATABASE_NAME = "InfluxDBSourceTest"; + private static final String DATABASE_NAME = "InfluxDbSourceTest"; private static Pair setupMaterializer() { // #init-mat @@ -64,7 +64,7 @@ public static void teardown() { @Before public void setUp() throws Exception { - populateDatabase(influxDB, InfluxDBSourceCpu.class); + populateDatabase(influxDB, InfluxDbSourceCpu.class); } @After @@ -77,11 +77,11 @@ public void cleanUp() { public void streamQueryResult() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); - CompletionStage> rows = - InfluxDBSource.typed(InfluxDBSourceCpu.class, InfluxDBSettings.Default(), influxDB, query) + CompletionStage> rows = + InfluxDbSource.typed(InfluxDbSourceCpu.class, InfluxDbSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); - List cpus = rows.toCompletableFuture().get(); + List cpus = rows.toCompletableFuture().get(); Assert.assertEquals(2, cpus.size()); } @@ -91,7 +91,7 @@ public void streamRawQueryResult() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage> completionStage = - InfluxDBSource.create(influxDB, query).runWith(Sink.seq(), materializer); + InfluxDbSource.create(influxDB, query).runWith(Sink.seq(), materializer); List queryResults = completionStage.toCompletableFuture().get(); QueryResult queryResult = queryResults.get(0); diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java similarity index 77% rename from influxdb/src/test/java/docs/javadsl/InfluxDBTest.java rename to influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index 8ea77630d7..b69c7bf827 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDBTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -25,10 +25,10 @@ import akka.japi.Pair; import akka.stream.ActorMaterializer; import akka.stream.Materializer; -import akka.stream.alpakka.influxdb.InfluxDBSettings; -import akka.stream.alpakka.influxdb.InfluxDBWriteMessage; -import akka.stream.alpakka.influxdb.javadsl.InfluxDBSink; -import akka.stream.alpakka.influxdb.javadsl.InfluxDBSource; +import akka.stream.alpakka.influxdb.InfluxDbSettings; +import akka.stream.alpakka.influxdb.InfluxDbWriteMessage; +import akka.stream.alpakka.influxdb.javadsl.InfluxDbSink; +import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; import akka.testkit.javadsl.TestKit; @@ -38,13 +38,13 @@ import static docs.javadsl.TestUtils.resultToPoint; import static docs.javadsl.TestUtils.setupConnection; -public class InfluxDBTest { +public class InfluxDbTest { private static ActorSystem system; private static Materializer materializer; private static InfluxDB influxDB; - private static final String DATABASE_NAME = "InfluxDBTest"; + private static final String DATABASE_NAME = "InfluxDbTest"; private static Pair setupMaterializer() { // #init-mat @@ -71,7 +71,7 @@ public static void teardown() { @Before public void setUp() throws Exception { - populateDatabase(influxDB, InfluxDBCpu.class); + populateDatabase(influxDB, InfluxDbCpu.class); } @After @@ -84,20 +84,20 @@ public void cleanUp() { public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = - InfluxDBSource.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB, query) + InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB, query) .map( cpu -> { - InfluxDBCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); - return InfluxDBWriteMessage.create(clonedCpu, NotUsed.notUsed()); + InfluxDbCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); + return InfluxDbWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) .runWith( - InfluxDBSink.typed(InfluxDBCpu.class, InfluxDBSettings.Default(), influxDB), + InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); CompletionStage> sources = - InfluxDBSource.typed(Cpu.class, InfluxDBSettings.Default(), influxDB, query) + InfluxDbSource.typed(Cpu.class, InfluxDbSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); Assert.assertEquals(4, sources.toCompletableFuture().get().size()); @@ -108,15 +108,15 @@ public void testConsumeAndPublishMeasurements() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = - InfluxDBSource.create(influxDB, query) + InfluxDbSource.create(influxDB, query) .map(queryResult -> points(queryResult)) .mapConcat(i -> i) - .runWith(InfluxDBSink.create(InfluxDBSettings.Default(), influxDB), materializer); + .runWith(InfluxDbSink.create(InfluxDbSettings.Default(), influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); List queryResult = - InfluxDBSource.create(influxDB, query) + InfluxDbSource.create(influxDB, query) .runWith(Sink.seq(), materializer) .toCompletableFuture() .get(); @@ -126,14 +126,14 @@ public void testConsumeAndPublishMeasurements() throws Exception { Assert.assertEquals(4, resultSize); } - private List> points(QueryResult queryResult) { - List> points = new ArrayList<>(); + private List> points(QueryResult queryResult) { + List> points = new ArrayList<>(); for (QueryResult.Result result : queryResult.getResults()) { for (QueryResult.Series series : result.getSeries()) { for (List rows : series.getValues()) { - InfluxDBWriteMessage influxDBWriteMessage = - InfluxDBWriteMessage.create(resultToPoint(series, rows), NotUsed.notUsed()); + InfluxDbWriteMessage influxDBWriteMessage = + InfluxDbWriteMessage.create(resultToPoint(series, rows), NotUsed.notUsed()); points.add(influxDBWriteMessage); } } diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index 8f6f1909aa..a6ce0871d6 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -8,8 +8,8 @@ import java.util.concurrent.TimeUnit import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage, InfluxDBWriteResult} -import akka.stream.alpakka.influxdb.scaladsl.InfluxDBFlow +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.scaladsl.InfluxDbFlow import akka.stream.scaladsl.{Sink, Source} import akka.testkit.TestKit import org.influxdb.InfluxDB @@ -40,16 +40,16 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B "invalid model" in assertAllStagesStopped { val result = Source( - List(InfluxDBWriteMessage(InvalidModel("Invalid measurement one")), - InfluxDBWriteMessage(InvalidModel("Invalid measurement two"))) - ).via(InfluxDBFlow.create[InvalidModel](InfluxDBSettings())) + List(InfluxDbWriteMessage(InvalidModel("Invalid measurement one")), + InfluxDbWriteMessage(InvalidModel("Invalid measurement two"))) + ).via(InfluxDbFlow.create[InvalidModel](InfluxDbSettings())) .recover { - case _: RuntimeException => InfluxDBWriteResult(null, Some("error occurred")) + case _: RuntimeException => InfluxDbWriteResult(null, Some("error occurred")) } .runWith(Sink.seq) .futureValue - result mustBe Seq(InfluxDBWriteResult(null, Some("error occurred"))) + result mustBe Seq(InfluxDbWriteResult(null, Some("error occurred"))) } "mixed model" in assertAllStagesStopped { @@ -61,14 +61,14 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B .addField("free", 1L) .build() - val validMessage = InfluxDBWriteMessage(point) + val validMessage = InfluxDbWriteMessage(point) .withDatabaseName(DatabaseName) val result = Source( List( validMessage ) - ).via(InfluxDBFlow.create[Point](InfluxDBSettings())) + ).via(InfluxDbFlow.create[Point](InfluxDbSettings())) .runWith(Sink.seq) .futureValue diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceCpu.java similarity index 56% rename from influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java rename to influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceCpu.java index 5ae1664d2c..7af349f5b0 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceCpu.java +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceCpu.java @@ -10,16 +10,16 @@ import docs.javadsl.Cpu; -@Measurement(name = "cpu", database = "InfluxDBSourceSpec") -public class InfluxDBSourceCpu extends Cpu { - public InfluxDBSourceCpu() { +@Measurement(name = "cpu", database = "InfluxDbSourceSpec") +public class InfluxDbSourceCpu extends Cpu { + public InfluxDbSourceCpu() { } - public InfluxDBSourceCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + public InfluxDbSourceCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { super(time, hostname, region, idle, happydevop, uptimeSecs); } - public InfluxDBSpecCpu cloneAt(Instant time) { - return new InfluxDBSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + public InfluxDbSpecCpu cloneAt(Instant time) { + return new InfluxDbSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala similarity index 84% rename from influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala rename to influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index 58d3399c85..26b62d0889 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.scaladsl.InfluxDBSource +import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit import org.influxdb.InfluxDB @@ -16,14 +16,14 @@ import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import docs.javadsl.TestUtils._ import org.influxdb.dto.Query -class InfluxDBSourceSpec +class InfluxDbSourceSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { - final val DatabaseName = "InfluxDBSourceSpec" + final val DatabaseName = "InfluxDbSourceSpec" implicit val system = ActorSystem() implicit val mat = ActorMaterializer() @@ -37,7 +37,7 @@ class InfluxDBSourceSpec TestKit.shutdownActorSystem(system) override def beforeEach(): Unit = - populateDatabase(influxDB, classOf[InfluxDBSourceCpu]) + populateDatabase(influxDB, classOf[InfluxDbSourceCpu]) override def afterEach() = cleanDatabase(influxDB, DatabaseName) @@ -46,7 +46,7 @@ class InfluxDBSourceSpec // #run-typed val query = new Query("SELECT * FROM cpu", DatabaseName); - val influxDBResult = InfluxDBSource(influxDB, query).runWith(Sink.seq) + val influxDBResult = InfluxDbSource(influxDB, query).runWith(Sink.seq) val resultToAssert = influxDBResult.futureValue.head val values = resultToAssert.getResults.get(0).getSeries().get(0).getValues diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala similarity index 67% rename from influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala rename to influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 962a7d2938..5b0b7aadf3 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -12,15 +12,15 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordS import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDBSettings, InfluxDBWriteMessage} -import akka.stream.alpakka.influxdb.scaladsl.{InfluxDBSink, InfluxDBSource} +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage} +import akka.stream.alpakka.influxdb.scaladsl.{InfluxDbSink, InfluxDbSource} import akka.testkit.TestKit import docs.javadsl.TestUtils._ import akka.stream.scaladsl.Sink import scala.collection.JavaConverters._ -class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { +class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { implicit val system = ActorSystem() implicit val mat = ActorMaterializer() @@ -36,7 +36,7 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi TestKit.shutdownActorSystem(system) override def beforeEach(): Unit = - populateDatabase(influxDB, classOf[InfluxDBSpecCpu]) + populateDatabase(influxDB, classOf[InfluxDbSpecCpu]) override def afterEach() = cleanDatabase(influxDB, DatabaseName) @@ -44,29 +44,29 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "support typed source" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); val measurements = - InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) measurements.futureValue.map(_.getHostname) mustBe List("local_1", "local_2") } - "InfluxDBFlow" should { + "InfluxDbFlow" should { "consume and publish measurements using typed" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); - val f1 = InfluxDBSource - .typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query) - .map { cpu: InfluxDBSpecCpu => + val f1 = InfluxDbSource + .typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query) + .map { cpu: InfluxDbSpecCpu => { val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) - InfluxDBWriteMessage(clonedCpu) + InfluxDbWriteMessage(clonedCpu) } } - .runWith(InfluxDBSink.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings())) + .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings())) f1.futureValue mustBe Done - val f2 = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + val f2 = InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } @@ -74,25 +74,25 @@ class InfluxDBSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "consume and publish measurements" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); - val f1 = InfluxDBSource(influxDB, query) + val f1 = InfluxDbSource(influxDB, query) .mapConcat(resultToPoints) - .runWith(InfluxDBSink(InfluxDBSettings())) + .runWith(InfluxDbSink(InfluxDbSettings())) f1.futureValue mustBe Done - val f2 = InfluxDBSource.typed(classOf[InfluxDBSpecCpu], InfluxDBSettings(), influxDB, query).runWith(Sink.seq) + val f2 = InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } - def resultToPoints(queryResult: QueryResult): List[InfluxDBWriteMessage[Point, NotUsed]] = { + def resultToPoints(queryResult: QueryResult): List[InfluxDbWriteMessage[Point, NotUsed]] = { val points = for { results <- queryResult.getResults.asScala series <- results.getSeries.asScala values <- series.getValues.asScala } yield ( - InfluxDBWriteMessage(resultToPoint(series, values)) + InfluxDbWriteMessage(resultToPoint(series, values)) ) points.toList } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java similarity index 56% rename from influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java rename to influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java index 3e0ac57e40..84ade2ea23 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDBSpecCpu.java +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java @@ -10,16 +10,16 @@ import docs.javadsl.Cpu; -@Measurement(name = "cpu", database = "InfluxDBSpec") -public class InfluxDBSpecCpu extends Cpu { - public InfluxDBSpecCpu() { +@Measurement(name = "cpu", database = "InfluxDbSpec") +public class InfluxDbSpecCpu extends Cpu { + public InfluxDbSpecCpu() { } - public InfluxDBSpecCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + public InfluxDbSpecCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { super(time, hostname, region, idle, happydevop, uptimeSecs); } - public InfluxDBSpecCpu cloneAt(Instant time) { - return new InfluxDBSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + public InfluxDbSpecCpu cloneAt(Instant time) { + return new InfluxDbSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); } } From fb3f93d108aa98ff3d7fc1fa4ddce883c4b9dc56 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 11:28:58 +0100 Subject: [PATCH 54/89] Separated logic on flow for InfluxDb connector --- .../influxdb/impl/InfluxDbFlowStage.scala | 148 ++++++++++-------- 1 file changed, 82 insertions(+), 66 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index b84a59de5a..3d98b394c9 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -31,84 +31,44 @@ private[influxdb] class InfluxDbFlowStage[T, C]( override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = clazz match { - case Some(c) => new InfluxDbMapperRecordLogic - case None => new InfluxDbRecordLogic + case Some(c) => new InfluxDbMapperRecordLogic(influxDB, in, out, shape) + case None => new InfluxDbRecordLogic(influxDB, in, out, shape) } - sealed abstract class InfluxDbLogic extends GraphStageLogic(shape) with InHandler with OutHandler { +} - protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit +private[influxdb] sealed abstract class InfluxDbLogic[T, C]( + influxDB: InfluxDB, + in: Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]], + out: Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]], + shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]] +) extends GraphStageLogic(shape) + with InHandler + with OutHandler { - setHandlers(in, out, this) + setHandlers(in, out, this) - override def onPull(): Unit = if (!isClosed(in) && !hasBeenPulled(in)) pull(in) + protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit - override def onPush(): Unit = { - val messages = grab(in) - if (messages.nonEmpty) { + override def onPull(): Unit = if (!isClosed(in) && !hasBeenPulled(in)) pull(in) - influxDB.enableBatch(BatchOptions.DEFAULTS) - write(messages) - val writtenMessages = messages.map(m => new InfluxDbWriteResult(m, None)) - influxDB.close() - push(out, writtenMessages) - } + override def onPush(): Unit = { + val messages = grab(in) + if (messages.nonEmpty) { - tryPull(in) + influxDB.enableBatch(BatchOptions.DEFAULTS) + write(messages) + val writtenMessages = messages.map(m => new InfluxDbWriteResult(m, None)) + influxDB.close() + push(out, writtenMessages) } - } - - final class InfluxDbRecordLogic extends InfluxDbLogic { - - override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = - messages - .filter { - case InfluxDbWriteMessage(_: Point, _, _, _) => { - true - } - case InfluxDbWriteMessage(_: AnyRef, _, _, _) => { - failStage(new RuntimeException(s"unexpected type Point required")) - false - } - } - .groupBy(im => (im.databaseName, im.retentionPolicy)) - .map(wm => toBatchPoints(wm._1._1, wm._1._2, wm._2)) - .foreach(influxDB.write) + tryPull(in) } - final class InfluxDbMapperRecordLogic extends InfluxDbLogic { - - private val mapperHelper: InfluxDbResultMapperHelper = new InfluxDbResultMapperHelper - - override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = - messages - .groupBy(groupByDbRp) - .map(convertToBatchPoints) - .foreach(influxDB.write) - - def groupByDbRp(im: InfluxDbWriteMessage[T, C]) = - ( - im.databaseName match { - case dbn: Some[String] => dbn - case None => Some(mapperHelper.databaseName(im.point)) - }, - im.retentionPolicy match { - case dbn: Some[String] => dbn - case None => Some(mapperHelper.retentionPolicy(im.point)) - } - ) - - def convertToBatchPoints(wm: ((Some[String], Some[String]), immutable.Seq[InfluxDbWriteMessage[T, C]])) = - toBatchPoints(wm._1._1, - wm._1._2, - wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) - } - - private def toBatchPoints(databaseName: Option[String], - retentionPolicy: Option[String], - seq: Seq[InfluxDbWriteMessage[T, C]]) = { - + protected def toBatchPoints(databaseName: Option[String], + retentionPolicy: Option[String], + seq: Seq[InfluxDbWriteMessage[T, C]]) = { val builder = databaseName match { case Some(databaseName) => BatchPoints.database(databaseName) case None => BatchPoints.builder() @@ -129,3 +89,59 @@ private[influxdb] class InfluxDbFlowStage[T, C]( } } + +private[influxdb] final class InfluxDbRecordLogic[T, C]( + influxDB: InfluxDB, + in: Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]], + out: Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]], + shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]] +) extends InfluxDbLogic(influxDB, in, out, shape) { + + override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = + messages + .filter { + case InfluxDbWriteMessage(_: Point, _, _, _) => { + true + } + case InfluxDbWriteMessage(_: AnyRef, _, _, _) => { + failStage(new RuntimeException(s"unexpected type Point required")) + false + } + } + .groupBy(im => (im.databaseName, im.retentionPolicy)) + .map(wm => toBatchPoints(wm._1._1, wm._1._2, wm._2)) + .foreach(influxDB.write) +} + +private[influxdb] final class InfluxDbMapperRecordLogic[T, C]( + influxDB: InfluxDB, + in: Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]], + out: Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]], + shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]] +) extends InfluxDbLogic(influxDB, in, out, shape) { + + private val mapperHelper: InfluxDbResultMapperHelper = new InfluxDbResultMapperHelper + + override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = + messages + .groupBy(groupByDbRp) + .map(convertToBatchPoints) + .foreach(influxDB.write) + + def groupByDbRp(im: InfluxDbWriteMessage[T, C]) = + ( + im.databaseName match { + case dbn: Some[String] => dbn + case None => Some(mapperHelper.databaseName(im.point)) + }, + im.retentionPolicy match { + case dbn: Some[String] => dbn + case None => Some(mapperHelper.retentionPolicy(im.point)) + } + ) + + def convertToBatchPoints(wm: ((Some[String], Some[String]), immutable.Seq[InfluxDbWriteMessage[T, C]])) = + toBatchPoints(wm._1._1, + wm._1._2, + wm._2.map(im => im.withPoint(mapperHelper.convertModelToPoint(im.point).asInstanceOf[T]))) +} From fe55ee898df5e8a6e467c2d47bdca442aaab23fe Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 11:44:14 +0100 Subject: [PATCH 55/89] Added @InternalApi on InfluxDbFlowStage for InfluxDb connector --- .../stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index 3d98b394c9..f2f815a1ed 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -4,6 +4,7 @@ package akka.stream.alpakka.influxdb.impl +import akka.annotation.InternalApi import akka.stream._ import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} @@ -14,9 +15,12 @@ import org.influxdb.BatchOptions import org.influxdb.dto.{BatchPoints, Point} import org.influxdb.impl.InfluxDbResultMapperHelper +import scala.annotation.tailrec + /** * INTERNAL API */ +@InternalApi private[influxdb] class InfluxDbFlowStage[T, C]( clazz: Option[Class[T]], influxDB: InfluxDB @@ -76,6 +80,7 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( if (retentionPolicy.isDefined) builder.retentionPolicy(retentionPolicy.get) + @tailrec def convert(messages: Seq[InfluxDbWriteMessage[T, C]]): BatchPoints = messages match { case head :: tail => { From 364cdd985e445849765c7def139f7940d645d356 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 11:49:45 +0100 Subject: [PATCH 56/89] Refactored file name for InfluxDb connector --- .../scaladsl/{InfluxDBSource.scala => InfluxDbSource.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/{InfluxDBSource.scala => InfluxDbSource.scala} (100%) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala similarity index 100% rename from influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDBSource.scala rename to influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala From 92e6943ad2576ad02db49c9965ebf4c452eeab58 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 12:54:36 +0100 Subject: [PATCH 57/89] Set source queries to be executed on the IODispatcher for InfluxDb connector --- .../influxdb/impl/InfluxDbSourceStage.scala | 27 ++++++++++++++----- .../test/java/docs/javadsl/InfluxDbTest.java | 6 ++--- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index e4ee510ae0..c738dfb78f 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -6,9 +6,9 @@ package akka.stream.alpakka.influxdb.impl import akka.annotation.InternalApi import akka.stream.alpakka.influxdb.InfluxDbSettings -import akka.stream.{Attributes, Outlet, SourceShape} +import akka.stream.{ActorAttributes, Attributes, Outlet, SourceShape} import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} -import org.influxdb.InfluxDB +import org.influxdb.{InfluxDB, InfluxDBException} import org.influxdb.dto.{Query, QueryResult} import org.influxdb.impl.InfluxDbResultMapperHelper @@ -27,6 +27,9 @@ private[influxdb] final class InfluxDbSourceStage[T](clazz: Class[T], val out: Outlet[T] = Outlet("InfluxDb.out") override val shape = SourceShape(out) + override protected def initialAttributes: Attributes = + super.initialAttributes and Attributes(ActorAttributes.IODispatcher) + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new InfluxDbSourceLogic[T](clazz, settings, influxDB, query, out, shape) @@ -47,20 +50,29 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], setHandler(outlet, this) + var queryExecuted: Boolean = false var dataRetrieved: Option[QueryResult] = None var resultMapperHelper: InfluxDbResultMapperHelper = _ override def preStart(): Unit = { resultMapperHelper = new InfluxDbResultMapperHelper resultMapperHelper.cacheClassFields(clazz) + } - val queryResult = influxDB.query(query) - if (!queryResult.hasError) { - dataRetrieved = Some(queryResult) + private def runQuery() = + if (!queryExecuted) { + val queryResult = influxDB.query(query) + if (!queryResult.hasError) { + dataRetrieved = Some(queryResult) + } else { + failStage(new InfluxDBException(queryResult.getError)) + dataRetrieved = None + } + queryExecuted = true } - } - override def onPull(): Unit = + override def onPull(): Unit = { + runQuery() dataRetrieved match { case None => completeStage() case Some(queryResult) => { @@ -72,6 +84,7 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], dataRetrieved = None } } + } } /** diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index b69c7bf827..530f5d22b1 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -82,6 +82,8 @@ public void cleanUp() { @Test public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { + Sink, CompletionStage> sink = + InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB); Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB, query) @@ -90,9 +92,7 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { InfluxDbCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); return InfluxDbWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) - .runWith( - InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB), - materializer); + .runWith(sink, materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); From a24957378125c8a80adc1ed6019d3587e2cf09dd Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 12:55:19 +0100 Subject: [PATCH 58/89] Set source queries to be executed on the IODispatcher for InfluxDb connector --- influxdb/src/test/java/docs/javadsl/InfluxDbTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index 530f5d22b1..b69c7bf827 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -82,8 +82,6 @@ public void cleanUp() { @Test public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { - Sink, CompletionStage> sink = - InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB); Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB, query) @@ -92,7 +90,9 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { InfluxDbCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); return InfluxDbWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) - .runWith(sink, materializer); + .runWith( + InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB), + materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); From 80d1489a324c4c205ef8f45b7fac61a97e8a2d2c Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 14:08:08 +0100 Subject: [PATCH 59/89] Set query to run on IO Dispatcher, added test on query failure on InfluxDb connector --- .../influxdb/impl/InfluxDbSourceStage.scala | 58 ++++++++++++------- .../docs/scaladsl/InfluxDbSourceSpec.scala | 16 ++++- 2 files changed, 53 insertions(+), 21 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index c738dfb78f..b743ff40d5 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -59,18 +59,6 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], resultMapperHelper.cacheClassFields(clazz) } - private def runQuery() = - if (!queryExecuted) { - val queryResult = influxDB.query(query) - if (!queryResult.hasError) { - dataRetrieved = Some(queryResult) - } else { - failStage(new InfluxDBException(queryResult.getError)) - dataRetrieved = None - } - queryExecuted = true - } - override def onPull(): Unit = { runQuery() dataRetrieved match { @@ -85,6 +73,19 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], } } } + + private def runQuery() = + if (!queryExecuted) { + val queryResult = influxDB.query(query) + if (!queryResult.hasError) { + dataRetrieved = Some(queryResult) + } else { + failStage(new InfluxDBException(queryResult.getError)) + dataRetrieved = None + } + queryExecuted = true + } + } /** @@ -97,6 +98,9 @@ private[influxdb] final class InfluxDbRawSourceStage(query: Query, influxDB: Inf val out: Outlet[QueryResult] = Outlet("InfluxDb.out") override val shape = SourceShape(out) + override protected def initialAttributes: Attributes = + super.initialAttributes and Attributes(ActorAttributes.IODispatcher) + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new InfluxDbSourceRawLogic(query, influxDB, out, shape) @@ -115,16 +119,11 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, setHandler(outlet, this) + var queryExecuted: Boolean = false var dataRetrieved: Option[QueryResult] = None - override def preStart(): Unit = { - val queryResult = influxDB.query(query) - if (!queryResult.hasError) { - dataRetrieved = Some(queryResult) - } - } - - override def onPull(): Unit = + override def onPull(): Unit = { + runQuery() dataRetrieved match { case None => completeStage() case Some(queryResult) => { @@ -132,5 +131,24 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, dataRetrieved = None } } + } + + private def runQuery() = + if (!queryExecuted) { + val queryResult = influxDB.query(query) + if (!queryResult.hasError) { + queryResult.getResults.forEach(failOnError) + dataRetrieved = Some(queryResult) + } else { + failStage(new InfluxDBException(queryResult.getError)) + dataRetrieved = None + } + queryExecuted = true + } + + private def failOnError(result: QueryResult.Result) = + if (result.hasError) { + failStage(new InfluxDBException(result.getError)) + } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index 26b62d0889..f9e420b2fb 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,10 +6,11 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer +import akka.stream.alpakka.influxdb.InfluxDbWriteResult import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit -import org.influxdb.InfluxDB +import org.influxdb.{InfluxDB, InfluxDBException} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped @@ -54,4 +55,17 @@ class InfluxDbSourceSpec values.size() mustBe 2 } + "exception on source" in assertAllStagesStopped { + val query = new Query("SELECT man() FROM invalid", DatabaseName); + + val result = InfluxDbSource(influxDB, query) //.runWith(Sink.seq) + .recover { + case e: InfluxDBException => InfluxDbWriteResult(null, Some(e.getMessage)) + } + .runWith(Sink.seq) + .futureValue + + result mustBe Seq(InfluxDbWriteResult(null, Some("undefined function man()"))) + } + } From 8876c2604bbb282b9153e2ac02f046988d3e87e3 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 19 May 2019 22:45:18 +0100 Subject: [PATCH 60/89] Fail stage on error for source on InfluxDb connector --- .../influxdb/impl/InfluxDbSourceStage.scala | 15 ++++++++++----- .../scala/docs/scaladsl/InfluxDbSourceSpec.scala | 16 +++++++++++++++- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index b743ff40d5..932241a887 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -64,11 +64,16 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], dataRetrieved match { case None => completeStage() case Some(queryResult) => { - for { - result <- queryResult.getResults.asScala - series <- result.getSeries.asScala - } emitMultiple(outlet, resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) - + for (result <- queryResult.getResults.asScala) { + if (result.hasError) { + failStage(new InfluxDBException(result.getError)) + } else { + for (series <- result.getSeries.asScala) { + emitMultiple(outlet, + resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) + } + } + } dataRetrieved = None } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index f9e420b2fb..523641d896 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.InfluxDbWriteResult +import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteResult} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit @@ -68,4 +68,18 @@ class InfluxDbSourceSpec result mustBe Seq(InfluxDbWriteResult(null, Some("undefined function man()"))) } + "exception on typed source" in assertAllStagesStopped { + val query = new Query("SELECT man() FROM invalid", DatabaseName); + + val result = InfluxDbSource + .typed(classOf[InfluxDbSourceCpu], InfluxDbSettings.Default, influxDB, query) //.runWith(Sink.seq) + .recover { + case e: InfluxDBException => InfluxDbWriteResult(null, Some(e.getMessage)) + } + .runWith(Sink.seq) + .futureValue + + result mustBe Seq(InfluxDbWriteResult(null, Some("undefined function man()"))) + } + } From ac63395972e4b3c8bf0a0bb22c32500da3ebcdf6 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 20 May 2019 08:00:35 +0100 Subject: [PATCH 61/89] Added error handling per source on InfluxDb connector --- .../influxdb/impl/InfluxDbSourceStage.scala | 13 +++++-- .../docs/scaladsl/InfluxDbSourceSpec.scala | 39 +++++++++++++++++-- 2 files changed, 44 insertions(+), 8 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index 932241a887..6373056b65 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -142,7 +142,7 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, if (!queryExecuted) { val queryResult = influxDB.query(query) if (!queryResult.hasError) { - queryResult.getResults.forEach(failOnError) + failOnError(queryResult) dataRetrieved = Some(queryResult) } else { failStage(new InfluxDBException(queryResult.getError)) @@ -151,9 +151,14 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, queryExecuted = true } - private def failOnError(result: QueryResult.Result) = - if (result.hasError) { - failStage(new InfluxDBException(result.getError)) + private def failOnError(result: QueryResult) = { + val totalErrors = result.getResults.asScala + .filter(_.hasError) + .map(_.getError) + if (totalErrors.size == result.getResults.size()) { + val errorMessage = totalErrors.reduceLeft((m1, m2) => m1 + ";" + m2) + failStage(new InfluxDBException(errorMessage)) } + } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index 523641d896..fca82ac6b1 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -60,12 +60,25 @@ class InfluxDbSourceSpec val result = InfluxDbSource(influxDB, query) //.runWith(Sink.seq) .recover { - case e: InfluxDBException => InfluxDbWriteResult(null, Some(e.getMessage)) + case e: InfluxDBException => e.getMessage } .runWith(Sink.seq) .futureValue - result mustBe Seq(InfluxDbWriteResult(null, Some("undefined function man()"))) + result mustBe Seq("undefined function man()") + } + + "partial error in query" in assertAllStagesStopped { + val query = new Query("SELECT*FROM cpu; SELECT man() FROM invalid", DatabaseName); + + val influxDBResult = InfluxDbSource(influxDB, query).runWith(Sink.seq) + val resultToAssert = influxDBResult.futureValue.head + + val valuesFetched = resultToAssert.getResults.get(0).getSeries().get(0).getValues + valuesFetched.size() mustBe 2 + + val error = resultToAssert.getResults.get(1).getError + error mustBe "undefined function man()" } "exception on typed source" in assertAllStagesStopped { @@ -74,12 +87,30 @@ class InfluxDbSourceSpec val result = InfluxDbSource .typed(classOf[InfluxDbSourceCpu], InfluxDbSettings.Default, influxDB, query) //.runWith(Sink.seq) .recover { - case e: InfluxDBException => InfluxDbWriteResult(null, Some(e.getMessage)) + case e: InfluxDBException => e.getMessage + } + .runWith(Sink.seq) + .futureValue + + result mustBe Seq("undefined function man()") + } + + "mixed exception on typed source" in assertAllStagesStopped { + val query = new Query("SELECT*FROM cpu;SELECT man() FROM invalid; SELECT*FROM cpu;", DatabaseName); + + val result = InfluxDbSource + .typed(classOf[InfluxDbSourceCpu], InfluxDbSettings.Default, influxDB, query) //.runWith(Sink.seq) + .recover { + case e: InfluxDBException => e.getMessage } .runWith(Sink.seq) .futureValue - result mustBe Seq(InfluxDbWriteResult(null, Some("undefined function man()"))) + val firstResult = result(0).asInstanceOf[InfluxDbSourceCpu] + firstResult.getHostname mustBe "local_1" + + val error = result(1).asInstanceOf[String] + error mustBe "not executed" } } From 7b82e2d0a95b58272597eb420f15aeacd24e9e15 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 20 May 2019 09:09:35 +0100 Subject: [PATCH 62/89] Optimized imports on InfluxDb connector --- influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index fca82ac6b1..2538c6df35 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbSettings} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit From 92f93d18ffc5884c7d638f45f42d0487e0e1e85a Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 20 May 2019 23:15:41 +0100 Subject: [PATCH 63/89] Changed the name for the result mapper on InfluxDb connector --- ...ultMapperHelper.java => AlpakkaResultMapperHelper.java} | 7 ++++++- .../stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 4 ++-- .../stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala | 6 +++--- 3 files changed, 11 insertions(+), 6 deletions(-) rename influxdb/src/main/java/org/influxdb/impl/{InfluxDbResultMapperHelper.java => AlpakkaResultMapperHelper.java} (94%) diff --git a/influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java similarity index 94% rename from influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java rename to influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java index 00a1310487..11effb42b9 100644 --- a/influxdb/src/main/java/org/influxdb/impl/InfluxDbResultMapperHelper.java +++ b/influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java @@ -16,7 +16,12 @@ import org.influxdb.dto.Point; import org.influxdb.dto.QueryResult; -public class InfluxDbResultMapperHelper { +/** + * This mapper helper is used in order to map the results to an InfluxDB measurement model for cases + * of multiple query results. Currently it needs to reside under the package org.influxdb.impl due + * to some package private methods from InfluxDBResultMapper. + */ +public class AlpakkaResultMapperHelper { private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index f2f815a1ed..c322acb1d6 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -13,7 +13,7 @@ import org.influxdb.InfluxDB import scala.collection.immutable import org.influxdb.BatchOptions import org.influxdb.dto.{BatchPoints, Point} -import org.influxdb.impl.InfluxDbResultMapperHelper +import org.influxdb.impl.AlpakkaResultMapperHelper import scala.annotation.tailrec @@ -125,7 +125,7 @@ private[influxdb] final class InfluxDbMapperRecordLogic[T, C]( shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]] ) extends InfluxDbLogic(influxDB, in, out, shape) { - private val mapperHelper: InfluxDbResultMapperHelper = new InfluxDbResultMapperHelper + private val mapperHelper: AlpakkaResultMapperHelper = new AlpakkaResultMapperHelper override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = messages diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index 6373056b65..2c7cebdca7 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -10,7 +10,7 @@ import akka.stream.{ActorAttributes, Attributes, Outlet, SourceShape} import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} import org.influxdb.{InfluxDB, InfluxDBException} import org.influxdb.dto.{Query, QueryResult} -import org.influxdb.impl.InfluxDbResultMapperHelper +import org.influxdb.impl.AlpakkaResultMapperHelper import scala.collection.JavaConverters._ @@ -52,10 +52,10 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], var queryExecuted: Boolean = false var dataRetrieved: Option[QueryResult] = None - var resultMapperHelper: InfluxDbResultMapperHelper = _ + var resultMapperHelper: AlpakkaResultMapperHelper = _ override def preStart(): Unit = { - resultMapperHelper = new InfluxDbResultMapperHelper + resultMapperHelper = new AlpakkaResultMapperHelper resultMapperHelper.cacheClassFields(clazz) } From f995ab96ba1761df07d96bc494753993f5ddc5b7 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 21 May 2019 07:55:15 +0100 Subject: [PATCH 64/89] Made no typed streams to points on InfluxDb connector --- .../influxdb/InfluxDbWriteMessage.scala | 22 +++++--- .../influxdb/impl/InfluxDbFlowStage.scala | 52 ++++++++++--------- .../influxdb/javadsl/InfluxDbFlow.scala | 13 ++--- .../influxdb/scaladsl/InfluxDbFlow.scala | 37 ++++++------- .../influxdb/scaladsl/InfluxDbSink.scala | 2 +- .../test/scala/docs/scaladsl/FlowSpec.scala | 18 +------ 6 files changed, 70 insertions(+), 74 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala index 2b557aa70b..239fd0f4ad 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala @@ -9,25 +9,33 @@ import akka.NotUsed object InfluxDbWriteMessage { // Apply method to use when not using passThrough def apply[T](point: T): InfluxDbWriteMessage[T, NotUsed] = - InfluxDbWriteMessage(point = point, passThrough = NotUsed) + new InfluxDbWriteMessage(point = point, passThrough = NotUsed) // Java-api - without passThrough def create[T](point: T): InfluxDbWriteMessage[T, NotUsed] = - InfluxDbWriteMessage(point, NotUsed) + new InfluxDbWriteMessage(point, NotUsed) // Java-api - with passThrough def create[T, C](point: T, passThrough: C) = - InfluxDbWriteMessage(point, passThrough) + new InfluxDbWriteMessage(point, passThrough) } -final case class InfluxDbWriteMessage[T, C](point: T, - passThrough: C, - databaseName: Option[String] = None, - retentionPolicy: Option[String] = None) { +final class InfluxDbWriteMessage[T, C](val point: T, + val passThrough: C, + val databaseName: Option[String] = None, + val retentionPolicy: Option[String] = None) { def withPoint(point: T): InfluxDbWriteMessage[T, C] = copy(point = point) + def withPassThrough[PT2](passThrough: PT2): InfluxDbWriteMessage[T, PT2] = + new InfluxDbWriteMessage[T, PT2]( + point = point, + passThrough = passThrough, + databaseName = databaseName, + retentionPolicy = retentionPolicy + ) + def withDatabaseName(databaseName: String): InfluxDbWriteMessage[T, C] = copy(databaseName = Some(databaseName)) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index c322acb1d6..a8c0cd36b9 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -8,10 +8,9 @@ import akka.annotation.InternalApi import akka.stream._ import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, OutHandler} -import org.influxdb.InfluxDB +import org.influxdb.{BatchOptions, InfluxDB} import scala.collection.immutable -import org.influxdb.BatchOptions import org.influxdb.dto.{BatchPoints, Point} import org.influxdb.impl.AlpakkaResultMapperHelper @@ -21,12 +20,13 @@ import scala.annotation.tailrec * INTERNAL API */ @InternalApi -private[influxdb] class InfluxDbFlowStage[T, C]( - clazz: Option[Class[T]], +private[influxdb] class InfluxDbFlowStage[C]( influxDB: InfluxDB -) extends GraphStage[FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]]] { - private val in = Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]]("in") - private val out = Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]]("out") +) extends GraphStage[ + FlowShape[immutable.Seq[InfluxDbWriteMessage[Point, C]], immutable.Seq[InfluxDbWriteResult[Point, C]]] + ] { + private val in = Inlet[immutable.Seq[InfluxDbWriteMessage[Point, C]]]("in") + private val out = Outlet[immutable.Seq[InfluxDbWriteResult[Point, C]]]("out") override val shape = FlowShape(in, out) @@ -34,10 +34,21 @@ private[influxdb] class InfluxDbFlowStage[T, C]( super.initialAttributes and Attributes(ActorAttributes.IODispatcher) override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = - clazz match { - case Some(c) => new InfluxDbMapperRecordLogic(influxDB, in, out, shape) - case None => new InfluxDbRecordLogic(influxDB, in, out, shape) - } + new InfluxDbRecordLogic(influxDB, in, out, shape) + +} +private[influxdb] class InfluxDbMapperFlowStage[T, C]( + clazz: Class[T], + influxDB: InfluxDB +) extends GraphStage[FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]]] { + + private val in = Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]]("in") + private val out = Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]]("out") + + override val shape = FlowShape(in, out) + + override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = + new InfluxDbMapperRecordLogic(influxDB, in, out, shape) } @@ -95,24 +106,15 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( } -private[influxdb] final class InfluxDbRecordLogic[T, C]( +private[influxdb] final class InfluxDbRecordLogic[C]( influxDB: InfluxDB, - in: Inlet[immutable.Seq[InfluxDbWriteMessage[T, C]]], - out: Outlet[immutable.Seq[InfluxDbWriteResult[T, C]]], - shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]]] + in: Inlet[immutable.Seq[InfluxDbWriteMessage[Point, C]]], + out: Outlet[immutable.Seq[InfluxDbWriteResult[Point, C]]], + shape: FlowShape[immutable.Seq[InfluxDbWriteMessage[Point, C]], immutable.Seq[InfluxDbWriteResult[Point, C]]] ) extends InfluxDbLogic(influxDB, in, out, shape) { - override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[T, C]]): Unit = + override protected def write(messages: immutable.Seq[InfluxDbWriteMessage[Point, C]]): Unit = messages - .filter { - case InfluxDbWriteMessage(_: Point, _, _, _) => { - true - } - case InfluxDbWriteMessage(_: AnyRef, _, _, _) => { - failStage(new RuntimeException(s"unexpected type Point required")) - false - } - } .groupBy(im => (im.databaseName, im.retentionPolicy)) .map(wm => toBatchPoints(wm._1._1, wm._1._2, wm._2)) .foreach(influxDB.write) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala index 61e69097df..79103ae50a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala @@ -9,13 +9,14 @@ import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, Inf import org.influxdb.InfluxDB import akka.stream.javadsl.Flow import akka.stream.alpakka.influxdb.scaladsl +import org.influxdb.dto.Point object InfluxDbFlow { - def create[T]( + def create( settings: InfluxDbSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = + ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = scaladsl.InfluxDbFlow.create(settings)(influxDB).asJava def typed[T]( @@ -25,10 +26,10 @@ object InfluxDbFlow { ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = scaladsl.InfluxDbFlow.typed(clazz, settings)(influxDB).asJava - def createWithPassThrough[T, C]( + def createWithPassThrough[C]( settings: InfluxDbSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = + ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = scaladsl.InfluxDbFlow.createWithPassThrough(settings)(influxDB).asJava def typedWithPassThrough[T, C]( @@ -38,10 +39,10 @@ object InfluxDbFlow { ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = scaladsl.InfluxDbFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava - def createWithContext[T, C]( + def createWithContext[C]( settings: InfluxDbSettings, influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = + ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = scaladsl.InfluxDbFlow.createWithContext(settings)(influxDB).asJava def typedWithContext[T, C]( diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala index ccfa261fa9..2d5d06fded 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -8,6 +8,7 @@ import akka.NotUsed import akka.stream.alpakka.influxdb.{impl, InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB +import org.influxdb.dto.Point import scala.collection.immutable @@ -16,12 +17,12 @@ import scala.collection.immutable */ object InfluxDbFlow { - def create[T](settings: InfluxDbSettings)( + def create(settings: InfluxDbSettings)( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = - Flow[InfluxDbWriteMessage[T, NotUsed]] + ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = + Flow[InfluxDbWriteMessage[Point, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[T, NotUsed](None, influxDB)) + .via(new impl.InfluxDbFlowStage[NotUsed](influxDB)) .mapConcat(identity) def typed[T](clazz: Class[T], settings: InfluxDbSettings)( @@ -29,15 +30,15 @@ object InfluxDbFlow { ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDbWriteMessage[T, NotUsed]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[T, NotUsed](Some(clazz), influxDB)) + .via(new impl.InfluxDbMapperFlowStage[T, NotUsed](clazz, influxDB)) .mapConcat(identity) - def createWithPassThrough[T, C](settings: InfluxDbSettings)( + def createWithPassThrough[C](settings: InfluxDbSettings)( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = - Flow[InfluxDbWriteMessage[T, C]] + ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = + Flow[InfluxDbWriteMessage[Point, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[T, C](None, influxDB)) + .via(new impl.InfluxDbFlowStage[C](influxDB)) .mapConcat(identity) def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDbSettings)( @@ -45,18 +46,19 @@ object InfluxDbFlow { ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = Flow[InfluxDbWriteMessage[T, C]] .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[T, C](Some(clazz), influxDB)) + .via(new impl.InfluxDbMapperFlowStage[T, C](clazz, influxDB)) .mapConcat(identity) - def createWithContext[T, C](settings: InfluxDbSettings)( + def createWithContext[C](settings: InfluxDbSettings)( implicit influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = - Flow[(InfluxDbWriteMessage[T, NotUsed], C)] + ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = + Flow[(InfluxDbWriteMessage[Point, NotUsed], C)] .map { - case (wm, pt) => - InfluxDbWriteMessage(wm.point, pt) + case (wm, pt) => wm.withPassThrough(pt) } - .via(createWithPassThrough(settings)) + .via( + createWithPassThrough(settings) + ) .map { wr => (wr, wr.writeMessage.passThrough) } @@ -66,8 +68,7 @@ object InfluxDbFlow { ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = Flow[(InfluxDbWriteMessage[T, NotUsed], C)] .map { - case (wm, pt) => - InfluxDbWriteMessage(wm.point, pt) + case (wm, pt) => wm.withPassThrough(pt) } .via(typedWithPassThrough(clazz, settings)) .map { wr => diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index f1664e5ce6..4f809caf1a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -17,7 +17,7 @@ object InfluxDbSink { def apply( settings: InfluxDbSettings )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[Point, NotUsed], Future[Done]] = - InfluxDbFlow.create[Point](settings).toMat(Sink.ignore)(Keep.right) + InfluxDbFlow.create(settings).toMat(Sink.ignore)(Keep.right) def typed[T]( clazz: Class[T], diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index a6ce0871d6..3af504fe3c 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -19,8 +19,6 @@ import docs.javadsl.TestUtils._ import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import org.influxdb.dto.Point -private final case class InvalidModel(description: String) {} - class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { implicit val system = ActorSystem() @@ -38,20 +36,6 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B TestKit.shutdownActorSystem(system) } - "invalid model" in assertAllStagesStopped { - val result = Source( - List(InfluxDbWriteMessage(InvalidModel("Invalid measurement one")), - InfluxDbWriteMessage(InvalidModel("Invalid measurement two"))) - ).via(InfluxDbFlow.create[InvalidModel](InfluxDbSettings())) - .recover { - case _: RuntimeException => InfluxDbWriteResult(null, Some("error occurred")) - } - .runWith(Sink.seq) - .futureValue - - result mustBe Seq(InfluxDbWriteResult(null, Some("error occurred"))) - } - "mixed model" in assertAllStagesStopped { val point = Point @@ -68,7 +52,7 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B List( validMessage ) - ).via(InfluxDbFlow.create[Point](InfluxDbSettings())) + ).via(InfluxDbFlow.create(InfluxDbSettings())) .runWith(Sink.seq) .futureValue From fe299a1366195c65c9c5b1cbd2d970c23a72d788 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 21 May 2019 08:08:31 +0100 Subject: [PATCH 65/89] Made more functional retention policy check on InfluxDb connector --- .../akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index a8c0cd36b9..d051d5352f 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -89,7 +89,7 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( case None => BatchPoints.builder() } - if (retentionPolicy.isDefined) builder.retentionPolicy(retentionPolicy.get) + retentionPolicy.foreach(builder.retentionPolicy) @tailrec def convert(messages: Seq[InfluxDbWriteMessage[T, C]]): BatchPoints = From 8ee2a6d08dc3a5c6cfae07d317d2e46dbfab96c2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 21 May 2019 08:10:53 +0100 Subject: [PATCH 66/89] Added @InternalApi on InfluxDb connector --- .../akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index d051d5352f..c4c5f1fbe9 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -37,6 +37,8 @@ private[influxdb] class InfluxDbFlowStage[C]( new InfluxDbRecordLogic(influxDB, in, out, shape) } + +@InternalApi private[influxdb] class InfluxDbMapperFlowStage[T, C]( clazz: Class[T], influxDB: InfluxDB From 7d7ad27b88810bb35dc61df5d22f808afdff2144 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Tue, 21 May 2019 08:50:27 +0100 Subject: [PATCH 67/89] Added read and write settings on InfluxDb connector --- .../influxdb/InfluxDbReadSettings.scala | 26 +++++++++++++++ .../alpakka/influxdb/InfluxDbSettings.scala | 33 ------------------- .../influxdb/InfluxDbWriteSettings.scala | 26 +++++++++++++++ .../influxdb/impl/InfluxDbSourceStage.scala | 6 ++-- .../influxdb/javadsl/InfluxDbFlow.scala | 14 ++++---- .../influxdb/javadsl/InfluxDbSink.scala | 6 ++-- .../influxdb/javadsl/InfluxDbSource.scala | 4 +-- .../influxdb/scaladsl/InfluxDbFlow.scala | 14 ++++---- .../influxdb/scaladsl/InfluxDbSink.scala | 6 ++-- .../influxdb/scaladsl/InfluxDbSource.scala | 4 +-- .../java/docs/javadsl/InfluxDbSourceTest.java | 6 ++-- .../test/java/docs/javadsl/InfluxDbTest.java | 11 ++++--- .../test/scala/docs/scaladsl/FlowSpec.scala | 4 +-- .../docs/scaladsl/InfluxDbSourceSpec.scala | 6 ++-- .../scala/docs/scaladsl/InfluxDbSpec.scala | 16 +++++---- 15 files changed, 103 insertions(+), 79 deletions(-) create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala new file mode 100644 index 0000000000..b60a802e79 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala @@ -0,0 +1,26 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package akka.stream.alpakka.influxdb + +import java.util.concurrent.TimeUnit + +object InfluxDbReadSettings { + val Default = new InfluxDbReadSettings(TimeUnit.MILLISECONDS) + + def apply(): InfluxDbReadSettings = Default + +} + +final class InfluxDbReadSettings private (val precision: TimeUnit) { + + def withPrecision(value: TimeUnit): InfluxDbReadSettings = copy(precision = value) + + private def copy( + precision: TimeUnit = precision + ): InfluxDbReadSettings = new InfluxDbReadSettings( + precision = precision + ) + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala deleted file mode 100644 index 03de317f34..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbSettings.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (C) 2016-2019 Lightbend Inc. - */ - -package akka.stream.alpakka.influxdb - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.TimeUnit - -object InfluxDbSettings { - val Default = new InfluxDbSettings(batchSize = 10, TimeUnit.MILLISECONDS) - - def apply(): InfluxDbSettings = Default - -} - -final class InfluxDbSettings private ( - val batchSize: Int, - val precision: TimeUnit -) { - - def withBatchSize(value: Int): InfluxDbSettings = copy(batchSize = value) - - private def copy( - batchSize: Int = batchSize, - precision: TimeUnit = precision - ): InfluxDbSettings = new InfluxDbSettings( - batchSize = batchSize, - precision = precision - ) - -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala new file mode 100644 index 0000000000..63d5644c00 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala @@ -0,0 +1,26 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package akka.stream.alpakka.influxdb + +object InfluxDbWriteSettings { + val Default = new InfluxDbWriteSettings(batchSize = 10) + + def apply(): InfluxDbWriteSettings = Default + +} + +final class InfluxDbWriteSettings private ( + val batchSize: Int, +) { + + def withBatchSize(value: Int): InfluxDbWriteSettings = copy(batchSize = value) + + private def copy( + batchSize: Int = batchSize, + ): InfluxDbWriteSettings = new InfluxDbWriteSettings( + batchSize = batchSize + ) + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index 2c7cebdca7..fec09d1479 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.impl import akka.annotation.InternalApi -import akka.stream.alpakka.influxdb.InfluxDbSettings +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} import akka.stream.{ActorAttributes, Attributes, Outlet, SourceShape} import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} import org.influxdb.{InfluxDB, InfluxDBException} @@ -19,7 +19,7 @@ import scala.collection.JavaConverters._ */ @InternalApi private[influxdb] final class InfluxDbSourceStage[T](clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbReadSettings, influxDB: InfluxDB, query: Query) extends GraphStage[SourceShape[T]] { @@ -40,7 +40,7 @@ private[influxdb] final class InfluxDbSourceStage[T](clazz: Class[T], */ @InternalApi private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbReadSettings, influxDB: InfluxDB, query: Query, outlet: Outlet[T], diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala index 79103ae50a..4b4527ba1a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} import org.influxdb.InfluxDB import akka.stream.javadsl.Flow import akka.stream.alpakka.influxdb.scaladsl @@ -14,40 +14,40 @@ import org.influxdb.dto.Point object InfluxDbFlow { def create( - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = scaladsl.InfluxDbFlow.create(settings)(influxDB).asJava def typed[T]( clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = scaladsl.InfluxDbFlow.typed(clazz, settings)(influxDB).asJava def createWithPassThrough[C]( - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = scaladsl.InfluxDbFlow.createWithPassThrough(settings)(influxDB).asJava def typedWithPassThrough[T, C]( clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = scaladsl.InfluxDbFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava def createWithContext[C]( - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = scaladsl.InfluxDbFlow.createWithContext(settings)(influxDB).asJava def typedWithContext[T, C]( clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = scaladsl.InfluxDbFlow.typedWithContext(clazz, settings)(influxDB).asJava diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala index a564013010..f8d7e11af1 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala @@ -7,7 +7,7 @@ package akka.stream.alpakka.influxdb.javadsl import java.util.concurrent.CompletionStage import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} import akka.stream.javadsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -18,7 +18,7 @@ import org.influxdb.dto.Point object InfluxDbSink { def create( - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB ): akka.stream.javadsl.Sink[InfluxDbWriteMessage[Point, NotUsed], CompletionStage[Done]] = InfluxDbFlow @@ -26,7 +26,7 @@ object InfluxDbSink { .toMat(Sink.ignore[InfluxDbWriteResult[Point, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) def typed[T](clazz: Class[T], - settings: InfluxDbSettings, + settings: InfluxDbWriteSettings, influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDbWriteMessage[T, NotUsed], CompletionStage[Done]] = InfluxDbFlow .typed(clazz, settings, influxDB) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala index c549b814d7..9816f3ae3f 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.InfluxDbSettings +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteSettings} import akka.stream.javadsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} @@ -25,7 +25,7 @@ object InfluxDbSource { /** * Java API: creates an [[InfluxDbSourceStage]] of elements of `T` from `query`. */ - def typed[T](clazz: Class[T], settings: InfluxDbSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDbReadSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( new InfluxDbSourceStage[T]( clazz, diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala index 2d5d06fded..c47dfebbca 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{impl, InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.{impl, InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -17,7 +17,7 @@ import scala.collection.immutable */ object InfluxDbFlow { - def create(settings: InfluxDbSettings)( + def create(settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = Flow[InfluxDbWriteMessage[Point, NotUsed]] @@ -25,7 +25,7 @@ object InfluxDbFlow { .via(new impl.InfluxDbFlowStage[NotUsed](influxDB)) .mapConcat(identity) - def typed[T](clazz: Class[T], settings: InfluxDbSettings)( + def typed[T](clazz: Class[T], settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = Flow[InfluxDbWriteMessage[T, NotUsed]] @@ -33,7 +33,7 @@ object InfluxDbFlow { .via(new impl.InfluxDbMapperFlowStage[T, NotUsed](clazz, influxDB)) .mapConcat(identity) - def createWithPassThrough[C](settings: InfluxDbSettings)( + def createWithPassThrough[C](settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = Flow[InfluxDbWriteMessage[Point, C]] @@ -41,7 +41,7 @@ object InfluxDbFlow { .via(new impl.InfluxDbFlowStage[C](influxDB)) .mapConcat(identity) - def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDbSettings)( + def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = Flow[InfluxDbWriteMessage[T, C]] @@ -49,7 +49,7 @@ object InfluxDbFlow { .via(new impl.InfluxDbMapperFlowStage[T, C](clazz, influxDB)) .mapConcat(identity) - def createWithContext[C](settings: InfluxDbSettings)( + def createWithContext[C](settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = Flow[(InfluxDbWriteMessage[Point, NotUsed], C)] @@ -63,7 +63,7 @@ object InfluxDbFlow { (wr, wr.writeMessage.passThrough) } - def typedWithContext[T, C](clazz: Class[T], settings: InfluxDbSettings)( + def typedWithContext[T, C](clazz: Class[T], settings: InfluxDbWriteSettings)( implicit influxDB: InfluxDB ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = Flow[(InfluxDbWriteMessage[T, NotUsed], C)] diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index 4f809caf1a..1ddc54da2a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteSettings} import akka.stream.scaladsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -15,13 +15,13 @@ import scala.concurrent.Future object InfluxDbSink { def apply( - settings: InfluxDbSettings + settings: InfluxDbWriteSettings )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[Point, NotUsed], Future[Done]] = InfluxDbFlow.create(settings).toMat(Sink.ignore)(Keep.right) def typed[T]( clazz: Class[T], - settings: InfluxDbSettings + settings: InfluxDbWriteSettings )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[T, NotUsed], Future[Done]] = InfluxDbFlow .typed(clazz, settings) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala index 607e846f88..bb13fac9d8 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.InfluxDbSettings +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} import akka.stream.alpakka.influxdb.impl.{InfluxDbRawSourceStage, InfluxDbSourceStage} import akka.stream.scaladsl.Source import org.influxdb.InfluxDB @@ -25,7 +25,7 @@ object InfluxDbSource { /** * Read elements of `T` from `className` or by `query`. */ - def typed[T](clazz: Class[T], settings: InfluxDbSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = + def typed[T](clazz: Class[T], settings: InfluxDbReadSettings, influxDB: InfluxDB, query: Query): Source[T, NotUsed] = Source.fromGraph( new InfluxDbSourceStage[T]( clazz, diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java index 56132a1d5c..8fed19ee65 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java @@ -21,7 +21,8 @@ import akka.japi.Pair; import akka.stream.ActorMaterializer; import akka.stream.Materializer; -import akka.stream.alpakka.influxdb.InfluxDbSettings; +import akka.stream.alpakka.influxdb.InfluxDbReadSettings; +import akka.stream.alpakka.influxdb.InfluxDbWriteSettings; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; @@ -78,7 +79,8 @@ public void streamQueryResult() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage> rows = - InfluxDbSource.typed(InfluxDbSourceCpu.class, InfluxDbSettings.Default(), influxDB, query) + InfluxDbSource.typed( + InfluxDbSourceCpu.class, InfluxDbReadSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); List cpus = rows.toCompletableFuture().get(); diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index b69c7bf827..93b2cbf784 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -25,7 +25,8 @@ import akka.japi.Pair; import akka.stream.ActorMaterializer; import akka.stream.Materializer; -import akka.stream.alpakka.influxdb.InfluxDbSettings; +import akka.stream.alpakka.influxdb.InfluxDbReadSettings; +import akka.stream.alpakka.influxdb.InfluxDbWriteSettings; import akka.stream.alpakka.influxdb.InfluxDbWriteMessage; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSink; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; @@ -84,20 +85,20 @@ public void cleanUp() { public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); CompletionStage completionStage = - InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB, query) + InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbReadSettings.Default(), influxDB, query) .map( cpu -> { InfluxDbCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); return InfluxDbWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) .runWith( - InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbSettings.Default(), influxDB), + InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbWriteSettings.Default(), influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); CompletionStage> sources = - InfluxDbSource.typed(Cpu.class, InfluxDbSettings.Default(), influxDB, query) + InfluxDbSource.typed(Cpu.class, InfluxDbReadSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); Assert.assertEquals(4, sources.toCompletableFuture().get().size()); @@ -111,7 +112,7 @@ public void testConsumeAndPublishMeasurements() throws Exception { InfluxDbSource.create(influxDB, query) .map(queryResult -> points(queryResult)) .mapConcat(i -> i) - .runWith(InfluxDbSink.create(InfluxDbSettings.Default(), influxDB), materializer); + .runWith(InfluxDbSink.create(InfluxDbWriteSettings.Default(), influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index 3af504fe3c..510f313434 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.TimeUnit import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbFlow import akka.stream.scaladsl.{Sink, Source} import akka.testkit.TestKit @@ -52,7 +52,7 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B List( validMessage ) - ).via(InfluxDbFlow.create(InfluxDbSettings())) + ).via(InfluxDbFlow.create(InfluxDbWriteSettings())) .runWith(Sink.seq) .futureValue diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index 2538c6df35..5cfdaeb1ac 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbSettings} +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteSettings} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit @@ -85,7 +85,7 @@ class InfluxDbSourceSpec val query = new Query("SELECT man() FROM invalid", DatabaseName); val result = InfluxDbSource - .typed(classOf[InfluxDbSourceCpu], InfluxDbSettings.Default, influxDB, query) //.runWith(Sink.seq) + .typed(classOf[InfluxDbSourceCpu], InfluxDbReadSettings.Default, influxDB, query) //.runWith(Sink.seq) .recover { case e: InfluxDBException => e.getMessage } @@ -99,7 +99,7 @@ class InfluxDbSourceSpec val query = new Query("SELECT*FROM cpu;SELECT man() FROM invalid; SELECT*FROM cpu;", DatabaseName); val result = InfluxDbSource - .typed(classOf[InfluxDbSourceCpu], InfluxDbSettings.Default, influxDB, query) //.runWith(Sink.seq) + .typed(classOf[InfluxDbSourceCpu], InfluxDbReadSettings.Default, influxDB, query) //.runWith(Sink.seq) .recover { case e: InfluxDBException => e.getMessage } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 5b0b7aadf3..6185de9d6c 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordS import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbSettings, InfluxDbWriteMessage} +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteMessage, InfluxDbWriteSettings} import akka.stream.alpakka.influxdb.scaladsl.{InfluxDbSink, InfluxDbSource} import akka.testkit.TestKit import docs.javadsl.TestUtils._ @@ -44,7 +44,7 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "support typed source" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); val measurements = - InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) + InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query).runWith(Sink.seq) measurements.futureValue.map(_.getHostname) mustBe List("local_1", "local_2") } @@ -55,18 +55,19 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi val query = new Query("SELECT * FROM cpu", DatabaseName); val f1 = InfluxDbSource - .typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query) + .typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query) .map { cpu: InfluxDbSpecCpu => { val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) InfluxDbWriteMessage(clonedCpu) } } - .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings())) + .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu], InfluxDbWriteSettings())) f1.futureValue mustBe Done - val f2 = InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) + val f2 = + InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } @@ -76,11 +77,12 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi val f1 = InfluxDbSource(influxDB, query) .mapConcat(resultToPoints) - .runWith(InfluxDbSink(InfluxDbSettings())) + .runWith(InfluxDbSink(InfluxDbWriteSettings())) f1.futureValue mustBe Done - val f2 = InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbSettings(), influxDB, query).runWith(Sink.seq) + val f2 = + InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query).runWith(Sink.seq) f2.futureValue.length mustBe 4 } From 018b2ab09eb726fc85df65697ff7130083c838e5 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 23 May 2019 08:52:24 +0100 Subject: [PATCH 68/89] excluding 2.11 scala version for influxdb connector --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index a3feed21f4..b4e0c99aaf 100644 --- a/build.sbt +++ b/build.sbt @@ -184,7 +184,10 @@ lazy val hbase = alpakkaProject("hbase", "hbase", Dependencies.HBase, fork in Te lazy val hdfs = alpakkaProject("hdfs", "hdfs", Dependencies.Hdfs, parallelExecution in Test := false) -lazy val influxdb = alpakkaProject("influxdb", "influxdb", Dependencies.InfluxDB).disablePlugins(MimaPlugin) +lazy val influxdb = alpakkaProject("influxdb", + "influxdb", + Dependencies.InfluxDB, + crossScalaVersions -= Dependencies.Scala211).disablePlugins(MimaPlugin) lazy val ironmq = alpakkaProject("ironmq", "ironmq", From 7c5965355ddf080e29938631f3f7239fd7dad23f Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 23 May 2019 08:55:44 +0100 Subject: [PATCH 69/89] Fixing copy paste error for InfluxDb connector --- .../akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala index bb13fac9d8..add343c965 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala @@ -17,7 +17,7 @@ import org.influxdb.dto.{Query, QueryResult} object InfluxDbSource { /** - * Java API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDbRawSourceStage]] from a given statement. + * Scala API: creates an [[akka.stream.alpakka.influxdb.impl.InfluxDbRawSourceStage]] from a given statement. */ def apply(influxDB: InfluxDB, query: Query): Source[QueryResult, NotUsed] = Source.fromGraph(new InfluxDbRawSourceStage(query, influxDB)) From dab160e4093951bd07ac47472484a2fa366f6ffc Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 23 May 2019 10:06:52 +0100 Subject: [PATCH 70/89] Added toString on InfluxDBWriteMessage for InfluxDb connector --- .../akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala index 239fd0f4ad..f69a4c22af 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala @@ -52,6 +52,9 @@ final class InfluxDbWriteMessage[T, C](val point: T, passThrough = passThrough, databaseName = databaseName, retentionPolicy = retentionPolicy) + + override def toString: String = + s"""InfluxDbWriteMessage(point=$point,passThrough=$passThrough,databaseName=$databaseName,retentionPolicy=$retentionPolicy)""" } final case class InfluxDbWriteResult[T, C](writeMessage: InfluxDbWriteMessage[T, C], error: Option[String]) From 659e1a39dbd78c328c7ecb95f2252cec06fd7b17 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 23 May 2019 12:39:45 +0100 Subject: [PATCH 71/89] Added toString methods on InfluxDb connector --- .../akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala | 3 +++ .../akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala | 2 +- .../akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala | 3 +++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala index b60a802e79..0746a64ec4 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala @@ -23,4 +23,7 @@ final class InfluxDbReadSettings private (val precision: TimeUnit) { precision = precision ) + override def toString: String = + s"""InfluxDbReadSettings(precision=$precision)""" + } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala index f69a4c22af..34fe01c7b5 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala @@ -20,7 +20,7 @@ object InfluxDbWriteMessage { new InfluxDbWriteMessage(point, passThrough) } -final class InfluxDbWriteMessage[T, C](val point: T, +final class InfluxDbWriteMessage[T, C] private (val point: T, val passThrough: C, val databaseName: Option[String] = None, val retentionPolicy: Option[String] = None) { diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala index 63d5644c00..e4cdee38c3 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala @@ -23,4 +23,7 @@ final class InfluxDbWriteSettings private ( batchSize = batchSize ) + override def toString: String = + s"""InfluxDbWriteSettings(batchSize=$batchSize)""" + } From 54bc4d7f9922a72dc6473839f5bae1d226e62a44 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 24 May 2019 22:29:47 +0100 Subject: [PATCH 72/89] Refactored from apply to create on InfluDb connector Changed to precision on InfluDb connector --- .../akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala | 2 +- .../akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala | 6 +++--- .../stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala | 2 +- influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala index 0746a64ec4..89b0a1bb54 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbReadSettings.scala @@ -15,7 +15,7 @@ object InfluxDbReadSettings { final class InfluxDbReadSettings private (val precision: TimeUnit) { - def withPrecision(value: TimeUnit): InfluxDbReadSettings = copy(precision = value) + def withPrecision(precision: TimeUnit): InfluxDbReadSettings = copy(precision = precision) private def copy( precision: TimeUnit = precision diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala index 34fe01c7b5..c7d4f801ba 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteMessage.scala @@ -21,9 +21,9 @@ object InfluxDbWriteMessage { } final class InfluxDbWriteMessage[T, C] private (val point: T, - val passThrough: C, - val databaseName: Option[String] = None, - val retentionPolicy: Option[String] = None) { + val passThrough: C, + val databaseName: Option[String] = None, + val retentionPolicy: Option[String] = None) { def withPoint(point: T): InfluxDbWriteMessage[T, C] = copy(point = point) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index 1ddc54da2a..84ec5b4868 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -14,7 +14,7 @@ import scala.concurrent.Future object InfluxDbSink { - def apply( + def create( settings: InfluxDbWriteSettings )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[Point, NotUsed], Future[Done]] = InfluxDbFlow.create(settings).toMat(Sink.ignore)(Keep.right) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 6185de9d6c..66f6b29790 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -77,7 +77,7 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi val f1 = InfluxDbSource(influxDB, query) .mapConcat(resultToPoints) - .runWith(InfluxDbSink(InfluxDbWriteSettings())) + .runWith(InfluxDbSink.create(InfluxDbWriteSettings())) f1.futureValue mustBe Done From 19af35bdd956a587ed8c7cf69dff9620cb1e9607 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 24 May 2019 22:44:11 +0100 Subject: [PATCH 73/89] Formatted sbt for InfluxDb connector --- build.sbt | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/build.sbt b/build.sbt index b4e0c99aaf..09db89fbf4 100644 --- a/build.sbt +++ b/build.sbt @@ -184,10 +184,9 @@ lazy val hbase = alpakkaProject("hbase", "hbase", Dependencies.HBase, fork in Te lazy val hdfs = alpakkaProject("hdfs", "hdfs", Dependencies.Hdfs, parallelExecution in Test := false) -lazy val influxdb = alpakkaProject("influxdb", - "influxdb", - Dependencies.InfluxDB, - crossScalaVersions -= Dependencies.Scala211).disablePlugins(MimaPlugin) +lazy val influxdb = + alpakkaProject("influxdb", "influxdb", Dependencies.InfluxDB, crossScalaVersions -= Dependencies.Scala211) + .disablePlugins(MimaPlugin) lazy val ironmq = alpakkaProject("ironmq", "ironmq", From f65689a7fb7156094c25d46e4f7a946303087afd Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Mon, 27 May 2019 10:28:04 +0100 Subject: [PATCH 74/89] Removed logs test on InfluxDb connector --- influxdb/src/test/java/docs/javadsl/TestUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index 38de4db8d0..e8b9642426 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -25,7 +25,6 @@ public class TestUtils { public static InfluxDB setupConnection(final String databaseName) { final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); influxDB.setDatabase(databaseName); - influxDB.setLogLevel(InfluxDB.LogLevel.FULL); influxDB.query(new Query("CREATE DATABASE " + databaseName, databaseName)); return influxDB; } From 14e692da8eb495908be0b13f8e4515ab5b719553 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Sun, 2 Jun 2019 22:20:34 +0100 Subject: [PATCH 75/89] Changed to databaseName.orNull on InfluxDb connector --- .../stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index c4c5f1fbe9..85a488cf2f 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -86,10 +86,8 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( protected def toBatchPoints(databaseName: Option[String], retentionPolicy: Option[String], seq: Seq[InfluxDbWriteMessage[T, C]]) = { - val builder = databaseName match { - case Some(databaseName) => BatchPoints.database(databaseName) - case None => BatchPoints.builder() - } + + val builder = BatchPoints.database(databaseName.orNull) retentionPolicy.foreach(builder.retentionPolicy) From 32aa72c1859e311284450ae67074e0f7a76be2aa Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 13 Jun 2019 10:42:12 +0100 Subject: [PATCH 76/89] Removed batch, in order for the user to submit immutable.Seq on InfluxDB Connector Removed InfluxDBWrite Settings due to not using batch size on InfluxDB Connector --- .../impl/AlpakkaResultMapperHelper.java | 156 ----------- .../influxdb/InfluxDbWriteSettings.scala | 29 -- .../impl/AlpakkaResultMapperHelper.scala | 260 ++++++++++++++++++ .../influxdb/impl/InfluxDbFlowStage.scala | 5 +- .../influxdb/impl/InfluxDbSourceStage.scala | 4 +- .../influxdb/javadsl/InfluxDbFlow.scala | 58 ++-- .../influxdb/javadsl/InfluxDbSink.scala | 21 +- .../influxdb/javadsl/InfluxDbSource.scala | 2 +- .../influxdb/scaladsl/InfluxDbFlow.scala | 66 +---- .../influxdb/scaladsl/InfluxDbSink.scala | 16 +- .../java/docs/javadsl/InfluxDbSourceTest.java | 1 - .../test/java/docs/javadsl/InfluxDbTest.java | 12 +- .../test/scala/docs/scaladsl/FlowSpec.scala | 9 +- .../docs/scaladsl/InfluxDbSourceSpec.scala | 2 +- .../scala/docs/scaladsl/InfluxDbSpec.scala | 12 +- 15 files changed, 349 insertions(+), 304 deletions(-) delete mode 100644 influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java delete mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala create mode 100644 influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/AlpakkaResultMapperHelper.scala diff --git a/influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java b/influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java deleted file mode 100644 index 11effb42b9..0000000000 --- a/influxdb/src/main/java/org/influxdb/impl/AlpakkaResultMapperHelper.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Copyright (C) 2016-2019 Lightbend Inc. - */ - -package org.influxdb.impl; - -import java.lang.reflect.Field; -import java.time.Instant; -import java.util.List; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import org.influxdb.InfluxDBMapperException; -import org.influxdb.annotation.Column; -import org.influxdb.dto.Point; -import org.influxdb.dto.QueryResult; - -/** - * This mapper helper is used in order to map the results to an InfluxDB measurement model for cases - * of multiple query results. Currently it needs to reside under the package org.influxdb.impl due - * to some package private methods from InfluxDBResultMapper. - */ -public class AlpakkaResultMapperHelper { - - private final InfluxDBResultMapper influxDBResultMapper = new InfluxDBResultMapper(); - - public void cacheClassFields(final Class clazz) { - influxDBResultMapper.cacheMeasurementClass(clazz); - } - - public String databaseName(T model) { - Class clazz = model.getClass(); - return influxDBResultMapper.getDatabaseName(clazz); - } - - public String retentionPolicy(T model) { - Class clazz = model.getClass(); - return influxDBResultMapper.getRetentionPolicy(clazz); - } - - public Point convertModelToPoint(T model) { - influxDBResultMapper.throwExceptionIfMissingAnnotation(model.getClass()); - influxDBResultMapper.cacheMeasurementClass(model.getClass()); - - ConcurrentMap colNameAndFieldMap = - influxDBResultMapper.getColNameAndFieldMap(model.getClass()); - - try { - Class modelType = model.getClass(); - String measurement = influxDBResultMapper.getMeasurementName(modelType); - TimeUnit timeUnit = influxDBResultMapper.getTimeUnit(modelType); - long time = timeUnit.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS); - Point.Builder pointBuilder = Point.measurement(measurement).time(time, timeUnit); - - for (String key : colNameAndFieldMap.keySet()) { - Field field = colNameAndFieldMap.get(key); - Column column = field.getAnnotation(Column.class); - String columnName = column.name(); - Class fieldType = field.getType(); - - if (!field.isAccessible()) { - field.setAccessible(true); - } - - Object value = field.get(model); - - if (column.tag()) { - /** Tags are strings either way. */ - pointBuilder.tag(columnName, value.toString()); - } else if ("time".equals(columnName)) { - if (value != null) { - setTime(pointBuilder, fieldType, timeUnit, value); - } - } else { - setField(pointBuilder, fieldType, columnName, value); - } - } - - return pointBuilder.build(); - } catch (IllegalAccessException e) { - throw new InfluxDBMapperException(e); - } - } - - private void setTime( - final Point.Builder pointBuilder, - final Class fieldType, - final TimeUnit timeUnit, - final Object value) { - if (Instant.class.isAssignableFrom(fieldType)) { - Instant instant = (Instant) value; - long time = timeUnit.convert(instant.toEpochMilli(), TimeUnit.MILLISECONDS); - pointBuilder.time(time, timeUnit); - } else { - throw new InfluxDBMapperException( - "Unsupported type " + fieldType + " for time: should be of Instant type"); - } - } - - private void setField( - final Point.Builder pointBuilder, - final Class fieldType, - final String columnName, - final Object value) { - if (boolean.class.isAssignableFrom(fieldType) || Boolean.class.isAssignableFrom(fieldType)) { - pointBuilder.addField(columnName, (boolean) value); - } else if (long.class.isAssignableFrom(fieldType) || Long.class.isAssignableFrom(fieldType)) { - pointBuilder.addField(columnName, (long) value); - } else if (double.class.isAssignableFrom(fieldType) - || Double.class.isAssignableFrom(fieldType)) { - pointBuilder.addField(columnName, (double) value); - } else if (int.class.isAssignableFrom(fieldType) || Integer.class.isAssignableFrom(fieldType)) { - pointBuilder.addField(columnName, (int) value); - } else if (String.class.isAssignableFrom(fieldType)) { - pointBuilder.addField(columnName, (String) value); - } else { - throw new InfluxDBMapperException( - "Unsupported type " + fieldType + " for column " + columnName); - } - } - - public List parseSeriesAs( - final Class clazz, final QueryResult.Series series, final TimeUnit precision) { - influxDBResultMapper.cacheMeasurementClass(clazz); - return series.getValues().stream() - .map(v -> parseRowAs(clazz, series.getColumns(), v, precision)) - .collect(Collectors.toList()); - } - - public T parseRowAs( - final Class clazz, List columns, final List values, TimeUnit precision) { - - try { - ConcurrentMap fieldMap = influxDBResultMapper.getColNameAndFieldMap(clazz); - - T object = null; - - for (int i = 0; i < columns.size(); i++) { - Field correspondingField = fieldMap.get(columns.get(i)); - - if (correspondingField != null) { - if (object == null) { - object = clazz.newInstance(); - } - - influxDBResultMapper.setFieldValue(object, correspondingField, values.get(i), precision); - } - } - - return object; - } catch (InstantiationException | IllegalAccessException e) { - throw new InfluxDBMapperException(e); - } - } -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala deleted file mode 100644 index e4cdee38c3..0000000000 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/InfluxDbWriteSettings.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (C) 2016-2019 Lightbend Inc. - */ - -package akka.stream.alpakka.influxdb - -object InfluxDbWriteSettings { - val Default = new InfluxDbWriteSettings(batchSize = 10) - - def apply(): InfluxDbWriteSettings = Default - -} - -final class InfluxDbWriteSettings private ( - val batchSize: Int, -) { - - def withBatchSize(value: Int): InfluxDbWriteSettings = copy(batchSize = value) - - private def copy( - batchSize: Int = batchSize, - ): InfluxDbWriteSettings = new InfluxDbWriteSettings( - batchSize = batchSize - ) - - override def toString: String = - s"""InfluxDbWriteSettings(batchSize=$batchSize)""" - -} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/AlpakkaResultMapperHelper.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/AlpakkaResultMapperHelper.scala new file mode 100644 index 0000000000..e7800d41d8 --- /dev/null +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/AlpakkaResultMapperHelper.scala @@ -0,0 +1,260 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package akka.stream.alpakka.influxdb.impl + +import java.lang.reflect.Field +import java.time.Instant +import java.time.format.DateTimeFormatterBuilder +import java.time.temporal.ChronoField +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} + +import org.influxdb.annotation.{Column, Measurement} +import org.influxdb.dto.QueryResult +import java.util.concurrent.TimeUnit +import org.influxdb.InfluxDBMapperException +import org.influxdb.dto.Point + +import scala.collection.JavaConverters._ + +private[impl] class AlpakkaResultMapperHelper { + + val CLASS_FIELD_CACHE: ConcurrentHashMap[String, ConcurrentMap[String, Field]] = new ConcurrentHashMap(); + + private val FRACTION_MIN_WIDTH = 0 + private val FRACTION_MAX_WIDTH = 9 + private val ADD_DECIMAL_POINT = true + + private val RFC3339_FORMATTER = new DateTimeFormatterBuilder() + .appendPattern("yyyy-MM-dd'T'HH:mm:ss") + .appendFraction(ChronoField.NANO_OF_SECOND, FRACTION_MIN_WIDTH, FRACTION_MAX_WIDTH, ADD_DECIMAL_POINT) + .appendZoneOrOffsetId + .toFormatter + + private[impl] def databaseName(point: Class[_]): String = + point.getAnnotation(classOf[Measurement]).database(); + + private[impl] def retentionPolicy(point: Class[_]): String = + point.getAnnotation(classOf[Measurement]).retentionPolicy(); + + private[impl] def convertModelToPoint[T](model: T): Point = { + throwExceptionIfMissingAnnotation(model.getClass) + cacheClassFields(model.getClass) + + val colNameAndFieldMap: ConcurrentMap[String, Field] = CLASS_FIELD_CACHE.get(model.getClass.getName) + + try { + val modelType = model.getClass(); + val measurement = measurementName(modelType); + val timeUnit: TimeUnit = this.timeUnit(modelType); + val time = timeUnit.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS); + val pointBuilder: Point.Builder = Point.measurement(measurement).time(time, timeUnit); + + for (key <- colNameAndFieldMap.keySet().asScala) { + val field = colNameAndFieldMap.get(key) + val column = field.getAnnotation(classOf[Column]) + val columnName: String = column.name() + val fieldType: Class[_] = field.getType() + + if (!field.isAccessible()) { + field.setAccessible(true); + } + + val value = field.get(model); + + if (column.tag()) { + pointBuilder.tag(columnName, value.toString()); + } else if ("time".equals(columnName)) { + if (value != null) { + setTime(pointBuilder, fieldType, timeUnit, value); + } + } else { + setField(pointBuilder, fieldType, columnName, value); + } + } + + pointBuilder.build(); + } catch { + case e: IllegalArgumentException => throw new InfluxDBMapperException(e); + } + } + + private[impl] def cacheClassFields(clazz: Class[_]) = + if (!CLASS_FIELD_CACHE.containsKey(clazz.getName)) { + val initialMap: ConcurrentMap[String, Field] = new ConcurrentHashMap() + var influxColumnAndFieldMap = CLASS_FIELD_CACHE.putIfAbsent(clazz.getName, initialMap) + + if (influxColumnAndFieldMap == null) { + influxColumnAndFieldMap = initialMap; + } + + var c = clazz; + + while (c != null) { + for (field <- c.getDeclaredFields()) { + val colAnnotation = field.getAnnotation(classOf[Column]); + if (colAnnotation != null) { + influxColumnAndFieldMap.put(colAnnotation.name(), field); + } + } + c = c.getSuperclass(); + } + } + + private[impl] def parseSeriesAs[T](clazz: Class[T], series: QueryResult.Series, precision: TimeUnit): List[T] = { + cacheClassFields(clazz) + series.getValues.asScala + .map((v: java.util.List[AnyRef]) => parseRowAs(clazz, series.getColumns, v, precision)) + .toList + } + + private def measurementName(point: Class[_]): String = + point.getAnnotation(classOf[Measurement]).name(); + + private def timeUnit(point: Class[_]): TimeUnit = + point.getAnnotation(classOf[Measurement]).timeUnit() + + private def setTime(pointBuilder: Point.Builder, fieldType: Class[_], timeUnit: TimeUnit, value: Any): Unit = + if (classOf[Instant].isAssignableFrom(fieldType)) { + val instant = value.asInstanceOf[Instant] + val time = timeUnit.convert(instant.toEpochMilli, TimeUnit.MILLISECONDS) + pointBuilder.time(time, timeUnit) + } else throw new InfluxDBMapperException("Unsupported type " + fieldType + " for time: should be of Instant type") + + private def setField(pointBuilder: Point.Builder, fieldType: Class[_], columnName: String, value: Any): Unit = + if (classOf[java.lang.Boolean].isAssignableFrom(fieldType) || classOf[Boolean].isAssignableFrom(fieldType)) + pointBuilder.addField(columnName, value.asInstanceOf[Boolean]) + else if (classOf[java.lang.Long].isAssignableFrom(fieldType) || classOf[Long].isAssignableFrom(fieldType)) + pointBuilder.addField(columnName, value.asInstanceOf[Long]) + else if (classOf[java.lang.Double].isAssignableFrom(fieldType) || classOf[Double].isAssignableFrom(fieldType)) + pointBuilder.addField(columnName, value.asInstanceOf[Double]) + else if (classOf[java.lang.Integer].isAssignableFrom(fieldType) || classOf[Integer].isAssignableFrom(fieldType)) + pointBuilder.addField(columnName, value.asInstanceOf[Int]) + else if (classOf[String].isAssignableFrom(fieldType)) pointBuilder.addField(columnName, value.asInstanceOf[String]) + else throw new InfluxDBMapperException("Unsupported type " + fieldType + " for column " + columnName) + + private def throwExceptionIfMissingAnnotation(clazz: Class[_]): Unit = + if (!clazz.isAnnotationPresent(classOf[Measurement])) + throw new IllegalArgumentException( + "Class " + clazz.getName + " is not annotated with @" + classOf[Measurement].getSimpleName + ) + + private def parseRowAs[T](clazz: Class[T], + columns: java.util.List[String], + values: java.util.List[AnyRef], + precision: TimeUnit): T = + try { + val fieldMap = CLASS_FIELD_CACHE.get(clazz.getName) + + val obj: T = clazz.newInstance() + for (i <- 0 until columns.size()) { + val correspondingField = fieldMap.get(columns.get(i)) + if (correspondingField != null) { + setFieldValue(obj, correspondingField, values.get(i), precision) + } + } + obj + } catch { + case e @ (_: InstantiationException | _: IllegalAccessException) => + throw new InfluxDBMapperException(e) + } + + @throws[IllegalArgumentException] + @throws[IllegalAccessException] + private def setFieldValue[T](obj: T, field: Field, value: Any, precision: TimeUnit): Unit = { + if (value == null) return + val fieldType = field.getType + try { + if (!field.isAccessible) field.setAccessible(true) + if (fieldValueModified(fieldType, field, obj, value, precision) || fieldValueForPrimitivesModified( + fieldType, + field, + obj, + value + ) || fieldValueForPrimitiveWrappersModified(fieldType, field, obj, value)) return + val msg = + s"""Class '${obj.getClass.getName}' field '${field.getName}' is from an unsupported type '${field.getType}'.""" + throw new InfluxDBMapperException(msg) + } catch { + case e: ClassCastException => + val msg = + s"""Class '${obj.getClass.getName}' field '${field.getName}' was defined with a different field type and caused a ClassCastException. + |The correct type is '${value.getClass.getName}' (current field value: '${value}')""".stripMargin + throw new InfluxDBMapperException(msg) + } + } + + @throws[IllegalArgumentException] + @throws[IllegalAccessException] + private def fieldValueForPrimitivesModified[T](fieldType: Class[_], field: Field, obj: T, value: Any): Boolean = + if (classOf[Double].isAssignableFrom(fieldType)) { + field.setDouble(obj, value.asInstanceOf[Double].doubleValue) + true + } else if (classOf[Long].isAssignableFrom(fieldType)) { + field.setLong(obj, value.asInstanceOf[Double].longValue) + true + } else if (classOf[Int].isAssignableFrom(fieldType)) { + field.setInt(obj, value.asInstanceOf[Double].intValue) + true + } else if (classOf[Boolean].isAssignableFrom(fieldType)) { + field.setBoolean(obj, String.valueOf(value).toBoolean) + true + } else { + false + } + + @throws[IllegalArgumentException] + @throws[IllegalAccessException] + private def fieldValueForPrimitiveWrappersModified[T](fieldType: Class[_], + field: Field, + obj: T, + value: Any): Boolean = + if (classOf[java.lang.Double].isAssignableFrom(fieldType)) { + field.set(obj, value) + true + } else if (classOf[java.lang.Long].isAssignableFrom(fieldType)) { + field.set(obj, value.asInstanceOf[Double].longValue()) + true + } else if (classOf[Integer].isAssignableFrom(fieldType)) { + field.set(obj, value.asInstanceOf[java.lang.Integer]) + true + } else if (classOf[java.lang.Boolean].isAssignableFrom(fieldType)) { + field.set(obj, value.asInstanceOf[java.lang.Boolean]) + true + } else { + false + } + + @throws[IllegalArgumentException] + @throws[IllegalAccessException] + private def fieldValueModified[T](fieldType: Class[_], + field: Field, + obj: T, + value: Any, + precision: TimeUnit): Boolean = + if (classOf[String].isAssignableFrom(fieldType)) { + field.set(obj, String.valueOf(value)) + true + } else if (classOf[Instant].isAssignableFrom(fieldType)) { + val instant: Instant = getInstant(field, value, precision) + field.set(obj, instant) + true + } else { + false + } + + private def getInstant(field: Field, value: Any, precision: TimeUnit): Instant = + if (value.isInstanceOf[String]) Instant.from(RFC3339_FORMATTER.parse(String.valueOf(value))) + else if (value.isInstanceOf[java.lang.Long]) Instant.ofEpochMilli(toMillis(value.asInstanceOf[Long], precision)) + else if (value.isInstanceOf[java.lang.Double]) + Instant.ofEpochMilli(toMillis(value.asInstanceOf[java.lang.Double].longValue, precision)) + else if (value.isInstanceOf[java.lang.Integer]) + Instant.ofEpochMilli(toMillis(value.asInstanceOf[Integer].longValue, precision)) + else { + throw new InfluxDBMapperException(s"""Unsupported type ${field.getClass} for field ${field.getName}""") + } + + private def toMillis(value: Long, precision: TimeUnit) = TimeUnit.MILLISECONDS.convert(value, precision) + +} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index 85a488cf2f..154a90bcb4 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -12,7 +12,6 @@ import org.influxdb.{BatchOptions, InfluxDB} import scala.collection.immutable import org.influxdb.dto.{BatchPoints, Point} -import org.influxdb.impl.AlpakkaResultMapperHelper import scala.annotation.tailrec @@ -139,11 +138,11 @@ private[influxdb] final class InfluxDbMapperRecordLogic[T, C]( ( im.databaseName match { case dbn: Some[String] => dbn - case None => Some(mapperHelper.databaseName(im.point)) + case None => Some(mapperHelper.databaseName(im.point.getClass)) }, im.retentionPolicy match { case dbn: Some[String] => dbn - case None => Some(mapperHelper.retentionPolicy(im.point)) + case None => Some(mapperHelper.retentionPolicy(im.point.getClass)) } ) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index fec09d1479..80f3990525 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -10,7 +10,6 @@ import akka.stream.{ActorAttributes, Attributes, Outlet, SourceShape} import akka.stream.stage.{GraphStage, GraphStageLogic, OutHandler} import org.influxdb.{InfluxDB, InfluxDBException} import org.influxdb.dto.{Query, QueryResult} -import org.influxdb.impl.AlpakkaResultMapperHelper import scala.collection.JavaConverters._ @@ -69,8 +68,7 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], failStage(new InfluxDBException(result.getError)) } else { for (series <- result.getSeries.asScala) { - emitMultiple(outlet, - resultMapperHelper.parseSeriesAs(clazz, series, settings.precision).asScala.toIterator) + emitMultiple(outlet, resultMapperHelper.parseSeriesAs(clazz, series, settings.precision)) } } } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala index 4b4527ba1a..2e3f8cde17 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala @@ -5,51 +5,57 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import org.influxdb.InfluxDB import akka.stream.javadsl.Flow import akka.stream.alpakka.influxdb.scaladsl import org.influxdb.dto.Point +import scala.collection.JavaConverters._ object InfluxDbFlow { def create( - settings: InfluxDbWriteSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = - scaladsl.InfluxDbFlow.create(settings)(influxDB).asJava + ): Flow[java.util.List[InfluxDbWriteMessage[Point, NotUsed]], + java.util.List[InfluxDbWriteResult[Point, NotUsed]], + NotUsed] = + akka.stream.scaladsl + .Flow[java.util.List[InfluxDbWriteMessage[Point, NotUsed]]] + .map(_.asScala.toList) + .via(scaladsl.InfluxDbFlow.create()(influxDB)) + .map(_.asJava) + .asJava def typed[T]( clazz: Class[T], - settings: InfluxDbWriteSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = - scaladsl.InfluxDbFlow.typed(clazz, settings)(influxDB).asJava + ): Flow[java.util.List[InfluxDbWriteMessage[T, NotUsed]], java.util.List[InfluxDbWriteResult[T, NotUsed]], NotUsed] = + akka.stream.scaladsl + .Flow[java.util.List[InfluxDbWriteMessage[T, NotUsed]]] + .map(_.asScala.toList) + .via(scaladsl.InfluxDbFlow.typed(clazz)(influxDB)) + .map(_.asJava) + .asJava def createWithPassThrough[C]( - settings: InfluxDbWriteSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = - scaladsl.InfluxDbFlow.createWithPassThrough(settings)(influxDB).asJava + ): Flow[java.util.List[InfluxDbWriteMessage[Point, C]], java.util.List[InfluxDbWriteResult[Point, C]], NotUsed] = + akka.stream.scaladsl + .Flow[java.util.List[InfluxDbWriteMessage[Point, C]]] + .map(_.asScala.toList) + .via(scaladsl.InfluxDbFlow.createWithPassThrough(influxDB)) + .map(_.asJava) + .asJava def typedWithPassThrough[T, C]( clazz: Class[T], - settings: InfluxDbWriteSettings, influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = - scaladsl.InfluxDbFlow.typedWithPassThrough(clazz, settings)(influxDB).asJava - - def createWithContext[C]( - settings: InfluxDbWriteSettings, - influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = - scaladsl.InfluxDbFlow.createWithContext(settings)(influxDB).asJava - - def typedWithContext[T, C]( - clazz: Class[T], - settings: InfluxDbWriteSettings, - influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = - scaladsl.InfluxDbFlow.typedWithContext(clazz, settings)(influxDB).asJava + ): Flow[java.util.List[InfluxDbWriteMessage[T, C]], java.util.List[InfluxDbWriteResult[T, C]], NotUsed] = + akka.stream.scaladsl + .Flow[java.util.List[InfluxDbWriteMessage[T, C]]] + .map(_.asScala.toList) + .via(scaladsl.InfluxDbFlow.typedWithPassThrough(clazz)(influxDB)) + .map(_.asJava) + .asJava } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala index f8d7e11af1..85e37c1ae0 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala @@ -7,7 +7,7 @@ package akka.stream.alpakka.influxdb.javadsl import java.util.concurrent.CompletionStage import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.javadsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -18,18 +18,19 @@ import org.influxdb.dto.Point object InfluxDbSink { def create( - settings: InfluxDbWriteSettings, influxDB: InfluxDB - ): akka.stream.javadsl.Sink[InfluxDbWriteMessage[Point, NotUsed], CompletionStage[Done]] = + ): akka.stream.javadsl.Sink[java.util.List[InfluxDbWriteMessage[Point, NotUsed]], CompletionStage[Done]] = InfluxDbFlow - .create(settings, influxDB) - .toMat(Sink.ignore[InfluxDbWriteResult[Point, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + .create(influxDB) + .toMat(Sink.ignore[java.util.List[InfluxDbWriteResult[Point, NotUsed]]], + Keep.right[NotUsed, CompletionStage[Done]]) - def typed[T](clazz: Class[T], - settings: InfluxDbWriteSettings, - influxDB: InfluxDB): akka.stream.javadsl.Sink[InfluxDbWriteMessage[T, NotUsed], CompletionStage[Done]] = + def typed[T]( + clazz: Class[T], + influxDB: InfluxDB + ): akka.stream.javadsl.Sink[java.util.List[InfluxDbWriteMessage[T, NotUsed]], CompletionStage[Done]] = InfluxDbFlow - .typed(clazz, settings, influxDB) - .toMat(Sink.ignore[InfluxDbWriteResult[T, NotUsed]], Keep.right[NotUsed, CompletionStage[Done]]) + .typed(clazz, influxDB) + .toMat(Sink.ignore[java.util.List[InfluxDbWriteResult[T, NotUsed]]], Keep.right[NotUsed, CompletionStage[Done]]) } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala index 9816f3ae3f..59432bfadd 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} import akka.stream.javadsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala index c47dfebbca..29909b72f6 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -5,7 +5,7 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{impl, InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{impl, InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -17,62 +17,26 @@ import scala.collection.immutable */ object InfluxDbFlow { - def create(settings: InfluxDbWriteSettings)( + def create()( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[Point, NotUsed], InfluxDbWriteResult[Point, NotUsed], NotUsed] = - Flow[InfluxDbWriteMessage[Point, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[NotUsed](influxDB)) - .mapConcat(identity) + ): Flow[immutable.Seq[InfluxDbWriteMessage[Point, NotUsed]], + immutable.Seq[InfluxDbWriteResult[Point, NotUsed]], + NotUsed] = + Flow.fromGraph(new impl.InfluxDbFlowStage[NotUsed](influxDB)) - def typed[T](clazz: Class[T], settings: InfluxDbWriteSettings)( + def typed[T](clazz: Class[T])( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, NotUsed], InfluxDbWriteResult[T, NotUsed], NotUsed] = - Flow[InfluxDbWriteMessage[T, NotUsed]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbMapperFlowStage[T, NotUsed](clazz, influxDB)) - .mapConcat(identity) + ): Flow[immutable.Seq[InfluxDbWriteMessage[T, NotUsed]], immutable.Seq[InfluxDbWriteResult[T, NotUsed]], NotUsed] = + Flow.fromGraph(new impl.InfluxDbMapperFlowStage[T, NotUsed](clazz, influxDB)) - def createWithPassThrough[C](settings: InfluxDbWriteSettings)( + def createWithPassThrough[C]( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[Point, C], InfluxDbWriteResult[Point, C], NotUsed] = - Flow[InfluxDbWriteMessage[Point, C]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbFlowStage[C](influxDB)) - .mapConcat(identity) + ): Flow[immutable.Seq[InfluxDbWriteMessage[Point, C]], immutable.Seq[InfluxDbWriteResult[Point, C]], NotUsed] = + Flow.fromGraph(new impl.InfluxDbFlowStage[C](influxDB)) - def typedWithPassThrough[T, C](clazz: Class[T], settings: InfluxDbWriteSettings)( + def typedWithPassThrough[T, C](clazz: Class[T])( implicit influxDB: InfluxDB - ): Flow[InfluxDbWriteMessage[T, C], InfluxDbWriteResult[T, C], NotUsed] = - Flow[InfluxDbWriteMessage[T, C]] - .batch(settings.batchSize, immutable.Seq(_))(_ :+ _) - .via(new impl.InfluxDbMapperFlowStage[T, C](clazz, influxDB)) - .mapConcat(identity) - - def createWithContext[C](settings: InfluxDbWriteSettings)( - implicit influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[Point, NotUsed], C), (InfluxDbWriteResult[Point, C], C), NotUsed] = - Flow[(InfluxDbWriteMessage[Point, NotUsed], C)] - .map { - case (wm, pt) => wm.withPassThrough(pt) - } - .via( - createWithPassThrough(settings) - ) - .map { wr => - (wr, wr.writeMessage.passThrough) - } - - def typedWithContext[T, C](clazz: Class[T], settings: InfluxDbWriteSettings)( - implicit influxDB: InfluxDB - ): Flow[(InfluxDbWriteMessage[T, NotUsed], C), (InfluxDbWriteResult[T, C], C), NotUsed] = - Flow[(InfluxDbWriteMessage[T, NotUsed], C)] - .map { - case (wm, pt) => wm.withPassThrough(pt) - } - .via(typedWithPassThrough(clazz, settings)) - .map { wr => - (wr, wr.writeMessage.passThrough) - } + ): Flow[immutable.Seq[InfluxDbWriteMessage[T, C]], immutable.Seq[InfluxDbWriteResult[T, C]], NotUsed] = + Flow.fromGraph(new impl.InfluxDbMapperFlowStage[T, C](clazz, influxDB)) } diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index 84ec5b4868..c9220dad9d 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -5,26 +5,24 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage} import akka.stream.scaladsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point import scala.concurrent.Future +import scala.collection.immutable object InfluxDbSink { - def create( - settings: InfluxDbWriteSettings - )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[Point, NotUsed], Future[Done]] = - InfluxDbFlow.create(settings).toMat(Sink.ignore)(Keep.right) + def create()(implicit influxDB: InfluxDB): Sink[immutable.Seq[InfluxDbWriteMessage[Point, NotUsed]], Future[Done]] = + InfluxDbFlow.create.toMat(Sink.ignore)(Keep.right) def typed[T]( - clazz: Class[T], - settings: InfluxDbWriteSettings - )(implicit influxDB: InfluxDB): Sink[InfluxDbWriteMessage[T, NotUsed], Future[Done]] = + clazz: Class[T] + )(implicit influxDB: InfluxDB): Sink[immutable.Seq[InfluxDbWriteMessage[T, NotUsed]], Future[Done]] = InfluxDbFlow - .typed(clazz, settings) + .typed(clazz) .toMat(Sink.ignore)(Keep.right) } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java index 8fed19ee65..0fea13ef5e 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbSourceTest.java @@ -22,7 +22,6 @@ import akka.stream.ActorMaterializer; import akka.stream.Materializer; import akka.stream.alpakka.influxdb.InfluxDbReadSettings; -import akka.stream.alpakka.influxdb.InfluxDbWriteSettings; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; import akka.stream.javadsl.Sink; import akka.stream.testkit.javadsl.StreamTestKit; diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index 93b2cbf784..282ae3c322 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -4,9 +4,12 @@ package docs.javadsl; +import java.time.Duration; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletionStage; +import java.util.concurrent.TimeUnit; import org.influxdb.InfluxDB; import org.influxdb.dto.Point; @@ -26,7 +29,6 @@ import akka.stream.ActorMaterializer; import akka.stream.Materializer; import akka.stream.alpakka.influxdb.InfluxDbReadSettings; -import akka.stream.alpakka.influxdb.InfluxDbWriteSettings; import akka.stream.alpakka.influxdb.InfluxDbWriteMessage; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSink; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; @@ -91,9 +93,8 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { InfluxDbCpu clonedCpu = cpu.cloneAt(cpu.getTime().plusSeconds(60000l)); return InfluxDbWriteMessage.create(clonedCpu, NotUsed.notUsed()); }) - .runWith( - InfluxDbSink.typed(InfluxDbCpu.class, InfluxDbWriteSettings.Default(), influxDB), - materializer); + .groupedWithin(10, Duration.of(50l, ChronoUnit.MILLIS)) + .runWith(InfluxDbSink.typed(InfluxDbCpu.class, influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); @@ -112,7 +113,8 @@ public void testConsumeAndPublishMeasurements() throws Exception { InfluxDbSource.create(influxDB, query) .map(queryResult -> points(queryResult)) .mapConcat(i -> i) - .runWith(InfluxDbSink.create(InfluxDbWriteSettings.Default(), influxDB), materializer); + .groupedWithin(10, Duration.of(50l, ChronoUnit.MILLIS)) + .runWith(InfluxDbSink.create(influxDB), materializer); Assert.assertNotNull(completionStage.toCompletableFuture().get()); diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index 510f313434..47b3432a7e 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.TimeUnit import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbFlow import akka.stream.scaladsl.{Sink, Source} import akka.testkit.TestKit @@ -18,6 +18,7 @@ import org.scalatest.concurrent.ScalaFutures import docs.javadsl.TestUtils._ import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import org.influxdb.dto.Point +import scala.concurrent.duration._ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { @@ -50,12 +51,12 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B val result = Source( List( - validMessage + List(validMessage) ) - ).via(InfluxDbFlow.create(InfluxDbWriteSettings())) + ).via(InfluxDbFlow.create()) .runWith(Sink.seq) .futureValue - result(0).error mustBe None + result(0)(0).error mustBe None } } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala index 5cfdaeb1ac..47b90e6da2 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSourceSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} import akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource import akka.stream.scaladsl.Sink import akka.testkit.TestKit diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 66f6b29790..58bd8da599 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -12,11 +12,12 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordS import org.scalatest.concurrent.ScalaFutures import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteMessage, InfluxDbWriteSettings} +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteMessage} import akka.stream.alpakka.influxdb.scaladsl.{InfluxDbSink, InfluxDbSource} import akka.testkit.TestKit import docs.javadsl.TestUtils._ import akka.stream.scaladsl.Sink +import scala.concurrent.duration._ import scala.collection.JavaConverters._ @@ -59,16 +60,17 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi .map { cpu: InfluxDbSpecCpu => { val clonedCpu = cpu.cloneAt(cpu.getTime.plusSeconds(60000)) - InfluxDbWriteMessage(clonedCpu) + List(InfluxDbWriteMessage(clonedCpu)) } } - .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu], InfluxDbWriteSettings())) + .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu])) f1.futureValue mustBe Done val f2 = InfluxDbSource.typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query).runWith(Sink.seq) + val aFutVal = f2.futureValue f2.futureValue.length mustBe 4 } @@ -76,8 +78,8 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi val query = new Query("SELECT * FROM cpu", DatabaseName); val f1 = InfluxDbSource(influxDB, query) - .mapConcat(resultToPoints) - .runWith(InfluxDbSink.create(InfluxDbWriteSettings())) + .map(resultToPoints) + .runWith(InfluxDbSink.create()) f1.futureValue mustBe Done From 9d9412ee30bcc7c7dfa1aa6e02310e969ebcc8e8 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 13 Jun 2019 14:35:16 +0100 Subject: [PATCH 77/89] formatted InfluxDBSpec for InfluxDB connector --- influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 58bd8da599..fcfea17d87 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -94,10 +94,9 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi results <- queryResult.getResults.asScala series <- results.getSeries.asScala values <- series.getValues.asScala - } yield - ( - InfluxDbWriteMessage(resultToPoint(series, values)) - ) + } yield ( + InfluxDbWriteMessage(resultToPoint(series, values)) + ) points.toList } From 045d84a747e5eab32063311f21618cf1943b3eec Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 13 Jun 2019 14:47:26 +0100 Subject: [PATCH 78/89] formatted Dependencies for InfluxDB connector --- project/Dependencies.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index a7eacc31b3..62b385cbdf 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -234,8 +234,8 @@ object Dependencies { val InfluxDB = Seq( libraryDependencies ++= Seq( - "org.influxdb" % "influxdb-java" % InfluxDBJavaVersion // MIT - ) + "org.influxdb" % "influxdb-java" % InfluxDBJavaVersion // MIT + ) ) val IronMq = Seq( From 77104f31c189fa5ede1240060a4a7d78a27c6bcb Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 14 Jun 2019 10:00:57 +0100 Subject: [PATCH 79/89] Removed influxDB.enableBatch(BatchOptions.DEFAULTS) and influxDB.close() as the InfluxDB connector handles the batching operation by itself. --- .../akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index 154a90bcb4..e82937472e 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -71,11 +71,8 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( override def onPush(): Unit = { val messages = grab(in) if (messages.nonEmpty) { - - influxDB.enableBatch(BatchOptions.DEFAULTS) write(messages) val writtenMessages = messages.map(m => new InfluxDbWriteResult(m, None)) - influxDB.close() push(out, writtenMessages) } From 475142a6cd9ab4951f375870aa31773a6cb080a2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 14 Jun 2019 10:08:38 +0100 Subject: [PATCH 80/89] Moved runQuery to preStart so that there is data available on the first pull on InfluxDb connector --- .../akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index 80f3990525..fdbb835d24 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -56,10 +56,10 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], override def preStart(): Unit = { resultMapperHelper = new AlpakkaResultMapperHelper resultMapperHelper.cacheClassFields(clazz) + runQuery() } override def onPull(): Unit = { - runQuery() dataRetrieved match { case None => completeStage() case Some(queryResult) => { From 2bc55c3e0a95a14a386a43cd462efc03baffccd6 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 14 Jun 2019 11:12:32 +0100 Subject: [PATCH 81/89] Applied formatting on InfluxDb connector --- .../stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index fdbb835d24..5869d5ccf3 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -59,7 +59,7 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], runQuery() } - override def onPull(): Unit = { + override def onPull(): Unit = dataRetrieved match { case None => completeStage() case Some(queryResult) => { @@ -75,7 +75,6 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], dataRetrieved = None } } - } private def runQuery() = if (!queryExecuted) { From 6a832183d6ccfe21ef3ba02f244b69054239c6a3 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Fri, 14 Jun 2019 15:54:55 +0100 Subject: [PATCH 82/89] Implemented base class for InfluxDbSourceLogic and InfluxDbSourceRawLogic for InfluxDb connector --- .../influxdb/impl/InfluxDbSourceStage.scala | 71 +++++++++---------- 1 file changed, 33 insertions(+), 38 deletions(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala index 5869d5ccf3..6792f325fe 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbSourceStage.scala @@ -44,23 +44,17 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], query: Query, outlet: Outlet[T], shape: SourceShape[T]) - extends GraphStageLogic(shape) - with OutHandler { - - setHandler(outlet, this) + extends InfluxDbBaseSourceLogic[T](influxDB, query, outlet, shape) { - var queryExecuted: Boolean = false - var dataRetrieved: Option[QueryResult] = None var resultMapperHelper: AlpakkaResultMapperHelper = _ override def preStart(): Unit = { resultMapperHelper = new AlpakkaResultMapperHelper resultMapperHelper.cacheClassFields(clazz) - runQuery() + super.preStart() } - override def onPull(): Unit = - dataRetrieved match { + this.dataRetrieved match { case None => completeStage() case Some(queryResult) => { for (result <- queryResult.getResults.asScala) { @@ -76,18 +70,6 @@ private[influxdb] final class InfluxDbSourceLogic[T](clazz: Class[T], } } - private def runQuery() = - if (!queryExecuted) { - val queryResult = influxDB.query(query) - if (!queryResult.hasError) { - dataRetrieved = Some(queryResult) - } else { - failStage(new InfluxDBException(queryResult.getError)) - dataRetrieved = None - } - queryExecuted = true - } - } /** @@ -116,6 +98,24 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, influxDB: InfluxDB, outlet: Outlet[QueryResult], shape: SourceShape[QueryResult]) + extends InfluxDbBaseSourceLogic[QueryResult](influxDB, query, outlet, shape) { + + override def onPull(): Unit = + dataRetrieved match { + case None => completeStage() + case Some(queryResult) => { + emit(outlet, queryResult) + dataRetrieved = None + } + } + + override protected def validateTotalResults: Boolean = true +} + +private[impl] sealed abstract class InfluxDbBaseSourceLogic[T](influxDB: InfluxDB, + query: Query, + outlet: Outlet[T], + shape: SourceShape[T]) extends GraphStageLogic(shape) with OutHandler { @@ -124,16 +124,8 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, var queryExecuted: Boolean = false var dataRetrieved: Option[QueryResult] = None - override def onPull(): Unit = { + override def preStart(): Unit = runQuery() - dataRetrieved match { - case None => completeStage() - case Some(queryResult) => { - emit(outlet, queryResult) - dataRetrieved = None - } - } - } private def runQuery() = if (!queryExecuted) { @@ -148,14 +140,17 @@ private[influxdb] final class InfluxDbSourceRawLogic(query: Query, queryExecuted = true } - private def failOnError(result: QueryResult) = { - val totalErrors = result.getResults.asScala - .filter(_.hasError) - .map(_.getError) - if (totalErrors.size == result.getResults.size()) { - val errorMessage = totalErrors.reduceLeft((m1, m2) => m1 + ";" + m2) - failStage(new InfluxDBException(errorMessage)) + protected def validateTotalResults: Boolean = false + + private def failOnError(result: QueryResult) = + if (validateTotalResults) { + val totalErrors = result.getResults.asScala + .filter(_.hasError) + .map(_.getError) + if (totalErrors.size == result.getResults.size()) { + val errorMessage = totalErrors.reduceLeft((m1, m2) => m1 + ";" + m2) + failStage(new InfluxDBException(errorMessage)) + } } - } } From b7bed8f1fa7420a21456f86ed881a0ed27608bdb Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 10:38:19 +0100 Subject: [PATCH 83/89] Added documentation for InfluxDb connector --- docs/src/main/paradox/index.md | 1 + docs/src/main/paradox/influxdb.md | 95 ++++++++++++ .../influxdb/impl/InfluxDbFlowStage.scala | 10 +- .../influxdb/javadsl/InfluxDbFlow.scala | 3 + .../influxdb/javadsl/InfluxDbSink.scala | 2 + .../influxdb/javadsl/InfluxDbSource.scala | 4 +- .../influxdb/scaladsl/InfluxDbFlow.scala | 4 +- .../influxdb/scaladsl/InfluxDbSink.scala | 4 +- .../influxdb/scaladsl/InfluxDbSource.scala | 4 +- .../test/java/docs/javadsl/InfluxDbTest.java | 140 +++++++++++++++++- .../src/test/java/docs/javadsl/TestUtils.java | 1 + .../test/scala/docs/scaladsl/FlowSpec.scala | 72 ++++++++- .../scala/docs/scaladsl/InfluxDbFlowCpu.java | 28 ++++ .../scala/docs/scaladsl/InfluxDbSpec.scala | 17 ++- 14 files changed, 364 insertions(+), 21 deletions(-) create mode 100644 docs/src/main/paradox/influxdb.md create mode 100644 influxdb/src/test/scala/docs/scaladsl/InfluxDbFlowCpu.java diff --git a/docs/src/main/paradox/index.md b/docs/src/main/paradox/index.md index 0bc7b35d24..03854d9f3d 100644 --- a/docs/src/main/paradox/index.md +++ b/docs/src/main/paradox/index.md @@ -40,6 +40,7 @@ The [Alpakka project](https://doc.akka.io/docs/alpakka/current/) is an open sour * [HTTP](external/http.md) * [IBM Bluemix Cloud Object storage](bluemix-cos.md) * [IBM DB2 Event Store](external/db2-event-store.md) +* [InfluxDB](influxdb.md) * [IronMQ](ironmq.md) * [JMS](jms/index.md) * [MongoDB](mongodb.md) diff --git a/docs/src/main/paradox/influxdb.md b/docs/src/main/paradox/influxdb.md new file mode 100644 index 0000000000..b808422d06 --- /dev/null +++ b/docs/src/main/paradox/influxdb.md @@ -0,0 +1,95 @@ +#InfluxDB + +The Alpakka InfluxDb connector provides Akka Streams integration for InfluxDB. + +For more information about InfluxDB, please visit the [InfluxDB Documentation](https://docs.influxdata.com/) + +@@project-info{ projectId="influxdb" } + +## Artifacts + +@@dependency [sbt,Maven,Gradle] { + group=com.lightbend.akka + artifact=akka-stream-alpakka-influxdb$scala.binary.version$ + version=$project.version$ +} + +The table below shows direct dependencies of this module and the second tab shows all libraries it depends on transitively. + +@@dependencies { projectId="influxdb" } + +## Set up InfluxDB client + +Sources, Flows and Sinks provided by this connector need a prepared `org.influxdb.InfluxDB` to +access to InfluxDB. + +Scala +: @@snip [snip](/inflxudb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #init-client } + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/TestUtils.java) { #init-client } + +## InfluxDB as Source and Sink + +Now we can stream messages from or to InfluxDB by providing the `InfluxDB` to the +@scala[@scaladoc[InfluxDbSource](akka.stream.alpakka.influxdb.scaladsl.InfluxDbSource$)] +@java[@scaladoc[InfluxDbSource](akka.stream.alpakka.influxdb.javadsl.InfluxDbSource$)] +or the +@scala[@scaladoc[InfluxDbSink](akka.stream.alpakka.influxdb.scaladsl.InfluxDbSink$).] +@java[@scaladoc[InfluxDbSink](akka.stream.alpakka.influxdb.javadsl.InfluxDbSink$).] + + +Scala +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #define-class } + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #define-class } + +### With typed source + +Use `InfluxDbSource.typed` and `InfluxDbSink.typed` to create source and sink. +@scala[The data is converted by InfluxDBMapper.] +@java[The data is converted by InfluxDBMapper.] + +Scala +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #run-typed } + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #run-typed } + +### With `QueryResult` source + +Use `InfluxDbSource.create` and `InfluxDbSink.create` to create source and sink. + +Scala +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #run-query-result} + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #run-query-result} + +TODO + +### Writing to InfluxDB + +You can also build flow stages. +@scala[@scaladoc[InfluxDbFlow](akka.stream.alpakka.influxdb.scaladsl.InfluxDbFlow$).] +@java[@scaladoc[InfluxDbFlow](akka.stream.alpakka.influxdb.javadsl.InfluxDbFlow$).] +The API is similar to creating Sinks. + +Scala +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala) { #run-flow } + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #run-flow } + +### Passing data through InfluxDbFlow + +When streaming documents from Kafka, you might want to commit to Kafka **AFTER** the document has been written to InfluxDB. + +Scala +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #kafka-example } + +Java +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #kafka-example } + + diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala index e82937472e..6b1e086e5a 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/impl/InfluxDbFlowStage.scala @@ -89,12 +89,10 @@ private[influxdb] sealed abstract class InfluxDbLogic[T, C]( @tailrec def convert(messages: Seq[InfluxDbWriteMessage[T, C]]): BatchPoints = - messages match { - case head :: tail => { - builder.point(head.point.asInstanceOf[Point]) - convert(tail) - } - case Nil => builder.build() + if (messages.size == 0) builder.build() + else { + builder.point(messages.head.point.asInstanceOf[Point]) + convert(messages.tail) } convert(seq) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala index 2e3f8cde17..f596b93047 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbFlow.scala @@ -5,13 +5,16 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed +import akka.annotation.ApiMayChange import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import org.influxdb.InfluxDB import akka.stream.javadsl.Flow import akka.stream.alpakka.influxdb.scaladsl import org.influxdb.dto.Point + import scala.collection.JavaConverters._ +@ApiMayChange object InfluxDbFlow { def create( diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala index 85e37c1ae0..fba0ed1fa9 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSink.scala @@ -6,6 +6,7 @@ package akka.stream.alpakka.influxdb.javadsl import java.util.concurrent.CompletionStage +import akka.annotation.ApiMayChange import akka.{Done, NotUsed} import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.javadsl.{Keep, Sink} @@ -15,6 +16,7 @@ import org.influxdb.dto.Point /** * Java API. */ +@ApiMayChange object InfluxDbSink { def create( diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala index 59432bfadd..5d23997e10 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/javadsl/InfluxDbSource.scala @@ -5,7 +5,8 @@ package akka.stream.alpakka.influxdb.javadsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} +import akka.annotation.ApiMayChange +import akka.stream.alpakka.influxdb.InfluxDbReadSettings import akka.stream.javadsl.Source import org.influxdb.InfluxDB import org.influxdb.dto.{Query, QueryResult} @@ -14,6 +15,7 @@ import akka.stream.alpakka.influxdb.impl.{InfluxDbRawSourceStage, InfluxDbSource /** * Java API to create InfluxDB sources. */ +@ApiMayChange object InfluxDbSource { /** diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala index 29909b72f6..59d1d2be1d 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -5,7 +5,8 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{impl, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.annotation.ApiMayChange +import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, impl} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -15,6 +16,7 @@ import scala.collection.immutable /** * Scala API to create InfluxDB flows. */ +@ApiMayChange object InfluxDbFlow { def create()( diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala index c9220dad9d..37138224a0 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSink.scala @@ -4,8 +4,9 @@ package akka.stream.alpakka.influxdb.scaladsl +import akka.annotation.ApiMayChange import akka.{Done, NotUsed} -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage} +import akka.stream.alpakka.influxdb.InfluxDbWriteMessage import akka.stream.scaladsl.{Keep, Sink} import org.influxdb.InfluxDB import org.influxdb.dto.Point @@ -13,6 +14,7 @@ import org.influxdb.dto.Point import scala.concurrent.Future import scala.collection.immutable +@ApiMayChange object InfluxDbSink { def create()(implicit influxDB: InfluxDB): Sink[immutable.Seq[InfluxDbWriteMessage[Point, NotUsed]], Future[Done]] = diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala index add343c965..cc98224d23 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbSource.scala @@ -5,7 +5,8 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed -import akka.stream.alpakka.influxdb.{InfluxDbReadSettings} +import akka.annotation.ApiMayChange +import akka.stream.alpakka.influxdb.InfluxDbReadSettings import akka.stream.alpakka.influxdb.impl.{InfluxDbRawSourceStage, InfluxDbSourceStage} import akka.stream.scaladsl.Source import org.influxdb.InfluxDB @@ -14,6 +15,7 @@ import org.influxdb.dto.{Query, QueryResult} /** * Scala API. */ +@ApiMayChange object InfluxDbSource { /** diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index 282ae3c322..347072d495 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -5,11 +5,17 @@ package docs.javadsl; import java.time.Duration; +import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.stream.Collectors; import org.influxdb.InfluxDB; import org.influxdb.dto.Point; @@ -30,9 +36,12 @@ import akka.stream.Materializer; import akka.stream.alpakka.influxdb.InfluxDbReadSettings; import akka.stream.alpakka.influxdb.InfluxDbWriteMessage; +import akka.stream.alpakka.influxdb.InfluxDbWriteResult; +import akka.stream.alpakka.influxdb.javadsl.InfluxDbFlow; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSink; import akka.stream.alpakka.influxdb.javadsl.InfluxDbSource; import akka.stream.javadsl.Sink; +import akka.stream.javadsl.Source; import akka.stream.testkit.javadsl.StreamTestKit; import akka.testkit.javadsl.TestKit; import static docs.javadsl.TestUtils.cleanDatabase; @@ -40,6 +49,7 @@ import static docs.javadsl.TestUtils.populateDatabase; import static docs.javadsl.TestUtils.resultToPoint; import static docs.javadsl.TestUtils.setupConnection; +import static org.junit.Assert.assertEquals; public class InfluxDbTest { @@ -57,6 +67,35 @@ private static Pair setupMaterializer() { return Pair.create(system, materializer); } + public static class MessageFromKafka { + + private InfluxDbCpu influxDbCpu; + + private KafkaOffset kafkaOffset; + + public MessageFromKafka(InfluxDbCpu influxDbCpu, KafkaOffset kafkaOffset) { + this.influxDbCpu = influxDbCpu; + this.kafkaOffset = kafkaOffset; + } + } + + public static class KafkaOffset { + + private int offset; + + public KafkaOffset(int offset) { + this.offset = offset; + } + + public void setOffset(int offset) { + this.offset = offset; + } + + public int getOffset() { + return offset; + } + } + @BeforeClass public static void setupDatabase() { final Pair sysmat = setupMaterializer(); @@ -86,6 +125,7 @@ public void cleanUp() { @Test public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); + // #run-typed CompletionStage completionStage = InfluxDbSource.typed(InfluxDbCpu.class, InfluxDbReadSettings.Default(), influxDB, query) .map( @@ -102,13 +142,14 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { InfluxDbSource.typed(Cpu.class, InfluxDbReadSettings.Default(), influxDB, query) .runWith(Sink.seq(), materializer); - Assert.assertEquals(4, sources.toCompletableFuture().get().size()); + assertEquals(4, sources.toCompletableFuture().get().size()); } @Test public void testConsumeAndPublishMeasurements() throws Exception { Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); + // #run-query-result CompletionStage completionStage = InfluxDbSource.create(influxDB, query) .map(queryResult -> points(queryResult)) @@ -126,7 +167,102 @@ public void testConsumeAndPublishMeasurements() throws Exception { final int resultSize = queryResult.get(0).getResults().get(0).getSeries().get(0).getValues().size(); - Assert.assertEquals(4, resultSize); + assertEquals(4, resultSize); + } + + @Test + public void testPointFlow() throws Exception { + // #run-flow + Point point = + Point.measurement("disk") + .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) + .addField("used", 80L) + .addField("free", 1L) + .build(); + + InfluxDbWriteMessage influxDbWriteMessage = InfluxDbWriteMessage.create(point); + + CompletableFuture>>> completableFuture = + Source.single(Collections.singletonList(influxDbWriteMessage)) + .via(InfluxDbFlow.create(influxDB)) + .runWith(Sink.seq(), materializer) + .toCompletableFuture(); + + InfluxDbWriteResult influxDbWriteResult = completableFuture.get().get(0).get(0); + Assert.assertTrue(influxDbWriteResult.error().isEmpty()); + } + + @Test + public void typedStreamWithPassThrough() throws Exception { + // #kafka-example + // We're going to pretend we got metrics from kafka. + // After we've written them to InfluxDb, we want + // to commit the offset to Kafka + + /** Just clean the previous data */ + influxDB.query(new Query("DELETE FROM cpu")); + + List committedOffsets = new ArrayList<>(); + List messageFromKafka = + Arrays.asList( + new MessageFromKafka( + new InfluxDbCpu( + Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123L), + new KafkaOffset(0)), + new MessageFromKafka( + new InfluxDbCpu( + Instant.now().minusSeconds(2000), "local_2", "eu-west-1", 2.5d, false, 125L), + new KafkaOffset(1)), + new MessageFromKafka( + new InfluxDbCpu( + Instant.now().minusSeconds(3000), "local_3", "eu-west-4", 3.1d, false, 251L), + new KafkaOffset(2))); + + Consumer commitToKafka = + kafkaOffset -> committedOffsets.add(kafkaOffset.getOffset()); + + Source.from(messageFromKafka) + .map( + kafkaMessage -> { + return InfluxDbWriteMessage.create( + kafkaMessage.influxDbCpu, kafkaMessage.kafkaOffset); + }) + .groupedWithin(10, Duration.ofMillis(10)) + .via(InfluxDbFlow.typedWithPassThrough(InfluxDbCpu.class, influxDB)) + .map( + messages -> { + messages.stream() + .forEach( + message -> { + KafkaOffset kafkaOffset = message.writeMessage().passThrough(); + commitToKafka.accept(kafkaOffset); + }); + return NotUsed.getInstance(); + }) + .runWith(Sink.seq(), materializer) + .toCompletableFuture() + .get(10, TimeUnit.SECONDS); + // #kafka-example + + assertEquals(Arrays.asList(0, 1, 2), committedOffsets); + + List result2 = + InfluxDbSource.typed( + InfluxDbCpu.class, + InfluxDbReadSettings.Default(), + influxDB, + new Query("SELECT*FROM cpu")) + .map(m -> m.getHostname()) + .runWith(Sink.seq(), materializer) + .toCompletableFuture() + .get(10, TimeUnit.SECONDS); + + assertEquals( + messageFromKafka.stream() + .map(m -> m.influxDbCpu.getHostname()) + .sorted() + .collect(Collectors.toList()), + result2.stream().sorted().collect(Collectors.toList())); } private List> points(QueryResult queryResult) { diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index e8b9642426..add3145e31 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -22,6 +22,7 @@ public class TestUtils { + // #define-class public static InfluxDB setupConnection(final String databaseName) { final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); influxDB.setDatabase(databaseName); diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index 47b3432a7e..d4f0c19a59 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -4,12 +4,14 @@ package docs.scaladsl +import java.time.Instant import java.util.concurrent.TimeUnit +import akka.Done import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult} -import akka.stream.alpakka.influxdb.scaladsl.InfluxDbFlow +import akka.stream.alpakka.influxdb.{InfluxDbReadSettings, InfluxDbWriteMessage, InfluxDbWriteResult} +import akka.stream.alpakka.influxdb.scaladsl.{InfluxDbFlow, InfluxDbSource} import akka.stream.scaladsl.{Sink, Source} import akka.testkit.TestKit import org.influxdb.InfluxDB @@ -17,7 +19,8 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordS import org.scalatest.concurrent.ScalaFutures import docs.javadsl.TestUtils._ import akka.stream.testkit.scaladsl.StreamTestKit.assertAllStagesStopped -import org.influxdb.dto.Point +import org.influxdb.dto.{Point, Query} + import scala.concurrent.duration._ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { @@ -38,7 +41,7 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B } "mixed model" in assertAllStagesStopped { - + //#run-flow val point = Point .measurement("disk") .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) @@ -59,4 +62,65 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B result(0)(0).error mustBe None } + + "kafka-example - store metrics and pass Responses with passThrough" in assertAllStagesStopped { + //#kafka-example + // We're going to pretend we got messages from kafka. + // After we've written them to InfluxDB, we want + // to commit the offset to Kafka + + case class KafkaOffset(offset: Int) + case class KafkaMessage(cpu: InfluxDbFlowCpu, offset: KafkaOffset) + + val messagesFromKafka = List( + KafkaMessage(new InfluxDbFlowCpu(Instant.now().minusSeconds(1000), "local_1", "eu-west-2", 1.4d, true, 123L), + KafkaOffset(0)), + KafkaMessage(new InfluxDbFlowCpu(Instant.now().minusSeconds(2000), "local_2", "eu-west-1", 2.5d, false, 125L), + KafkaOffset(1)), + KafkaMessage(new InfluxDbFlowCpu(Instant.now().minusSeconds(3000), "local_3", "eu-west-4", 3.1d, false, 251L), + KafkaOffset(2)) + ) + + var committedOffsets = List[KafkaOffset]() + + def commitToKafka(offset: KafkaOffset): Unit = + committedOffsets = committedOffsets :+ offset + + val f1 = Source(messagesFromKafka) + .map { kafkaMessage: KafkaMessage => + val cpu = kafkaMessage.cpu + val id = cpu.getHostname + println("hostname: " + cpu.getHostname) + + InfluxDbWriteMessage(cpu).withPassThrough(kafkaMessage.offset) + } + .groupedWithin(10, 50.millis) + .via( + InfluxDbFlow.typedWithPassThrough(classOf[InfluxDbFlowCpu]) + ) + .map { messages: Seq[InfluxDbWriteResult[InfluxDbFlowCpu, KafkaOffset]] => + messages.foreach { message => + commitToKafka(message.writeMessage.passThrough) + } + } + .runWith(Sink.ignore) + + //#kafka-example + f1.futureValue mustBe Done + assert(List(0, 1, 2) == committedOffsets.map(_.offset)) + + val f2 = InfluxDbSource + .typed(classOf[InfluxDbFlowCpu], InfluxDbReadSettings.Default, influxDB, new Query("SELECT*FROM cpu")) + .map { cpu => + cpu.getHostname + } + .runWith(Sink.seq) + + f2.futureValue.sorted mustBe Seq( + "local_1", + "local_2", + "local_3" + ) + } + } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbFlowCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDbFlowCpu.java new file mode 100644 index 0000000000..e2b8cc6dd6 --- /dev/null +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbFlowCpu.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2016-2019 Lightbend Inc. + */ + +package docs.scaladsl; + +import java.time.Instant; + +import org.influxdb.annotation.Measurement; + +import docs.javadsl.Cpu; + +@Measurement(name = "cpu", database = "FlowSpec") +public class InfluxDbFlowCpu extends Cpu { + + public InfluxDbFlowCpu() { + } + + public InfluxDbFlowCpu(Instant time, String hostname, String region, Double idle, Boolean happydevop, Long uptimeSecs) { + super(time, hostname, region, idle, happydevop, uptimeSecs); + } + + public InfluxDbFlowCpu cloneAt(Instant time) { + return new InfluxDbFlowCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); + } + +} + diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index fcfea17d87..f9c5c8c7f3 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -6,7 +6,7 @@ package docs.scaladsl import akka.actor.ActorSystem import akka.stream.ActorMaterializer -import org.influxdb.InfluxDB +import org.influxdb.{InfluxDB, InfluxDBFactory} import org.influxdb.dto.{Point, Query, QueryResult} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, MustMatchers, WordSpec} import org.scalatest.concurrent.ScalaFutures @@ -17,10 +17,12 @@ import akka.stream.alpakka.influxdb.scaladsl.{InfluxDbSink, InfluxDbSource} import akka.testkit.TestKit import docs.javadsl.TestUtils._ import akka.stream.scaladsl.Sink -import scala.concurrent.duration._ +import scala.concurrent.duration._ import scala.collection.JavaConverters._ +import docs.javadsl.TestConstants.{INFLUXDB_URL, PASSWORD, USERNAME} + class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with BeforeAndAfterAll with ScalaFutures { implicit val system = ActorSystem() @@ -30,8 +32,12 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi implicit var influxDB: InfluxDB = _ - override protected def beforeAll(): Unit = - influxDB = setupConnection(DatabaseName) + //#define-class + override protected def beforeAll(): Unit = { + influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); + influxDB.setDatabase(DatabaseName); + influxDB.query(new Query("CREATE DATABASE " + DatabaseName, DatabaseName)); + } override protected def afterAll(): Unit = TestKit.shutdownActorSystem(system) @@ -55,6 +61,7 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "consume and publish measurements using typed" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); + //#run-typed val f1 = InfluxDbSource .typed(classOf[InfluxDbSpecCpu], InfluxDbReadSettings(), influxDB, query) .map { cpu: InfluxDbSpecCpu => @@ -76,7 +83,7 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi "consume and publish measurements" in assertAllStagesStopped { val query = new Query("SELECT * FROM cpu", DatabaseName); - + //#run-query-result val f1 = InfluxDbSource(influxDB, query) .map(resultToPoints) .runWith(InfluxDbSink.create()) From f5dd3d827bada0859bef3365806c9111e44a7bcb Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 11:54:03 +0100 Subject: [PATCH 84/89] Formatted InfluxDbFlow for InfluxDb connector --- .../akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala index 59d1d2be1d..7306404943 100644 --- a/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala +++ b/influxdb/src/main/scala/akka/stream/alpakka/influxdb/scaladsl/InfluxDbFlow.scala @@ -6,7 +6,7 @@ package akka.stream.alpakka.influxdb.scaladsl import akka.NotUsed import akka.annotation.ApiMayChange -import akka.stream.alpakka.influxdb.{InfluxDbWriteMessage, InfluxDbWriteResult, impl} +import akka.stream.alpakka.influxdb.{impl, InfluxDbWriteMessage, InfluxDbWriteResult} import akka.stream.scaladsl.Flow import org.influxdb.InfluxDB import org.influxdb.dto.Point From 4185743120ab03b320d26ad56f86778961d87dee Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 12:10:32 +0100 Subject: [PATCH 85/89] Added to project-info --- project/project-info.conf | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/project/project-info.conf b/project/project-info.conf index a3b8854cd0..48bb92ab85 100644 --- a/project/project-info.conf +++ b/project/project-info.conf @@ -348,6 +348,24 @@ project-info { } ] } + influxdb: ${project-info.shared-info} { + title: "Alpakka InfluxDB" + jpms-name: "akka.stream.alpakka.influxdb" + issues.url: ${project-info.labels}"influxdb" + levels: [ + { + readiness: CommunityDriven + since: "2019-05-05" + since-version: "1.0.3" + } + ] + api-docs: [ + { + url: ${project-info.scaladoc}"influxdb/index.html" + text: "API (Scaladoc)" + } + ] + } hdfs: ${project-info.shared-info} { title: "Alpakka HDFS" jpms-name: "akka.stream.alpakka.hdfs" From de06fd60002c633dfe998c4bcf193406471906c2 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 13:22:49 +0100 Subject: [PATCH 86/89] Typo fix --- docs/src/main/paradox/influxdb.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/src/main/paradox/influxdb.md b/docs/src/main/paradox/influxdb.md index b808422d06..d071df7040 100644 --- a/docs/src/main/paradox/influxdb.md +++ b/docs/src/main/paradox/influxdb.md @@ -24,7 +24,7 @@ Sources, Flows and Sinks provided by this connector need a prepared `org.influxd access to InfluxDB. Scala -: @@snip [snip](/inflxudb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #init-client } +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #init-client } Java : @@snip [snip](/influxdb/src/test/java/docs/javadsl/TestUtils.java) { #init-client } From efdb55341994a1df3b0e025e7af991a208bce72c Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 13:36:52 +0100 Subject: [PATCH 87/89] Added #init-client on InfluxdbSPec for InfluxDB connector --- influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index f9c5c8c7f3..530603a884 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -34,9 +34,11 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi //#define-class override protected def beforeAll(): Unit = { + //#init-client influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); influxDB.setDatabase(DatabaseName); influxDB.query(new Query("CREATE DATABASE " + DatabaseName, DatabaseName)); + //#init-client } override protected def afterAll(): Unit = From d96e68bbb5306d757bba91af806d284367c48626 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 14:05:58 +0100 Subject: [PATCH 88/89] Added #init-client on InfluxDB connector --- influxdb/src/test/java/docs/javadsl/TestUtils.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/influxdb/src/test/java/docs/javadsl/TestUtils.java b/influxdb/src/test/java/docs/javadsl/TestUtils.java index add3145e31..e1cc5b1c0f 100644 --- a/influxdb/src/test/java/docs/javadsl/TestUtils.java +++ b/influxdb/src/test/java/docs/javadsl/TestUtils.java @@ -24,10 +24,12 @@ public class TestUtils { // #define-class public static InfluxDB setupConnection(final String databaseName) { + //#init-client final InfluxDB influxDB = InfluxDBFactory.connect(INFLUXDB_URL, USERNAME, PASSWORD); influxDB.setDatabase(databaseName); influxDB.query(new Query("CREATE DATABASE " + databaseName, databaseName)); return influxDB; + //#init-client } public static void populateDatabase(InfluxDB influxDB, Class clazz) throws Exception { From 441125d68ef44ad4d18d2d69319c8c2f0587cbc3 Mon Sep 17 00:00:00 2001 From: gkatzioura Date: Thu, 27 Jun 2019 16:26:48 +0100 Subject: [PATCH 89/89] Documentation fixes on InfluxDB connector --- docs/src/main/paradox/influxdb.md | 6 +++--- influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java | 2 ++ influxdb/src/test/java/docs/javadsl/InfluxDbTest.java | 7 +++++-- influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala | 3 ++- influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala | 5 ++++- influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java | 2 ++ 6 files changed, 18 insertions(+), 7 deletions(-) diff --git a/docs/src/main/paradox/influxdb.md b/docs/src/main/paradox/influxdb.md index d071df7040..2caf8f3a39 100644 --- a/docs/src/main/paradox/influxdb.md +++ b/docs/src/main/paradox/influxdb.md @@ -40,10 +40,10 @@ or the Scala -: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #define-class } +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java) { #define-class } Java -: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #define-class } +: @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java) { #define-class } ### With typed source @@ -87,7 +87,7 @@ Java When streaming documents from Kafka, you might want to commit to Kafka **AFTER** the document has been written to InfluxDB. Scala -: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala) { #kafka-example } +: @@snip [snip](/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala) { #kafka-example } Java : @@snip [snip](/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java) { #kafka-example } diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java b/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java index 2b11154a37..36ac91614b 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbCpu.java @@ -8,6 +8,7 @@ import org.influxdb.annotation.Measurement; +//#define-class @Measurement(name = "cpu", database = "InfluxDbTest") public class InfluxDbCpu extends Cpu { @@ -28,3 +29,4 @@ public InfluxDbCpu cloneAt(Instant time) { time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); } } +//#define-class diff --git a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java index 347072d495..a0e0a07e32 100644 --- a/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java +++ b/influxdb/src/test/java/docs/javadsl/InfluxDbTest.java @@ -135,6 +135,7 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { }) .groupedWithin(10, Duration.of(50l, ChronoUnit.MILLIS)) .runWith(InfluxDbSink.typed(InfluxDbCpu.class, influxDB), materializer); + //#run-typed Assert.assertNotNull(completionStage.toCompletableFuture().get()); @@ -147,15 +148,16 @@ public void testConsumeAndPublishMeasurementsUsingTyped() throws Exception { @Test public void testConsumeAndPublishMeasurements() throws Exception { + //#run-query-result Query query = new Query("SELECT * FROM cpu", DATABASE_NAME); - // #run-query-result CompletionStage completionStage = InfluxDbSource.create(influxDB, query) .map(queryResult -> points(queryResult)) .mapConcat(i -> i) .groupedWithin(10, Duration.of(50l, ChronoUnit.MILLIS)) .runWith(InfluxDbSink.create(influxDB), materializer); + //#run-query-result Assert.assertNotNull(completionStage.toCompletableFuture().get()); @@ -172,7 +174,6 @@ public void testConsumeAndPublishMeasurements() throws Exception { @Test public void testPointFlow() throws Exception { - // #run-flow Point point = Point.measurement("disk") .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) @@ -182,11 +183,13 @@ public void testPointFlow() throws Exception { InfluxDbWriteMessage influxDbWriteMessage = InfluxDbWriteMessage.create(point); + //#run-flow CompletableFuture>>> completableFuture = Source.single(Collections.singletonList(influxDbWriteMessage)) .via(InfluxDbFlow.create(influxDB)) .runWith(Sink.seq(), materializer) .toCompletableFuture(); + //#run-flow InfluxDbWriteResult influxDbWriteResult = completableFuture.get().get(0).get(0); Assert.assertTrue(influxDbWriteResult.error().isEmpty()); diff --git a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala index d4f0c19a59..e2c6d796fc 100644 --- a/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/FlowSpec.scala @@ -41,7 +41,6 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B } "mixed model" in assertAllStagesStopped { - //#run-flow val point = Point .measurement("disk") .time(System.currentTimeMillis(), TimeUnit.MILLISECONDS) @@ -52,6 +51,7 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B val validMessage = InfluxDbWriteMessage(point) .withDatabaseName(DatabaseName) + //#run-flow val result = Source( List( List(validMessage) @@ -59,6 +59,7 @@ class FlowSpec extends WordSpec with MustMatchers with BeforeAndAfterEach with B ).via(InfluxDbFlow.create()) .runWith(Sink.seq) .futureValue + //#run-flow result(0)(0).error mustBe None } diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala index 530603a884..18d2979f82 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpec.scala @@ -73,6 +73,7 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi } } .runWith(InfluxDbSink.typed(classOf[InfluxDbSpecCpu])) + //#run-typed f1.futureValue mustBe Done @@ -84,11 +85,13 @@ class InfluxDbSpec extends WordSpec with MustMatchers with BeforeAndAfterEach wi } "consume and publish measurements" in assertAllStagesStopped { - val query = new Query("SELECT * FROM cpu", DatabaseName); //#run-query-result + val query = new Query("SELECT * FROM cpu", DatabaseName); + val f1 = InfluxDbSource(influxDB, query) .map(resultToPoints) .runWith(InfluxDbSink.create()) + //#run-query-result f1.futureValue mustBe Done diff --git a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java index 84ade2ea23..b1543fb2e0 100644 --- a/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java +++ b/influxdb/src/test/scala/docs/scaladsl/InfluxDbSpecCpu.java @@ -10,6 +10,7 @@ import docs.javadsl.Cpu; +//#define-class @Measurement(name = "cpu", database = "InfluxDbSpec") public class InfluxDbSpecCpu extends Cpu { public InfluxDbSpecCpu() { @@ -23,3 +24,4 @@ public InfluxDbSpecCpu cloneAt(Instant time) { return new InfluxDbSpecCpu(time, getHostname(), getRegion(), getIdle(), getHappydevop(), getUptimeSecs()); } } +//#define-class \ No newline at end of file