├── .github └── workflows │ └── main.yaml ├── .gitignore ├── .scalafmt.conf ├── README.me ├── build.sbt ├── example ├── Makefile ├── docker-compose.yml ├── kafka-connect-influxdb.json └── sample-points.txt └── src ├── main └── scala │ └── com │ └── influxdata │ └── influxdb │ ├── InfluxDBConnector.scala │ ├── InfluxDBSinkConfig.scala │ ├── InfluxDBTask.scala │ └── InfluxDBWriter.scala └── test └── scala └── com └── influxdata └── influxdb ├── TestInfluxDBBase.scala ├── TestInfluxDBSinkConfig.scala └── TestInfluxDBWriter.scala /.github/workflows/main.yaml: -------------------------------------------------------------------------------- 1 | name: Scala Build 2 | on: [push] 3 | 4 | jobs: 5 | build: 6 | runs-on: ubuntu-18.04 7 | name: Scala Build 8 | steps: 9 | - uses: actions/checkout@v1 10 | - name: Tests 11 | uses: lokkju/github-action-sbt@master 12 | with: 13 | commands: test 14 | sbt_project_directory: ./build 15 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .bloop 2 | .metals 3 | project 4 | target 5 | -------------------------------------------------------------------------------- /.scalafmt.conf: -------------------------------------------------------------------------------- 1 | version = "2.4.1" 2 | -------------------------------------------------------------------------------- /README.me: -------------------------------------------------------------------------------- 1 | # Kafka Connect - InfluxDB 2 | 3 | This is an InfluxDB connector for Kafka 4 | 5 | Work in Progress 6 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | scalaVersion := "2.12.10" 2 | 3 | name := "kafka-connect-influxdb" 4 | organization := "com.influxdata" 5 | version := "1.0" 6 | 7 | resolvers += "Akka Snapshot Repository" at "https://repo.akka.io/snapshots/" 8 | resolvers += "Sonatype OSS Snapshots" at "https://oss.sonatype.org/content/repositories/snapshots/" 9 | 10 | libraryDependencies ++= Seq( 11 | "ch.qos.logback" % "logback-classic" % "1.2.3", 12 | "com.influxdb" % "influxdb-client-scala" % "1.5.0", 13 | "com.lihaoyi" %% "requests" % "0.5.1", 14 | "com.lihaoyi" %% "ujson" % "0.9.5", 15 | "com.typesafe.akka" %% "akka-actor" % "2.5.20", 16 | "com.typesafe.akka" %% "akka-stream" % "2.5.20", 17 | "com.typesafe.scala-logging" %% "scala-logging" % "3.9.2", 18 | "org.apache.kafka" % "connect-api" % "2.4.0", 19 | "org.scalactic" %% "scalactic" % "3.1.1" 20 | ) 21 | 22 | val testcontainersScalaVersion = "0.35.2" 23 | 24 | libraryDependencies ++= Seq( 25 | "org.scalatest" %% "scalatest" % "3.1.1" % "test", 26 | "org.apache.kafka" %% "kafka" % "2.4.0" % "test", 27 | "com.dimafeng" %% "testcontainers-scala-scalatest" % testcontainersScalaVersion % "test", 28 | "com.dimafeng" %% "testcontainers-scala-kafka" % testcontainersScalaVersion % "test", 29 | "com.typesafe.akka" %% "akka-stream-testkit" % "2.5.20" % "test" 30 | ) 31 | 32 | publishArtifact in Test := true 33 | publishArtifact in (Compile, packageDoc) := true 34 | publishArtifact in (Compile, packageSrc) := true 35 | publishArtifact in (Compile, packageBin) := true 36 | 37 | fork in run := true 38 | 39 | licenses += ("MIT", url( 40 | "https://github.com/influxdata/kafka-connect-influxdata/blob/master/LICENSE" 41 | )) 42 | publishMavenStyle := true 43 | -------------------------------------------------------------------------------- /example/Makefile: -------------------------------------------------------------------------------- 1 | .DEFAULT: _all 2 | 3 | _all: up sleep kafka-topics influxdb-setup sleep kafka-connect-register 4 | 5 | clean: 6 | @docker-compose down -v 7 | 8 | up: 9 | @docker-compose up -d 10 | 11 | sleep: 12 | @sleep 10 13 | 14 | influxdb-setup: 15 | docker-compose exec influxdb influx setup --force --org connect --bucket connect --retention 0 --token connect --username connect --password connect1234 16 | 17 | kafka-topics: 18 | @docker-compose exec kafka kafka-topics --zookeeper=zookeeper:2181 --create --topic influxdb-poc-config --partitions 1 --replication-factor 1 --if-not-exists 19 | @docker-compose exec kafka kafka-topics --zookeeper=zookeeper:2181 --create --topic influxdb-poc-offsets --partitions 1 --replication-factor 1 --if-not-exists 20 | @docker-compose exec kafka kafka-topics --zookeeper=zookeeper:2181 --create --topic influxdb-poc-status --partitions 1 --replication-factor 1 --if-not-exists 21 | @docker-compose exec kafka kafka-topics --zookeeper=zookeeper:2181 --create --topic points --partitions 1 --replication-factor 1 --if-not-exists 22 | 23 | kafka-connect-register: 24 | @curl -X POST http://localhost:8083/connectors \ 25 | -H 'Content-Type:application/json' \ 26 | -H 'Accept:application/json' \ 27 | -d @./kafka-connect-influxdb.json 28 | 29 | write-to-kafka: 30 | @docker-compose exec kafka bash -c "kafka-console-producer --broker-list kafka:9092 --topic points < /data/sample-points.txt" 31 | -------------------------------------------------------------------------------- /example/docker-compose.yml: -------------------------------------------------------------------------------- 1 | version: "2" 2 | 3 | services: 4 | influxdb: 5 | image: quay.io/influxdb/influxdb:2.0.0-beta 6 | ports: 7 | - "9999:9999" 8 | 9 | zookeeper: 10 | image: zookeeper:3.4.9 11 | environment: 12 | ZOO_MY_ID: 1 13 | ZOO_PORT: 2181 14 | ZOO_SERVERS: server.1=zookeeper:2888:3888 15 | 16 | kafka: 17 | image: confluentinc/cp-kafka:5.4.0 18 | environment: 19 | KAFKA_BROKER_ID: 1 20 | KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 21 | KAFKA_ADVERTISED_LISTENERS: PLAINTEXT://kafka:9092 22 | KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 23 | volumes: 24 | - ./sample-points.txt:/data/sample-points.txt 25 | depends_on: 26 | - zookeeper 27 | 28 | schema-registry: 29 | image: confluentinc/cp-schema-registry:5.0.0 30 | environment: 31 | SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: zookeeper:2181 32 | SCHEMA_REGISTRY_HOST_NAME: schema-registry 33 | SCHEMA_REGISTRY_LISTENERS: http://schema-registry:8081 34 | 35 | kafka-connect: 36 | image: confluentinc/cp-kafka-connect-base:latest 37 | depends_on: 38 | - kafka 39 | - schema-registry 40 | - zookeeper 41 | ports: 42 | - "8083:8083" 43 | environment: 44 | CONNECT_BOOTSTRAP_SERVERS: kafka:9092 45 | CONNECT_REST_ADVERTISED_HOST_NAME: kafka-connect 46 | CONNECT_REST_PORT: 8083 47 | CONNECT_GROUP_ID: influxdb 48 | CONNECT_CONFIG_STORAGE_TOPIC: influxdb-poc-config 49 | CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR: 1 50 | CONNECT_OFFSET_FLUSH_INTERVAL_MS: 10000 51 | CONNECT_OFFSET_STORAGE_TOPIC: influxdb-poc-offsets 52 | CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR: 1 53 | CONNECT_STATUS_STORAGE_TOPIC: influxdb-poc-status 54 | CONNECT_STATUS_STORAGE_REPLICATION_FACTOR: 1 55 | CONNECT_KEY_CONVERTER: org.apache.kafka.connect.json.JsonConverter 56 | CONNECT_KEY_CONVERTER_SCHEMAS_ENABLE: "false" 57 | CONNECT_VALUE_CONVERTER: org.apache.kafka.connect.json.JsonConverter 58 | CONNECT_VALUE_CONVERTER_SCHEMAS_ENABLE: "false" 59 | CONNECT_INTERNAL_KEY_CONVERTER: org.apache.kafka.connect.json.JsonConverter 60 | CONNECT_INTERNAL_VALUE_CONVERTER: org.apache.kafka.connect.json.JsonConverter 61 | CONNECT_ZOOKEEPER_CONNECT: zookeeper:2181 62 | CONNECT_PLUGIN_PATH: /usr/share/java,/etc/kafka-connect/jars 63 | volumes: 64 | - ../target/scala-2.12/kafka-connect-influxdb-assembly-1.0.jar:/etc/kafka-connect/jars/kafka-connect-influxdb.jar 65 | -------------------------------------------------------------------------------- /example/kafka-connect-influxdb.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "connect.sink.influxdb", 3 | "config": { 4 | "connector.class": "com.influxdata.influxdb.InfluxDBConnector", 5 | "tasks.max": "1", 6 | "topics": "points", 7 | "key.converter": "org.apache.kafka.connect.json.JsonConverter", 8 | "key.converter.schemas.enable": false, 9 | "value.converter": "org.apache.kafka.connect.json.JsonConverter", 10 | "value.converter.schemas.enable": false, 11 | "influxdb.url": "http://influxdb:9999", 12 | "influxdb.org": "connect", 13 | "influxdb.token": "connect", 14 | "influxdb.bucket": "connect", 15 | "influxdb.measurement": "connect" 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /example/sample-points.txt: -------------------------------------------------------------------------------- 1 | {"measurement": "temperature", "city": "Glasgow", "country": "Scotland", "value": 5} 2 | {"measurement": "temperature", "city": "London", "country": "England", "value": 12, "humid": true} 3 | {"measurement": "temperature", "city": "Barcelona", "country": "Spain", "value": 20, "degrees": 1.456} 4 | {"measurement": "temperature", "city": "Paris", "country": "France", "value": 16} 5 | {"measurement": "temperature", "city": "Berlin", "country": "Germany", "value": 12} 6 | -------------------------------------------------------------------------------- /src/main/scala/com/influxdata/influxdb/InfluxDBConnector.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import java.util 4 | 5 | import org.apache.kafka.connect.connector.Task 6 | import org.apache.kafka.connect.errors.ConnectException 7 | import org.apache.kafka.connect.sink.SinkConnector 8 | 9 | import scala.collection.JavaConverters._ 10 | import scala.util.{Failure, Try} 11 | import com.typesafe.scalalogging.LazyLogging 12 | 13 | class InfluxDBConnector extends SinkConnector with LazyLogging { 14 | private var configProps: util.Map[String, String] = null 15 | 16 | /** 17 | * Return config definition for sink connector 18 | */ 19 | override def config() = InfluxDBSinkConfig.config 20 | 21 | /** 22 | * States which SinkTask class to use 23 | * */ 24 | override def taskClass(): Class[_ <: Task] = classOf[InfluxDBTask] 25 | 26 | /** 27 | * Set the configuration for each work and determine the split 28 | * 29 | * @param maxTasks The max number of task workers be can spawn 30 | * @return a List of configuration properties per worker 31 | * */ 32 | override def taskConfigs( 33 | maxTasks: Int 34 | ): util.List[util.Map[String, String]] = { 35 | (1 to maxTasks).map(c => configProps).toList.asJava 36 | } 37 | 38 | /** 39 | * Start the sink and set to configuration 40 | * 41 | * @param props A map of properties for the connector and worker 42 | * */ 43 | override def start(props: util.Map[String, String]): Unit = { 44 | configProps = props 45 | Try(new InfluxDBSinkConfig(props)) match { 46 | case Failure(f) => 47 | throw new ConnectException( 48 | "Couldn't start InfluxDBConnector due to configuration error.", 49 | f 50 | ) 51 | case _ => 52 | } 53 | } 54 | 55 | override def stop(): Unit = {} 56 | override def version(): String = "" 57 | } 58 | -------------------------------------------------------------------------------- /src/main/scala/com/influxdata/influxdb/InfluxDBSinkConfig.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import java.util 4 | 5 | import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} 6 | import org.apache.kafka.common.config.ConfigDef.{Importance, Type} 7 | import org.apache.kafka.connect.sink.SinkTask 8 | 9 | object InfluxDBSinkConfig { 10 | val INFLUXDB_URL = "influxdb.url" 11 | val INFLUXDB_URL_DOC = 12 | "The URL of your InfluxDB server (with port number and protocol)" 13 | 14 | val INFLUXDB_ORG = "influxdb.org" 15 | val INFLUXDB_ORG_DOC = "The InfluxDB organization name or ID" 16 | 17 | val INFLUXDB_TOKEN = "influxdb.token" 18 | val INFLUXDB_TOKEN_DOC = "The InfluxDB token" 19 | 20 | val INFLUXDB_BUCKET = "influxdb.bucket" 21 | val INFLUXDB_BUCKET_DOC = "The InfluxDB bucket name" 22 | 23 | val INFLUXDB_MEASUREMENT = "influxdb.measurement" 24 | val INFLUXDB_MEASUREMENT_DOC = 25 | "The name of the measurement for storing points" 26 | 27 | val INFLUXDB_MEASUREMENT_FIELD = "influxdb.measurement_field" 28 | val INFLUXDB_MEASUREMENT_FIELD_DOC = 29 | "The field name in the payload to use as the measurement name" 30 | 31 | val TOPICS = SinkTask.TOPICS_CONFIG 32 | val TOPICS_DOC = "The Kafka topic to read from." 33 | 34 | val config: ConfigDef = new ConfigDef() 35 | .define(INFLUXDB_URL, Type.STRING, Importance.HIGH, INFLUXDB_URL_DOC) 36 | .define(INFLUXDB_ORG, Type.STRING, Importance.HIGH, INFLUXDB_ORG_DOC) 37 | .define(INFLUXDB_TOKEN, Type.STRING, Importance.HIGH, INFLUXDB_TOKEN_DOC) 38 | .define(INFLUXDB_BUCKET, Type.STRING, Importance.HIGH, INFLUXDB_BUCKET_DOC) 39 | .define( 40 | INFLUXDB_MEASUREMENT, 41 | Type.STRING, 42 | "", 43 | Importance.LOW, 44 | INFLUXDB_MEASUREMENT_DOC 45 | ) 46 | .define( 47 | INFLUXDB_MEASUREMENT_FIELD, 48 | Type.STRING, 49 | "", 50 | Importance.LOW, 51 | INFLUXDB_MEASUREMENT_FIELD_DOC 52 | ) 53 | .define(TOPICS, Type.LIST, Importance.HIGH, TOPICS_DOC) 54 | } 55 | 56 | class InfluxDBSinkConfig(props: util.Map[String, String]) 57 | extends AbstractConfig(InfluxDBSinkConfig.config, props) {} 58 | -------------------------------------------------------------------------------- /src/main/scala/com/influxdata/influxdb/InfluxDBTask.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import java.util 4 | import java.util.HashMap 5 | import org.apache.kafka.clients.consumer.OffsetAndMetadata 6 | import org.apache.kafka.common.TopicPartition 7 | import org.apache.kafka.connect.sink.{SinkRecord, SinkTask} 8 | import scala.collection.JavaConverters._ 9 | import scala.util.{Success, Failure} 10 | import com.typesafe.scalalogging.LazyLogging 11 | import com.influxdb.client.write.Point 12 | import java.time.Instant 13 | import com.influxdb.client.domain.WritePrecision 14 | import org.apache.kafka.connect.data.Struct 15 | 16 | class InfluxDBTask extends SinkTask with LazyLogging { 17 | var writer: Option[InfluxDBWriter] = None 18 | var sinkConfig: Option[InfluxDBSinkConfig] = None 19 | 20 | override def start(props: util.Map[String, String]): Unit = { 21 | this.sinkConfig = Some(new InfluxDBSinkConfig(props)) 22 | 23 | this.writer = Some( 24 | new InfluxDBWriter( 25 | url = sinkConfig.get.getString(InfluxDBSinkConfig.INFLUXDB_URL), 26 | org = sinkConfig.get.getString(InfluxDBSinkConfig.INFLUXDB_ORG), 27 | token = sinkConfig.get.getString(InfluxDBSinkConfig.INFLUXDB_TOKEN), 28 | bucket = sinkConfig.get.getString(InfluxDBSinkConfig.INFLUXDB_BUCKET) 29 | ) 30 | ) 31 | } 32 | 33 | override def put(records: util.Collection[SinkRecord]): Unit = { 34 | if (records.size() > 0) { 35 | records.forEach((record: SinkRecord) => { 36 | val topic = record.topic() 37 | 38 | var struct: HashMap[String, Any] = 39 | record.value().asInstanceOf[HashMap[String, Any]] 40 | 41 | // TODO: allow config to specify a field in the record as the timestamp 42 | val timestamp = record.timestamp() 43 | 44 | this.writer match { 45 | case Some(writer) => { 46 | val p = Point 47 | .measurement( 48 | this.sinkConfig.get 49 | .getString(InfluxDBSinkConfig.INFLUXDB_MEASUREMENT) 50 | ) 51 | .time(timestamp, WritePrecision.MS) 52 | 53 | struct.forEach((k, v) => { 54 | v match { 55 | case s: String => p.addTag(k, s) 56 | case l: java.lang.Long => p.addField(k, l) 57 | case d: java.lang.Double => p.addField(k, d) 58 | case i: java.lang.Integer => p.addField(k, i) 59 | case b: java.lang.Boolean => p.addField(k, b) 60 | } 61 | }) 62 | 63 | writer.writePoint(p) 64 | } 65 | 66 | case None => 67 | } 68 | }) 69 | } 70 | } 71 | 72 | override def stop(): Unit = {} 73 | 74 | override def flush(map: util.Map[TopicPartition, OffsetAndMetadata]) = {} 75 | override def version(): String = "1.0.0" 76 | } 77 | -------------------------------------------------------------------------------- /src/main/scala/com/influxdata/influxdb/InfluxDBWriter.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import scala.util.Try 4 | import com.typesafe.scalalogging.LazyLogging 5 | import com.influxdb.client.InfluxDBClientFactory 6 | import com.influxdb.client.domain.WritePrecision 7 | import com.influxdb.client.write.Point 8 | 9 | class InfluxDBWriter(url: String, org: String, token: String, bucket: String) 10 | extends LazyLogging { 11 | private val influxDBClient = 12 | InfluxDBClientFactory.create(url, token.toCharArray()) 13 | 14 | def writePoint(point: Point): Boolean = { 15 | this.influxDBClient 16 | .getWriteApiBlocking() 17 | .writePoint(bucket, org, point) 18 | return true 19 | } 20 | 21 | def close(): Unit = { 22 | this.influxDBClient.close() 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/test/scala/com/influxdata/influxdb/TestInfluxDBBase.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} 4 | 5 | trait TestInfluxDBBase extends FunSuite with Matchers with BeforeAndAfter { 6 | def getSinkConfig: Map[String, String] = { 7 | Map( 8 | InfluxDBSinkConfig.INFLUXDB_URL -> "http://influxdb:9999", 9 | InfluxDBSinkConfig.INFLUXDB_ORG -> "org", 10 | InfluxDBSinkConfig.INFLUXDB_TOKEN -> "auth_token", 11 | InfluxDBSinkConfig.INFLUXDB_BUCKET -> "my_bucket", 12 | InfluxDBSinkConfig.TOPICS -> "test-sink-topic" 13 | ) 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /src/test/scala/com/influxdata/influxdb/TestInfluxDBSinkConfig.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import scala.collection.JavaConverters._ 4 | 5 | /** 6 | * Test class for {@link InfluxDBSinkConfig}. 7 | * 8 | */ 9 | class TestInfluxDBSinkConfig extends TestInfluxDBBase { 10 | test("A test InfluxDBSinkConfig should be correctly configured") { 11 | val config = getSinkConfig 12 | val taskConfig = new InfluxDBSinkConfig(config.asJava) 13 | taskConfig.getList(InfluxDBSinkConfig.TOPICS) shouldBe Seq( 14 | "test-sink-topic" 15 | ).asJava 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/test/scala/com/influxdata/influxdb/TestInfluxDBWriter.scala: -------------------------------------------------------------------------------- 1 | package com.influxdata.influxdb 2 | 3 | import akka.actor.ActorSystem 4 | import akka.stream.testkit.scaladsl.TestSink 5 | import akka.stream.ActorMaterializer 6 | import com.dimafeng.testcontainers.{ 7 | ForAllTestContainer, 8 | GenericContainer, 9 | KafkaContainer, 10 | MultipleContainers 11 | } 12 | import com.influxdb.client.domain.Query 13 | import com.influxdb.client.scala.InfluxDBClientScalaFactory 14 | import com.influxdb.client.write.Point 15 | import com.influxdb.query.FluxRecord 16 | import java.util.Properties 17 | import org.apache.kafka.clients.KafkaClient 18 | import org.apache.kafka.clients.consumer.KafkaConsumer 19 | import org.apache.kafka.clients.producer.KafkaProducer 20 | import org.apache.kafka.clients.producer.ProducerRecord 21 | import org.apache.kafka.common.serialization.{ 22 | StringDeserializer, 23 | StringSerializer 24 | } 25 | import org.scalatest.flatspec.AnyFlatSpec 26 | import org.testcontainers.containers.wait.Wait 27 | import com.influxdb.client.domain.WritePrecision 28 | 29 | class TestInfluxDBWriter extends AnyFlatSpec with ForAllTestContainer { 30 | implicit val system: ActorSystem = ActorSystem("it-tests") 31 | implicit val materializer: ActorMaterializer = ActorMaterializer() 32 | 33 | val influxDBContainer = GenericContainer( 34 | "quay.io/influxdb/influxdb:2.0.0-beta", 35 | exposedPorts = Seq(9999), 36 | waitStrategy = Wait.forHttp("/ready") 37 | ) 38 | 39 | override val container = MultipleContainers(influxDBContainer) 40 | 41 | "Writer" should "be able to write a Point to InfluxDB" in { 42 | val result = requests.post( 43 | "http://" + influxDBContainer.containerIpAddress + ":" + influxDBContainer 44 | .mappedPort(9999) + "/api/v2/setup", 45 | data = ujson.Obj( 46 | "username" -> "test", 47 | "password" -> "test5678", 48 | "token" -> "test", 49 | "org" -> "test", 50 | "bucket" -> "test" 51 | ) 52 | ) 53 | assert(result.statusCode == 201) 54 | 55 | val writer = 56 | new InfluxDBWriter( 57 | url = 58 | "http://" + influxDBContainer.containerIpAddress + ":" + influxDBContainer 59 | .mappedPort(9999), 60 | org = "test", 61 | token = "test", 62 | bucket = "test" 63 | ) 64 | 65 | val rand = scala.util.Random 66 | val pointTagKey = rand.nextString(5) 67 | val pointTagValue = rand.nextString(5) 68 | val pointFieldKey = rand.nextString(5) 69 | val pointFieldValue = rand.nextInt() 70 | 71 | val written = writer.writePoint( 72 | Point 73 | .measurement("test") 74 | .time(System.currentTimeMillis(), WritePrecision.MS) 75 | .addTag(pointTagKey, pointTagValue) 76 | .addField(pointFieldKey, pointFieldValue) 77 | ) 78 | 79 | assert(written == true) 80 | 81 | val influxDBClient = InfluxDBClientScalaFactory 82 | .create( 83 | "http://" + influxDBContainer.containerIpAddress + ":" + influxDBContainer 84 | .mappedPort(9999), 85 | "test".toCharArray() 86 | ) 87 | 88 | val countQuery = ("from(bucket: \"test\")\n" 89 | + " |> range(start: -1m)") 90 | 91 | //Result is returned as a stream 92 | val results = influxDBClient 93 | .getQueryScalaApi() 94 | .query(countQuery, "test") 95 | .runWith(TestSink.probe[FluxRecord]) 96 | 97 | val firstRecord = results.requestNext() 98 | 99 | assert(firstRecord.getMeasurement().equals("test")) 100 | assert(firstRecord.getValueByKey(pointTagKey) == pointTagValue) 101 | 102 | assert(firstRecord.getValueByKey("_field") == pointFieldKey) 103 | assert(firstRecord.getField() == pointFieldKey) 104 | assert(firstRecord.getValue() == pointFieldValue) 105 | 106 | influxDBClient.close() 107 | } 108 | } 109 | --------------------------------------------------------------------------------