From 8cf4edbbe8502d1ccf5a901b06f41811f0a11a19 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 28 Mar 2019 16:42:37 -0700 Subject: [PATCH 01/74] feat(query): Parallelize scanPartitions and all QE work in query threadpool (#293) --- .../scala/filodb.coordinator/QueryActor.scala | 4 ++- .../NodeCoordinatorActorSpec.scala | 27 +++++++++++++++++ core/src/main/resources/filodb-defaults.conf | 30 ++++++------------- .../filodb.jmh/QueryAndIngestBenchmark.scala | 18 +++++------ .../query/exec/AggrOverRangeVectors.scala | 4 ++- .../scala/filodb/query/exec/ExecPlan.scala | 10 +++++-- .../query/exec/PeriodicSamplesMapper.scala | 4 ++- 7 files changed, 61 insertions(+), 36 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 7f458dc617..d01bb625e6 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -8,6 +8,7 @@ import akka.actor.{ActorRef, ActorSystem, Props} import akka.dispatch.{Envelope, UnboundedStablePriorityMailbox} import com.typesafe.config.Config import kamon.Kamon +import monix.execution.Scheduler import filodb.coordinator.queryengine2.QueryEngine import filodb.core._ @@ -52,11 +53,12 @@ final class QueryActor(memStore: MemStore, import QueryActor._ import client.QueryCommands._ - implicit val scheduler = monix.execution.Scheduler(context.dispatcher) val config = context.system.settings.config val queryEngine2 = new QueryEngine(dataset, shardMapFunc) val queryConfig = new QueryConfig(config.getConfig("filodb.query")) + val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") * sys.runtime.availableProcessors) + implicit val scheduler = Scheduler.fixedPool(s"query-${dataset.ref}", numSchedThreads.toInt) private val tags = Map("dataset" -> dataset.ref.toString) private val lpRequests = Kamon.counter("queryactor-logicalPlan-requests").refine(tags) diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index e2369a1010..011ea417d4 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -238,6 +238,33 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew } } + it("should parse and execute concurrent LogicalPlan queries") { + val ref = setupTimeSeries() + probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(40)))) + probe.expectMsg(Ack(0L)) + + memStore.commitIndexForTesting(dataset1.ref) + + val numQueries = 6 + + val series2 = (2 to 4).map(n => s"Series $n").toSet.asInstanceOf[Set[Any]] + val multiFilter = Seq(ColumnFilter("series", Filter.In(series2))) + val q2 = LogicalPlan2Query(ref, + Aggregate(AggregationOperator.Avg, + PeriodicSeries( + RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), qOpt) + (0 until numQueries).foreach { i => probe.send(coordinatorActor, q2) } + + (0 until numQueries).foreach { _ => + probe.expectMsgPF() { + case QueryResult(_, schema, vectors) => + schema shouldEqual timeMinSchema + vectors should have length (1) + vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0) + } + } + } + ignore("should aggregate from multiple shards") { val ref = setupTimeSeries(2) probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(30)))) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index d4f72108bf..33f753c5c4 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -38,6 +38,9 @@ filodb { # Minimum step required for a query min-step = 5 seconds + + # Parallelism (query threadpool per dataset) ... ceil(available processors * factor) + threads-factor = 1.0 } shard-manager { @@ -243,6 +246,12 @@ akka { "filodb.query.LogicalPlan" = kryo } + # Reduce the number of threads used by default by the fork-join pool, as it's not really doing much work. + default-dispatcher.fork-join-executor { + parallelism-factor = 2.0 + parallelism-max = 32 + } + # Just the defaults to start with. TODO optimize and pick the executor needed. shard-status-dispatcher { type = Dispatcher @@ -303,27 +312,6 @@ akka { } } - # Just the defaults to start with. TODO optimize and pick the executor needed. - shard-status-dispatcher { - # Dispatcher is the name of the event-based dispatcher - type = Dispatcher - # What kind of ExecutionService to use - executor = "fork-join-executor" - # Configuration for the fork join pool - fork-join-executor { - # Min number of threads to cap factor-based parallelism number to - parallelism-min = 2 - # Parallelism (threads) ... ceil(available processors * factor) - parallelism-factor = 2.0 - # Max number of threads to cap factor-based parallelism number to - parallelism-max = 10 - } - # Throughput defines the maximum number of messages to be - # processed per actor before the thread jumps to the next actor. - # Set to 1 for as fair as possible. - throughput = 100 - } - # Be sure to terminate/exit JVM process after Akka shuts down. This is important for the # custom downing provider's split brain resolution to work properly. Basically, the minority # group will shut down itself and exit the process, helping to bring newer nodes online. diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 6f980ee40e..9d7111cb40 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -41,12 +41,12 @@ class QueryAndIngestBenchmark extends StrictLogging { import NodeClusterActor._ val numShards = 2 - val numSamples = 720 // 2 hours * 3600 / 10 sec interval - val numSeries = 100 + val numSamples = 1080 // 3 hours * 3600 / 10 sec interval + val numSeries = 800 var startTime = System.currentTimeMillis - (3600*1000) - val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below - val queryIntervalMin = 55 // # minutes between start and stop - val queryStep = 60 // # of seconds between each query sample "step" + val numQueries = 200 // Please make sure this number matches the OperationsPerInvocation below + val queryIntervalMin = 180 // # minutes between start and stop + val queryStep = 60 // # of seconds between each query sample "step" val spread = 1 // TODO: move setup and ingestion to another trait @@ -122,14 +122,14 @@ class QueryAndIngestBenchmark extends StrictLogging { * They are designed to match all the time series (common case) under a particular metric and job */ val queries = Seq("heap_usage{_ns=\"App-2\"}", // raw time series - """sum(rate(heap_usage{_ns="App-2"}[5m]))""", + """sum(rate(heap_usage{_ns="App-2"}[1m]))""", """quantile(0.75, heap_usage{_ns="App-2"})""", - """sum_over_time(heap_usage{_ns="App-2"}[5m])""") + """sum_over_time(heap_usage{_ns="App-2"}[1m])""") val queryTime = startTime + (5 * 60 * 1000) // 5 minutes from start until 60 minutes from start val qParams = TimeStepParams(queryTime/1000, queryStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams) } val queryCommands = logicalPlans.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 20000)) + LogicalPlan2Query(dataset.ref, plan, QueryOptions(1, 1000000)) } private var testProducingFut: Option[Future[Unit]] = None @@ -156,7 +156,7 @@ class QueryAndIngestBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @OutputTimeUnit(TimeUnit.SECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(200) def parallelQueries(): Unit = { val futures = (0 until numQueries).map { n => val f = asyncAsk(coordinator, queryCommands(n % queryCommands.length)) diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 332a4d7f4b..29e96190e0 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -5,6 +5,7 @@ import java.nio.ByteBuffer import scala.collection.mutable import com.tdunning.math.stats.{ArrayDigest, TDigest} +import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable import filodb.core.binaryrecord2.RecordBuilder @@ -111,7 +112,7 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, * * This singleton is the facade for the above operations. */ -object RangeVectorAggregator { +object RangeVectorAggregator extends StrictLogging { /** * This method is the facade for map and reduce steps of the aggregation. @@ -146,6 +147,7 @@ object RangeVectorAggregator { rowAgg: RowAggregator, skipMapPhase: Boolean, grouping: RangeVector => RangeVectorKey): Map[RangeVectorKey, Iterator[rowAgg.AggHolderType]] = { + logger.trace(s"mapReduceInternal on ${rvs.size} RangeVectors...") var acc = rowAgg.zero val mapInto = rowAgg.newRowToMapInto rvs.groupBy(grouping).mapValues { rvs => diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index fe4e1be5df..0d4d7ec6b0 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -98,7 +98,10 @@ trait ExecPlan extends QueryCommand { queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = { - try { + // NOTE: we launch the preparatory steps as a Task too. This is important because scanPartitions, + // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and + // we don't want these to happen in a single thread. + Task { qLogger.debug(s"queryId: ${id} Started ExecPlan ${getClass.getSimpleName} with $args") val res = doExecute(source, dataset, queryConfig) val schema = schemaOfDoExecute(dataset) @@ -148,9 +151,10 @@ trait ExecPlan extends QueryCommand { qLogger.error(s"queryId: ${id} Exception during execution of query: ${printTree(false)}", ex) QueryError(id, ex) } - } catch { case NonFatal(ex) => + }.flatten + .onErrorRecover { case NonFatal(ex) => qLogger.error(s"queryId: ${id} Exception during orchestration of query: ${printTree(false)}", ex) - Task(QueryError(id, ex)) + QueryError(id, ex) } } diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index 9a382d1491..de0c9cae83 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -11,6 +11,7 @@ import filodb.core.store.{ChunkSetInfo, WindowedChunkIterator} import filodb.memory.format.{vectors => bv, RowReader} import filodb.query.{BadQueryException, Query, QueryConfig, RangeFunctionId} import filodb.query.exec.rangefn.{ChunkedRangeFunction, RangeFunction, Window} +import filodb.query.Query.qLogger import filodb.query.util.IndexedArrayQueue /** @@ -62,6 +63,7 @@ final case class PeriodicSamplesMapper(start: Long, } case c: ChunkedRangeFunction[_] => source.map { rv => + qLogger.trace(s"Creating ChunkedWindowIterator for rv=${rv.key}, step=$step windowLength=$windowLength") IteratorBackedRangeVector(rv.key, new ChunkedWindowIteratorD(rv.asInstanceOf[RawDataRangeVector], start, step, end, windowLength, rangeFuncGen().asChunkedD, queryConfig)) @@ -142,7 +144,7 @@ extends Iterator[R] with StrictLogging { case e: Exception => val timestampVector = nextInfo.vectorPtr(rv.timestampColID) val tsReader = bv.LongBinaryVector(timestampVector) - logger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + + qLogger.error(s"addChunks Exception: info.numRows=${nextInfo.numRows} " + s"info.endTime=${nextInfo.endTime} curWindowEnd=${wit.curWindowEnd} " + s"tsReader=$tsReader timestampVectorLength=${tsReader.length(timestampVector)}") throw e From b8e3d51225b59d7b69ab0d58fa1a076f87eaf16a Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Sun, 31 Mar 2019 12:25:02 -0700 Subject: [PATCH 02/74] bug(coordinator): Use a thread-safe random numnber generator for dispatcher selection. (#303) --- .../filodb.coordinator/queryengine2/QueryEngine.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 5b5ffa1a99..d003c9eda7 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -1,6 +1,7 @@ package filodb.coordinator.queryengine2 -import java.util.{SplittableRandom, UUID} +import java.util.UUID +import java.util.concurrent.ThreadLocalRandom import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration @@ -362,10 +363,7 @@ class QueryEngine(dataset: Dataset, val childTargets = children.map(_.dispatcher) // Above list can contain duplicate dispatchers, and we don't make them distinct. // Those with more shards must be weighed higher - childTargets.iterator.drop(QueryEngine.random.nextInt(childTargets.size)).next + val rnd = ThreadLocalRandom.current() + childTargets.iterator.drop(rnd.nextInt(childTargets.size)).next } } - -object QueryEngine { - val random = new SplittableRandom() -} From c7fc73f8b24f6524bc4e7e27c1a6040ea84b7bfa Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 1 Apr 2019 17:07:08 -0700 Subject: [PATCH 03/74] feat(gateway): Add --gen-prom-data and --gen-hist-data to dev gateway; multiple schema support (#279) --- README.md | 9 +- conf/histogram-dev-source.conf | 57 ++++++ .../scala/filodb.core/metadata/Dataset.scala | 3 +- dev-gateway.sh | 8 +- .../scala/filodb/gateway/GatewayServer.scala | 82 +++++++-- .../gateway/conversion/InputRecord.scala | 33 +++- .../timeseries/TestTimeseriesProducer.scala | 164 ++++++++++-------- .../filodb/http/ClusterApiRouteSpec.scala | 23 +-- .../filodb.jmh/PartKeyIndexBenchmark.scala | 12 +- .../filodb.jmh/QueryAndIngestBenchmark.scala | 7 +- .../scala/filodb/kafka/TestConsumer.scala | 31 ++-- project/FiloBuild.scala | 2 +- 12 files changed, 290 insertions(+), 141 deletions(-) create mode 100644 conf/histogram-dev-source.conf diff --git a/README.md b/README.md index 1ed775acfb..f74c72f070 100644 --- a/README.md +++ b/README.md @@ -202,10 +202,10 @@ You can also check the server logs at `logs/filodb-server-N.log`. Now run the time series generator. This will ingest 20 time series (the default) with 100 samples each into the Kafka topic with current timestamps. The required argument is the path to the source config. Use `--help` for all the options. ``` -java -cp gateway/target/scala-2.11/gateway-*-SNAPSHOT filodb.timeseries.TestTimeseriesProducer -c conf/timeseries-dev-source.conf +./dev-gateway.sh --gen-prom-data conf/timeseries-dev-source.conf ``` -NOTE: The `TestTimeseriesProducer` logs to logs/gateway-server.log. +NOTE: Check logs/gateway-server.log for logs. At this point, you should be able to confirm such a message in the server logs: `KAMON counter name=memstore-rows-ingested count=4999` @@ -220,8 +220,9 @@ You can also look at Cassandra to check for persisted data. Look at the tables i If the above does not work, try the following: 1) Delete the Kafka topic and re-create it. Note that Kafka topic deletion might not happen until the server is stopped and restarted +1a) Restart Kafka, this is sometimes necessary. 2) `./filodb-dev-stop.sh` and restart filodb instances like above -3) Re-run the `TestTimeseriesProducer`. You can check consumption via running the `TestConsumer`, like this: `java -Xmx4G -cp standalone/target/scala-2.11/standalone-assembly-0.8-SNAPSHOT.jar filodb.kafka.TestConsumer conf/timeseries-dev-source.conf`. Also, the `memstore_rows_ingested` metric which is logged to `logs/filodb-server-N.log` should become nonzero. +3) Re-run `./dev-gateway.sh --gen-prom-data`. You can check consumption via running the `TestConsumer`, like this: `java -Xmx4G -Dconfig.file=conf/timeseries-filodb-server.conf -cp standalone/target/scala-2.11/standalone-assembly-0.8-SNAPSHOT.jar filodb.kafka.TestConsumer conf/timeseries-dev-source.conf`. Also, the `memstore_rows_ingested` metric which is logged to `logs/filodb-server-N.log` should become nonzero. To stop the dev server. Note that this will stop all the FiloDB servers if multiple are running. ``` @@ -308,7 +309,7 @@ Now if you curl the cluster status you should see 128 shards which are slowly tu Generate records: ``` -java -cp gateway/target/scala-2.11/gateway-*.telemetry-SNAPSHOT filodb.timeseries.TestTimeseriesProducer -c conf/timeseries-128shards-source.conf -p 5000 +./dev-gateway.sh --gen-prom-data -p 5000 conf/timeseries-128shards-source.conf ``` ## Understanding the FiloDB Data Model diff --git a/conf/histogram-dev-source.conf b/conf/histogram-dev-source.conf new file mode 100644 index 0000000000..51b7c6e39c --- /dev/null +++ b/conf/histogram-dev-source.conf @@ -0,0 +1,57 @@ + dataset = "histogram" + num-shards = 4 + min-num-nodes = 2 + sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" + + sourceconfig { + # Required FiloDB configurations + filo-topic-name = "histogram-dev" + + # Standard kafka configurations, e.g. + # This accepts both the standard kafka value of a comma-separated + # string and a Typesafe list of String values + # EXCEPT: do not populate value.deserializer, as the Kafka format is fixed in FiloDB to be messages of RecordContainer's + bootstrap.servers = "localhost:9092" + group.id = "filo-db-histogram-ingestion" + + # Values controlling in-memory store chunking, flushing, etc. + store { + # Interval it takes to flush ALL time series in a shard. This time is further divided by groups-per-shard + flush-interval = 1h + + # TTL for on-disk / C* data. Data older than this may be purged. + disk-time-to-live = 24 hours + + # amount of time paged chunks should be retained in memory. + # We need to have a minimum of x hours free blocks or else init won't work. + demand-paged-chunk-retention-period = 12 hours + + max-chunks-size = 400 + + # Write buffer size, in bytes, for blob columns (histograms, UTF8Strings). Since these are variable data types, + # we need a maximum size, not a maximum number of items. + max-blob-buffer-size = 15000 + + # Number of bytes of offheap mem to allocate to chunk storage in each shard. Ex. 1000MB, 1G, 2GB + # Assume 5 bytes per sample, should be roughly equal to (# samples per time series) * (# time series) + shard-mem-size = 512MB + + # Number of bytes of offheap mem to allocate to write buffers in each shard. Ex. 1000MB, 1G, 2GB + # Scales with the number of time series a shard should hold + ingestion-buffer-mem-size = 50MB + + # Number of time series to evict at a time. + # num-partitions-to-evict = 1000 + + # Number of subgroups within each shard. Persistence to a ChunkSink occurs one subgroup at a time, as does + # recovery from failure. This many batches of flushes must occur to cover persistence of every partition + groups-per-shard = 20 + + # Use a "MultiPartitionScan" or Cassandra MULTIGET for on-demand paging. Might improve performance. + multi-partition-odp = false + } + downsample { + # can be disabled by setting this flag to false + enabled = false + } + } \ No newline at end of file diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index b7725dfc69..decc5629d2 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -12,7 +12,7 @@ import filodb.core.downsample.ChunkDownsampler import filodb.core.query.ColumnInfo import filodb.core.store.ChunkSetInfo import filodb.memory.{BinaryRegion, MemFactory} -import filodb.memory.format.{BinaryVector, RowReader, TypedIterator} +import filodb.memory.format.{BinaryVector, RowReader, TypedIterator, ZeroCopyUTF8String => ZCUTF8} /** * A dataset describes the schema (column name & type) and distribution for a stream/set of data. @@ -169,6 +169,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], val nonMetricShardColumns = shardKeyColumns.filterNot(_ == metricColumn).sorted val nonMetricShardKeyBytes = nonMetricShardColumns.map(_.getBytes).toArray + val nonMetricShardKeyUTF8 = nonMetricShardColumns.map(ZCUTF8.apply).toArray val nonMetricShardKeyHash = nonMetricShardKeyBytes.map(BinaryRegion.hash32) val ignorePartKeyHashTags = ignoreTagsOnPartitionKeyHash.toSet diff --git a/dev-gateway.sh b/dev-gateway.sh index f4d4d45764..bfa98bd9f4 100755 --- a/dev-gateway.sh +++ b/dev-gateway.sh @@ -1,2 +1,8 @@ #!/usr/bin/env bash -java -cp gateway/target/scala-2.11/gateway-* filodb.gateway.GatewayServer conf/timeseries-dev-source.conf & \ No newline at end of file +# +# Starts a local Gateway for ingesting data into FiloDB (run with no options) +# Type --help to see options - options include generating random test data and exiting. +args=${@:-"conf/timeseries-dev-source.conf"} +java -Dconfig.file=conf/timeseries-filodb-server.conf \ + -Dkamon.prometheus.embedded-server.port=9097 \ + -cp gateway/target/scala-2.11/gateway-* filodb.gateway.GatewayServer $args & \ No newline at end of file diff --git a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala index e730b9ddd8..49e5f6712e 100644 --- a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala +++ b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala @@ -4,8 +4,8 @@ import java.net.InetSocketAddress import java.nio.charset.Charset import java.util.concurrent.Executors +import scala.concurrent.{Await, Future} import scala.concurrent.duration._ -import scala.concurrent.Future import scala.util.control.NonFatal import com.typesafe.config.{Config, ConfigFactory} @@ -25,25 +25,42 @@ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.ssl.SslContext import org.jboss.netty.handler.ssl.util.SelfSignedCertificate import org.jctools.queues.MpscGrowableArrayQueue +import org.rogach.scallop._ -import filodb.coordinator.{GlobalConfig, ShardMapper} +import filodb.coordinator.{FilodbSettings, GlobalConfig, ShardMapper, StoreFactory} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.gateway.conversion._ import filodb.memory.MemFactory -import filodb.prometheus.FormatConversion +import filodb.timeseries.TestTimeseriesProducer /** * Gateway server to ingest source streams of data, shard, batch, and write output to Kafka * built using high performance Netty TCP code * - * It takes exactly one arg: the source config file which contains # Kafka partitions/shards and other config - * Also pass in -Dconfig.file=.... as usual + * It usually takes one arg: the source config file which contains # Kafka partitions/shards and other config + * Also pass in -Dconfig.file=.... as usual, with a config that points to the dataset metadata. + * For local setups, simply run `./dev-gateway.sh`. + * For help pass in `--help`. + * + * NOTE: set `kamon.prometheus.embedded-server.port` to avoid conflicting with FiloDB itself. + * + * There are options that can be used to generate test data, such as `--gen-hist-data`. The -n and -p options can + * also be used together to control the # of samples per series and # of time series. + * To generate Histogram schema test data, one must create the following dataset: + * ./filo-cli -Dconfig.file=conf/timeseries-filodb-server.conf --command create --dataset histogram \ + * --dataColumns timestamp:ts,sum:long,count:long,h:hist --partitionColumns metric:string,tags:map \ + * --shardKeyColumns metric --metricColumn metric + * create a Kafka topic: + * kafka-topics --create --zookeeper localhost:2181 --replication-factor 1 --partitions 4 --topic histogram-dev + * and use the `conf/histogram-dev-source.conf` config file. + * Oh, and you have to observe on shards 1 and 3. */ object GatewayServer extends StrictLogging { // Get global configuration using universal FiloDB/Akka-based config val config = GlobalConfig.systemConfig + val storeFactory = StoreFactory(new FilodbSettings(config), Scheduler.io()) // ==== Metrics ==== val numInfluxMessages = Kamon.counter("num-influx-messages") @@ -52,21 +69,29 @@ object GatewayServer extends StrictLogging { val numContainersSent = Kamon.counter("num-containers-sent") val containersSize = Kamon.histogram("containers-size-bytes") + // Most options are for generating test data + class GatewayOptions(args: Seq[String]) extends ScallopConf(args) { + val samplesPerSeries = opt[Int](short = 'n', default = Some(100), + descr="# of samples per time series") + val numSeries = opt[Int](short='p', default = Some(20), descr="# of total time series") + val sourceConfigPath = trailArg[String](descr="Path to source config, eg conf/timeseries-dev-source.conf") + val genHistData = toggle(noshort=true, descrYes="Generate histogram-schema test data and exit") + val genPromData = toggle(noshort=true, descrYes="Generate Prometheus-schema test data and exit") + verify() + } + + //scalastyle:off method.length def main(args: Array[String]): Unit = { Kamon.loadReportersFromConfig() + val userOpts = new GatewayOptions(args) + val numSamples = userOpts.samplesPerSeries() * userOpts.numSeries() + val numSeries = userOpts.numSeries() - if (args.length < 1) { - //scalastyle:off - println("Arguments: [path/to/source-config.conf]") - //scalastyle:on - sys.exit(1) - } - - val sourceConfig = ConfigFactory.parseFile(new java.io.File(args.head)) + val sourceConfig = ConfigFactory.parseFile(new java.io.File(userOpts.sourceConfigPath())) val numShards = sourceConfig.getInt("num-shards") - // TODO: get the dataset from source config and read the definition from Metastore - val dataset = FormatConversion.dataset + val datasetStr = sourceConfig.getString("dataset") + val dataset = Await.result(storeFactory.metaStore.getDataset(datasetStr), 30.seconds) // NOTE: the spread MUST match the default spread used in the HTTP module for consistency between querying // and ingestion sharding @@ -97,8 +122,33 @@ object GatewayServer extends StrictLogging { // TODO: allow configurable sinks, maybe multiple sinks for say writing to multiple Kafka clusters/DCs setupKafkaProducer(sourceConfig, containerStream) - setupTCPService(config, calcShardAndQueueHandler) + + val genHist = userOpts.genHistData.getOrElse(false) + val genProm = userOpts.genPromData.getOrElse(false) + if (genHist || genProm) { + val startTime = System.currentTimeMillis + logger.info(s"Generating $numSamples samples starting at $startTime....") + + val stream = if (genHist) TestTimeseriesProducer.genHistogramData(startTime, dataset, numSeries) + else TestTimeseriesProducer.timeSeriesData(startTime, numSeries) + + stream.take(numSamples).foreach { rec => + val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) + if (!shardQueues(shard).offer(rec)) { + // Prioritize recent data. This means dropping messages when full, so new data may have a chance. + logger.warn(s"Queue for shard=$shard is full. Dropping data.") + numDroppedMessages.increment + } + } + Thread sleep 10000 + TestTimeseriesProducer.logQueryHelp(numSamples, numSeries, startTime) + logger.info(s"Waited for containers to be sent, exiting...") + sys.exit(0) + } else { + setupTCPService(config, calcShardAndQueueHandler) + } } + //scalastyle:on method.length def setupTCPService(config: Config, handler: ChannelBuffer => Unit): Unit = { val influxPort = config.getInt("gateway.influx-port") diff --git a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala index c6107ca38d..b31afdcaf8 100644 --- a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala +++ b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala @@ -7,6 +7,7 @@ import scalaxy.loops._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset +import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String => ZCUTF8} /** * An InputRecord represents one "record" of timeseries data for input to FiloDB system. @@ -124,4 +125,34 @@ object PrometheusInputRecord { } tags ++ extraTags } -} \ No newline at end of file +} + +/** + * A generic InputRecord that can serve different data schemas, so long as the partition key consists of: + * - a single metric StringColumn + * - a MapColumn of tags + * + * Can be used to adapt custom dataset/schemas for input into FiloDB using the gateway. + * Not going to be the fastest InputRecord but extremely flexible. + * + * @param values the data column values, first one is probably timestamp + */ +class MetricTagInputRecord(values: Seq[Any], + metric: String, + tags: Map[ZCUTF8, ZCUTF8], + dataset: Dataset) extends InputRecord { + final def shardKeyHash: Int = RecordBuilder.shardKeyHash(nonMetricShardValues, metric) + // NOTE: this is probably not very performant right now. + final def partitionKeyHash: Int = tags.hashCode + + val nonMetricShardValues: Seq[String] = + dataset.options.nonMetricShardKeyUTF8.flatMap(tags.get).map(_.toString).toSeq + final def getMetric: String = metric + + def addToBuilder(builder: RecordBuilder): Unit = { + require(builder.schema == dataset.ingestionSchema) + + val reader = SeqRowReader(values :+ metric :+ tags) + builder.addFromReader(reader) + } +} diff --git a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala index ddcfba0c7d..32ce1f7a1c 100644 --- a/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala +++ b/gateway/src/main/scala/filodb/timeseries/TestTimeseriesProducer.scala @@ -3,67 +3,33 @@ package filodb.timeseries import java.net.URLEncoder import java.nio.charset.StandardCharsets -import scala.concurrent.{Await, Future} +import scala.concurrent.Future import scala.concurrent.duration._ import scala.util.Random -import com.typesafe.config.{Config, ConfigFactory} +import com.typesafe.config.Config import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable -import org.rogach.scallop._ import filodb.coordinator.{GlobalConfig, ShardMapper} -import filodb.core.metadata.Dataset +import filodb.core.metadata.{Column, Dataset} import filodb.gateway.GatewayServer -import filodb.gateway.conversion.PrometheusInputRecord +import filodb.gateway.conversion.{InputRecord, MetricTagInputRecord, PrometheusInputRecord} +import filodb.memory.format.{vectors => bv, ZeroCopyUTF8String => ZCUTF8} import filodb.prometheus.FormatConversion -sealed trait DataOrCommand -final case class DataSample(tags: Map[String, String], - metric: String, - timestamp: Long, - value: Double) extends DataOrCommand -case object FlushCommand extends DataOrCommand - /** - * Simple driver to produce time series data into local kafka similar. Data format is similar to - * prometheus metric sample. - * This is for development testing purposes only. TODO: Later evolve this to accept prometheus formats. - * - * Run as `java -cp classpath filodb.timeseries.TestTimeseriesProducer --help` - * + * Utilities to produce time series data into local Kafka for development testing. + * Please see GatewayServer for the app to run, or README for docs. */ object TestTimeseriesProducer extends StrictLogging { val dataset = FormatConversion.dataset - class ProducerOptions(args: Seq[String]) extends ScallopConf(args) { - val samplesPerSeries = opt[Int](short = 'n', default = Some(100), - descr="# of samples per time series") - val startMinutesAgo = opt[Int](short='t') - val numSeries = opt[Int](short='p', default = Some(20), descr="# of total time series") - val sourceConfigPath = opt[String](required = true, short = 'c', - descr="Path to source conf file eg conf/timeseries-dev-source.conf") - verify() - } - val oneBitMask = 0x1 val twoBitMask = 0x3 val rand = Random // start from a random day in the last 5 years - def main(args: Array[String]): Unit = { - val conf = new ProducerOptions(args) - val sourceConfig = ConfigFactory.parseFile(new java.io.File(conf.sourceConfigPath())) - val numSamples = conf.samplesPerSeries() * conf.numSeries() - val numTimeSeries = conf.numSeries() - // to get default start time, look at numSamples and calculate a startTime that ends generation at current time - val startMinutesAgo = conf.startMinutesAgo.toOption - .getOrElse((numSamples.toDouble / numTimeSeries / 6).ceil.toInt ) // at 6 samples per minute - - Await.result(produceMetrics(sourceConfig, numSamples, numTimeSeries, startMinutesAgo), 1.hour) - sys.exit(0) - } - import scala.concurrent.ExecutionContext.Implicits.global /** @@ -89,36 +55,36 @@ object TestTimeseriesProducer extends StrictLogging { s"from about ${(System.currentTimeMillis() - startTime) / 1000 / 60} minutes ago") producingFut.map { _ => - // Give enough time for the last containers to be sent off successfully to Kafka - Thread sleep 2000 - logger.info(s"Finished producing $numSamples messages into topic $topicName with timestamps " + - s"from about ${(System.currentTimeMillis() - startTime) / 1000 / 60} minutes ago at $startTime") - val startQuery = startTime / 1000 - val endQuery = startQuery + 300 - val query = - s"""./filo-cli '-Dakka.remote.netty.tcp.hostname=127.0.0.1' --host 127.0.0.1 --dataset prometheus """ + - s"""--promql 'heap_usage{dc="DC0",_ns="App-0"}' --start $startQuery --end $endQuery --limit 15""" - logger.info(s"Periodic Samples CLI Query : \n$query") - - val periodQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}""", StandardCharsets.UTF_8.toString) - val periodicSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query_range?" + - s"query=$periodQuery&start=$startQuery&end=$endQuery&step=15" - logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") - - val rawQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", - StandardCharsets.UTF_8.toString) - val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$rawQuery&time=$endQuery" - logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") - - val downsampleSumQuery = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", - StandardCharsets.UTF_8.toString) - val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + - s"query=$downsampleSumQuery&time=$endQuery" - logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") - + logQueryHelp(numSamples, numTimeSeries, startTime) } } + + def logQueryHelp(numSamples: Int, numTimeSeries: Int, startTime: Long): Unit = { + val samplesDuration = (numSamples.toDouble / numTimeSeries / 6).ceil.toInt * 60L * 1000L + + logger.info(s"Finished producing $numSamples records for ${samplesDuration / 1000} seconds") + val startQuery = startTime / 1000 + val endQuery = startQuery + 300 + val query = + s"""./filo-cli '-Dakka.remote.netty.tcp.hostname=127.0.0.1' --host 127.0.0.1 --dataset prometheus """ + + s"""--promql 'heap_usage{dc="DC0",_ns="App-0"}' --start $startQuery --end $endQuery --limit 15""" + logger.info(s"Periodic Samples CLI Query : \n$query") + + val q = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0"}[2m]""", StandardCharsets.UTF_8.toString) + val periodicSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query_range?" + + s"query=$q&start=$startQuery&end=$endQuery&step=15" + logger.info(s"Periodic Samples query URL: \n$periodicSamplesUrl") + + val q2 = URLEncoder.encode("""heap_usage{dc="DC0",_ns="App-0",__col__="sum"}[2m]""", + StandardCharsets.UTF_8.toString) + val rawSamplesUrl = s"http://localhost:8080/promql/prometheus/api/v1/query?query=$q2&time=$endQuery" + logger.info(s"Raw Samples query URL: \n$rawSamplesUrl") + val downsampledSamplesUrl = s"http://localhost:8080/promql/prometheus_ds_1m/api/v1/query?" + + s"query=$q2&time=$endQuery" + logger.info(s"Downsampled Samples query URL: \n$downsampledSamplesUrl") + } + def metricsToContainerStream(startTime: Long, numShards: Int, numTimeSeries: Int, @@ -129,10 +95,9 @@ object TestTimeseriesProducer extends StrictLogging { val (shardQueues, containerStream) = GatewayServer.shardingPipeline(GlobalConfig.systemConfig, numShards, dataset) val producingFut = Future { - timeSeriesData(startTime, numShards, numTimeSeries) + timeSeriesData(startTime, numTimeSeries) .take(numSamples) - .foreach { sample => - val rec = PrometheusInputRecord(sample.tags, sample.metric, dataset, sample.timestamp, sample.value) + .foreach { rec => val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) while (!shardQueues(shard).offer(rec)) { Thread sleep 50 } } @@ -141,14 +106,13 @@ object TestTimeseriesProducer extends StrictLogging { } /** - * Generate time series data. + * Generate Prometheus-schema time series data. * * @param startTime Start time stamp - * @param numShards the number of shards or Kafka partitions * @param numTimeSeries number of instances or time series * @return stream of a 2-tuple (kafkaParitionId , sampleData) */ - def timeSeriesData(startTime: Long, numShards: Int, numTimeSeries: Int = 16): Stream[DataSample] = { + def timeSeriesData(startTime: Long, numTimeSeries: Int = 16): Stream[InputRecord] = { // TODO For now, generating a (sinusoidal + gaussian) time series. Other generators more // closer to real world data can be added later. Stream.from(0).map { n => @@ -165,7 +129,57 @@ object TestTimeseriesProducer extends StrictLogging { "partition" -> s"partition-$partition", "host" -> s"H$host", "instance" -> s"Instance-$instance") - DataSample(tags, "heap_usage", timestamp, value) + + PrometheusInputRecord(tags, "heap_usage", dataset, timestamp, value) + } + } + + import ZCUTF8._ + import Column.ColumnType._ + + val dcUTF8 = "dc".utf8 + val nsUTF8 = "_ns".utf8 + val partUTF8 = "partition".utf8 + val hostUTF8 = "host".utf8 + val instUTF8 = "instance".utf8 + + /** + * Generate a stream of random Histogram data, with the metric name "http_request_latency" + * Schema: (timestamp:ts, sum:long, count:long, h:hist) for data, plus (metric:string, tags:map) + * The dataset must match the above schema + */ + def genHistogramData(startTime: Long, dataset: Dataset, numTimeSeries: Int = 16): Stream[InputRecord] = { + require(dataset.dataColumns.map(_.columnType) == Seq(TimestampColumn, LongColumn, LongColumn, HistogramColumn)) + val numBuckets = 10 + + val histBucketScheme = bv.GeometricBuckets(2.0, 3.0, numBuckets) + val buckets = new Array[Long](numBuckets) + def updateBuckets(bucketNo: Int): Unit = { + for { b <- bucketNo until numBuckets } { + buckets(b) += 1 + } + } + + Stream.from(0).map { n => + val instance = n % numTimeSeries + val dc = instance & oneBitMask + val partition = (instance >> 1) & twoBitMask + val app = (instance >> 3) & twoBitMask + val host = (instance >> 4) & twoBitMask + val timestamp = startTime + (n.toLong / numTimeSeries) * 10000 // generate 1 sample every 10s for each instance + + updateBuckets(n % numBuckets) + val hist = bv.LongHistogram(histBucketScheme, buckets.map(x => x)) + val count = util.Random.nextInt(100).toLong + val sum = buckets.sum + + val tags = Map(dcUTF8 -> s"DC$dc".utf8, + nsUTF8 -> s"App-$app".utf8, + partUTF8 -> s"partition-$partition".utf8, + hostUTF8 -> s"H$host".utf8, + instUTF8 -> s"Instance-$instance".utf8) + + new MetricTagInputRecord(Seq(timestamp, sum, count, hist), "http_request_latency", tags, dataset) } } } diff --git a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala index f0e054b0e7..31d0efc46d 100644 --- a/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/ClusterApiRouteSpec.scala @@ -1,7 +1,5 @@ package filodb.http -import scala.concurrent.duration._ - import akka.actor.ActorSystem import akka.http.scaladsl.model.headers.RawHeader import akka.http.scaladsl.model.{StatusCodes, ContentTypes} @@ -83,21 +81,9 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest it("should return shard status after dataset is setup") { setupDataset() - // Repeatedly query cluster status until we know it is OK - var statuses: Seq[ShardStatus] = Nil - do { - probe.send(clusterProxy, NodeClusterActor.GetShardMap(dataset6.ref)) - Thread sleep 500 - statuses = probe.expectMsgPF(3.seconds) { - case CurrentShardSnapshot(_, mapper) => mapper.statuses - } - println(s"Current statuses = $statuses") - info(s"Current statuses = $statuses") - if (statuses.exists(_ == ShardStatusError)) { - info(s"ERROR in status, breaking") - throw new RuntimeException(s"Got error in statuses $statuses") - } - } while (statuses.take(2) != Seq(ShardStatusActive, ShardStatusActive)) + + // Give the coordinator nodes some time to get started + Thread sleep 1000 Get(s"/api/v1/cluster/${dataset6.ref}/status") ~> clusterRoute ~> check { handled shouldBe true @@ -106,7 +92,8 @@ class ClusterApiRouteSpec extends FunSpec with ScalatestRouteTest with AsyncTest val resp = responseAs[HttpList[HttpShardState]] resp.status shouldEqual "success" resp.data should have length 4 - resp.data.map(_.status).filter(_ contains "Active") should have length 2 // Two active nodes + // Exact status of assigned nodes doesn't matter much. This is an HTTP route test, not a sharding test + resp.data.map(_.status).filter(_ contains "Unassigned") should have length 2 // Two unassigned nodes } } diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index e416ce8983..f52faf427f 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -24,17 +24,9 @@ class PartKeyIndexBenchmark { val dataset = FormatConversion.dataset val partKeyIndex = new PartKeyLuceneIndex(dataset, 0, TestData.storeConf) val numSeries = 1000000 - val partKeyData = TestTimeseriesProducer.timeSeriesData(0, 1, numSeries) take numSeries + val partKeyData = TestTimeseriesProducer.timeSeriesData(0, numSeries) take numSeries val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, dataset.partKeySchema) - partKeyData.foreach { s => - partKeyBuilder.startNewRecord() - partKeyBuilder.startMap() - s.tags.foreach { case (k, v) => - partKeyBuilder.addMapKeyValue(k.getBytes(), v.getBytes()) - } - partKeyBuilder.endMap() - partKeyBuilder.endRecord() - } + partKeyData.foreach(_.addToBuilder(partKeyBuilder)) var partId = 1 val now = System.currentTimeMillis() diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 9d7111cb40..8a0df92511 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -100,12 +100,11 @@ class QueryAndIngestBenchmark extends StrictLogging { val shards = (0 until numShards).map { s => memstore.getShardE(dataset.ref, s) } private def ingestSamples(noSamples: Int): Future[Unit] = Future { - TestTimeseriesProducer.timeSeriesData(startTime, numShards, numSeries) + TestTimeseriesProducer.timeSeriesData(startTime, numSeries) .take(noSamples * numSeries) - .foreach { sample => - val rec = PrometheusInputRecord(sample.tags, sample.metric, dataset, sample.timestamp, sample.value) + .foreach { rec => // we shouldn't ingest samples for same timestamps repeatedly. This will also result in out-of-order samples. - startTime = Math.max(startTime, sample.timestamp + 10000) + startTime = Math.max(startTime, rec.asInstanceOf[PrometheusInputRecord].timestamp + 10000) val shard = shardMapper.ingestionShard(rec.shardKeyHash, rec.partitionKeyHash, spread) while (!shardQueues(shard).offer(rec)) { Thread sleep 50 } } diff --git a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala index 9daa4a4d49..4e814abf1d 100644 --- a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala +++ b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala @@ -4,33 +4,44 @@ import scala.concurrent.Await import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory +import monix.execution.Scheduler +import filodb.coordinator.{FilodbSettings, IngestionStreamFactory, StoreFactory} import filodb.core.memstore.SomeData -import filodb.core.metadata.Dataset +import filodb.core.store.IngestionConfig /** - * A simple app which uses the KafkaIngestionStream plus a sourceconfig of your choice to test reading - * data from Kafka and test reading from certain offsets. + * A simple app which uses a sourceconfig of your choice to test reading + * data from Kafka (or whatever configured source factory) and test reading from certain offsets. + * It reads dataset definition from MetaStore, so please pass the server.conf with Cassandra/metastore details. * - * To launch: java -Xmx4G -cp /standalone-assembly-0.7.0.jar filodb.kafka.TestConsumer \ - * my-kafka-sourceconfig.conf + * To launch: java -Xmx4G -Dconfig.file=conf/timeseries-filodb-server.conf \ + * -cp /standalone-assembly-0.7.0.jar filodb.kafka.TestConsumer \ + * my-kafka-sourceconfig.conf * It will read 10 records and then quit, printing out the offsets of each record. * Optional: pass in a second arg which is the offset to seek to. */ object TestConsumer extends App { + val settings = new FilodbSettings() + val storeFactory = StoreFactory(settings, Scheduler.io()) + val sourceConfPath = args(0) val offsetOpt = args.drop(1).headOption.map(_.toLong) + val shard = if (args.length > 1) args(1).toInt else 0 val sourceConf = ConfigFactory.parseFile(new java.io.File(sourceConfPath)) //scalastyle:off - println(s"TestConsumer starting with config $sourceConf\nand offset $offsetOpt") + println(s"TestConsumer starting with shard $shard, config $sourceConf\nand offset $offsetOpt") import monix.execution.Scheduler.Implicits.global - // For now, hard code dataset to a Prometheus like dataset - // TODO: allow specification of dataset, then load from the MetaStore - val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:long", "value:double")) - val stream = new KafkaIngestionStream(sourceConf, dataset, 0, offsetOpt) + val ingestConf = IngestionConfig(sourceConf, classOf[KafkaIngestionStreamFactory].getClass.getName).get + val dataset = Await.result(storeFactory.metaStore.getDataset(ingestConf.ref), 30.seconds) + + val ctor = Class.forName(ingestConf.streamFactoryClass).getConstructors.head + val streamFactory = ctor.newInstance().asInstanceOf[IngestionStreamFactory] + + val stream = streamFactory.create(sourceConf, dataset, shard, offsetOpt) val fut = stream.get.take(10) .foreach { case SomeData(container, offset) => println(s"\n----- Offset $offset -----") diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index b6a1607cdf..8ff4f496cb 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -152,7 +152,7 @@ object FiloBuild extends Build { .settings(libraryDependencies ++= gatewayDeps) .settings(gatewayAssemblySettings: _*) .dependsOn(coordinator % "compile->compile; test->test", - prometheus) + prometheus, cassandra) // Zookeeper pulls in slf4j-log4j12 which we DON'T want val excludeZK = ExclusionRule(organization = "org.apache.zookeeper") From 3b2359a807ac0c0a766f699f6b98552ad6e157a6 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Mon, 1 Apr 2019 22:12:14 -0700 Subject: [PATCH 04/74] feat(core): Optimize index recovery for long term retention (#296) Earlier approach of creating every TSP was ok for 3 day retention given most partitions are ingesting, but does not scale for long term retention where majority of partitions are not ingesting. This commit reduces number of entries in partitions and partSet data structures, thereby reducing memory requirements especially for long term datasets. We pay the cost of a temporary map during recovery that holds all partKeys on heap, but is discarded before ingestion starts. Also optimize the Lucene recovery by scanning through records in reverse order of buckets. This avoids the expensive Lucene upserts - adds become sufficient. --- .../memstore/TimeSeriesShard.scala | 69 +++++++++++++------ .../memstore/TimeSeriesMemStoreSpec.scala | 29 +++++--- .../scala/filodb/query/exec/ExecPlan.scala | 9 ++- 3 files changed, 76 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 75777a42ee..e32073b1b1 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -17,6 +17,7 @@ import monix.eval.Task import monix.execution.{Scheduler, UncaughtExceptionReporter} import monix.execution.atomic.AtomicBoolean import monix.reactive.Observable +import org.apache.lucene.util.BytesRef import org.jctools.maps.NonBlockingHashMapLong import scalaxy.loops._ @@ -429,16 +430,23 @@ class TimeSeriesShard(val dataset: Dataset, .withTag("dataset", dataset.name) .withTag("shard", shardNum).start() + /* We need this map to track partKey->partId because lucene index cannot be looked up + using partKey efficiently, and more importantly, it is eventually consistent. + The map and contents will be garbage collected after we are done with recovery */ + val partIdMap = debox.Map.empty[BytesRef, Int] + val earliestTimeBucket = Math.max(0, currentIndexTimeBucket - numTimeBucketsToRetain) logger.info(s"Recovering timebuckets $earliestTimeBucket until $currentIndexTimeBucket " + s"for dataset=${dataset.ref} shard=$shardNum ") - val timeBuckets = for { tb <- earliestTimeBucket until currentIndexTimeBucket } yield { + // go through the buckets in reverse order to first one wins and we need not rewrite + // entries in lucene + val timeBuckets = for { tb <- currentIndexTimeBucket until earliestTimeBucket by -1 } yield { colStore.getPartKeyTimeBucket(dataset, shardNum, tb).map { b => new IndexData(tb, b.segmentId, RecordContainer(b.segment.array())) } } val fut = Observable.flatten(timeBuckets: _*) - .foreach(tb => extractTimeBucket(tb))(ingestSched) + .foreach(tb => extractTimeBucket(tb, partIdMap))(ingestSched) .map(_ => completeIndexRecovery()) fut.onComplete(_ => tracer.finish()) fut @@ -450,7 +458,8 @@ class TimeSeriesShard(val dataset: Dataset, logger.info(s"Bootstrapped index for dataset=${dataset.ref} shard=$shardNum") } - private[memstore] def extractTimeBucket(segment: IndexData): Unit = { + // scalastyle:off method.length + private[memstore] def extractTimeBucket(segment: IndexData, partIdMap: debox.Map[BytesRef, Int]): Unit = { var numRecordsProcessed = 0 segment.records.iterate(indexTimeBucketSchema).foreach { row => // read binary record and extract the indexable data fields @@ -459,44 +468,63 @@ class TimeSeriesShard(val dataset: Dataset, val partKeyBaseOnHeap = row.getBlobBase(2).asInstanceOf[Array[Byte]] val partKeyOffset = row.getBlobOffset(2) val partKeyNumBytes = row.getBlobNumBytes(2) + val partKeyBytesRef = new BytesRef(partKeyBaseOnHeap, + PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset), + partKeyNumBytes) - // look up partId in partSet if it already exists before assigning new partId. + // look up partKey in partIdMap if it already exists before assigning new partId. // We cant look it up in lucene because we havent flushed index yet - val partId = partSet.getWithPartKeyBR(partKeyBaseOnHeap, partKeyOffset) match { - case None => val group = partKeyGroup(dataset.partKeySchema, partKeyBaseOnHeap, partKeyOffset, numGroups) + if (partIdMap.get(partKeyBytesRef).isEmpty) { + val partId = if (endTime == Long.MaxValue) { + // this is an actively ingesting partition + val group = partKeyGroup(dataset.partKeySchema, partKeyBaseOnHeap, partKeyOffset, numGroups) val part = createNewPartition(partKeyBaseOnHeap, partKeyOffset, group, CREATE_NEW_PARTID, 4) // In theory, we should not get an OutOfMemPartition here since - // it should have occurred before node failed too, and with data sropped, + // it should have occurred before node failed too, and with data stopped, // index would not be updated. But if for some reason we see it, drop data if (part == OutOfMemPartition) { logger.error("Could not accommodate partKey while recovering index. " + "WriteBuffer size may not be configured correctly") - -1 + None } else { val stamp = partSetLock.writeLock() try { partSet.add(part) // createNewPartition doesn't add part to partSet + Some(part.partID) } finally { partSetLock.unlockWrite(stamp) } - part.partID } - case Some(p) => p.partID - } - if (partId != -1) { - // upsert into lucene since there can be multiple records for one partKey, and most recent wins. - partKeyIndex.upsertPartKey(partKeyBaseOnHeap, partId, startTime, endTime, - PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset))(partKeyNumBytes) - timeBucketBitmaps.get(segment.timeBucket).set(partId) - activelyIngesting.synchronized { - if (endTime == Long.MaxValue) activelyIngesting.set(partId) else activelyIngesting.clear(partId) + } else { + // partition assign a new partId to non-ingesting partition, + // but no need to create a new TSPartition heap object + val id = nextPartitionID + incrementPartitionID() + Some(id) } + + // add newly assigned partId to lucene index + partId.foreach { partId => + partIdMap(partKeyBytesRef) = partId + partKeyIndex.addPartKey(partKeyBaseOnHeap, partId, startTime, endTime, + PartKeyLuceneIndex.unsafeOffsetToBytesRefOffset(partKeyOffset))(partKeyNumBytes) + timeBucketBitmaps.get(segment.timeBucket).set(partId) + activelyIngesting.synchronized { + if (endTime == Long.MaxValue) activelyIngesting.set(partId) + else activelyIngesting.clear(partId) + } + } + } else { + // partId has already been assigned for this partKey because we previously processed a later record in time. + // Time buckets are processed in reverse order, and given last one wins and is used for index, + // we skip this record and move on. } numRecordsProcessed += 1 } shardStats.indexRecoveryNumRecordsProcessed.increment(numRecordsProcessed) - logger.info(s"Recovered partition keys from timebucket for dataset=${dataset.ref} shard=$shardNum" + - s" timebucket=${segment.timeBucket} segment=${segment.segment} numRecordsProcessed=$numRecordsProcessed") + logger.info(s"Recovered partKeys for dataset=${dataset.ref} shard=$shardNum" + + s" timebucket=${segment.timeBucket} segment=${segment.segment} numRecordsInBucket=$numRecordsProcessed" + + s" numPartsInIndex=${partIdMap.size} numIngestingParts=${partitions.size}") } def indexNames: Iterator[String] = partKeyIndex.indexNames @@ -902,7 +930,6 @@ class TimeSeriesShard(val dataset: Dataset, activelyIngesting.clear(p.partID) } else if (partFlushChunks.nonEmpty && !activelyIngesting.get(p.partID)) { // Partition started re-ingesting. - // TODO: we can do better than this for intermittent time series. Address later. updatePartEndTimeInIndex(p, Long.MaxValue) timeBucketBitmaps.get(timeBucket).set(p.partID) activelyIngesting.set(p.partID) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index c7c273bfd0..a59f323bf5 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -6,6 +6,7 @@ import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import monix.execution.ExecutionModel.BatchedExecution import monix.reactive.Observable +import org.apache.lucene.util.BytesRef import org.scalatest.{BeforeAndAfter, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} @@ -85,7 +86,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.commitIndexForTesting(dataset1.ref) val split = memStore.getScanSplits(dataset1.ref, 1).head val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(split)).map(_.getDouble(0)).sum - agg1 shouldEqual ((1 to 20).map(_.toDouble).sum) + agg1 shouldEqual (1 to 20).map(_.toDouble).sum } it("should ingest map/tags column as partition key and aggregate") { @@ -272,6 +273,9 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w tsShard.timeBucketBitmaps.keySet.asScala.toSeq.sorted shouldEqual 19.to(25) // 6 buckets retained + one for current } + /** + * Tries to write partKeys into time bucket record container and extracts them back into the shard + */ def indexRecoveryTest(dataset: Dataset, partKeys: Seq[Long]): Unit = { memStore.metastore.writeHighestIndexTimeBucket(dataset.ref, 0, 0) memStore.setup(dataset, 0, @@ -282,24 +286,33 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w partKeys.zipWithIndex.foreach { case (off, i) => timeBucketRb.startNewRecord() - timeBucketRb.addLong(i + 10) - timeBucketRb.addLong(i + 20) + timeBucketRb.addLong(i + 10) // startTime + timeBucketRb.addLong(if (i%2 == 0) Long.MaxValue else i + 20) // endTime val numBytes = BinaryRegionLarge.numBytes(UnsafeUtils.ZeroPointer, off) - timeBucketRb.addBlob(UnsafeUtils.ZeroPointer, off, numBytes + 4) + timeBucketRb.addBlob(UnsafeUtils.ZeroPointer, off, numBytes + 4) // partKey timeBucketRb.endRecord(false) } tsShard.initTimeBuckets() + val partIdMap = debox.Map.empty[BytesRef, Int] + timeBucketRb.optimalContainerBytes(true).foreach { bytes => - tsShard.extractTimeBucket(new IndexData(1, 0, RecordContainer(bytes))) + tsShard.extractTimeBucket(new IndexData(1, 0, RecordContainer(bytes)), partIdMap) } tsShard.commitPartKeyIndexBlocking() + partIdMap.size shouldEqual partKeys.size partKeys.zipWithIndex.foreach { case (off, i) => val readPartKey = tsShard.partKeyIndex.partKeyFromPartId(i).get val expectedPartKey = dataset1.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, off) - readPartKey.bytes.drop(readPartKey.offset).take(readPartKey.length) shouldEqual expectedPartKey - tsShard.partitions.get(i).partKeyBytes shouldEqual expectedPartKey - tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off).get.partID shouldEqual i + readPartKey.bytes.slice(readPartKey.offset, readPartKey.offset + readPartKey.length) shouldEqual expectedPartKey + if (i%2 == 0) { + tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off).get.partID shouldEqual i + tsShard.partitions.containsKey(i) shouldEqual true // since partition is ingesting + } + else { + tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off) shouldEqual None + tsShard.partitions.containsKey(i) shouldEqual false // since partition is not ingesting + } } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 0d4d7ec6b0..1d0eb59ac3 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -102,16 +102,17 @@ trait ExecPlan extends QueryCommand { // Lucene index lookup, and On-Demand Paging orchestration work could suck up nontrivial time and // we don't want these to happen in a single thread. Task { - qLogger.debug(s"queryId: ${id} Started ExecPlan ${getClass.getSimpleName} with $args") + qLogger.debug(s"queryId: ${id} Setting up ExecPlan ${getClass.getSimpleName} with $args") val res = doExecute(source, dataset, queryConfig) val schema = schemaOfDoExecute(dataset) val finalRes = rangeVectorTransformers.foldLeft((res, schema)) { (acc, transf) => - qLogger.debug(s"queryId: ${id} Started Transformer ${transf.getClass.getSimpleName} with ${transf.args}") + qLogger.debug(s"queryId: ${id} Setting up Transformer ${transf.getClass.getSimpleName} with ${transf.args}") (transf.apply(acc._1, queryConfig, limit, acc._2), transf.schema(dataset, acc._2)) } val recSchema = SerializableRangeVector.toSchema(finalRes._2.columns, finalRes._2.brSchemas) val builder = SerializableRangeVector.toBuilder(recSchema) var numResultSamples = 0 // BEWARE - do not modify concurrently!! + qLogger.debug(s"queryId: ${id} Materializing SRVs from iterators if necessary") finalRes._1 .map { case srv: SerializableRangeVector => @@ -261,6 +262,10 @@ abstract class NonLeafExecPlan extends ExecPlan { /** * Sub-class non-leaf nodes should provide their own implementation of how * to compose the sub-query results here. + * + * @param childResponses observable of a pair. First element of pair is the QueryResponse for + * a child ExecPlan, the second element is the index of the child plan. + * There is one response per child plan. */ protected def compose(dataset: Dataset, childResponses: Observable[(QueryResponse, Int)], From 686b37daad171e260d494c4439a48760e23ac6f8 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 3 Apr 2019 11:11:19 -0700 Subject: [PATCH 05/74] bug(coord): Remove publish snapshot for RecoveryInProgress (#305) * Don't send snapshot on RecoveryInProgress events since it results in too many events on the cluster singleton causing HTTP timeouts. * Save context.parent on IngestionActor startup and use that instead of context.parent from future thread. --- .../src/main/scala/filodb.coordinator/IngestionActor.scala | 7 ++++--- .../src/main/scala/filodb.coordinator/ShardManager.scala | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 50cc0e09d2..228225249c 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -60,6 +60,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, final val streamSubscriptions = new HashMap[Int, CancelableFuture[Unit]] final val streams = new HashMap[Int, IngestionStream] + final val nodeCoord = context.parent // Params for creating the default memStore flush scheduler private final val numGroups = storeConfig.groupsPerShard @@ -164,7 +165,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, create(shard, offset) map { ingestionStream => val stream = ingestionStream.get logger.info(s"Starting normal/active ingestion for dataset=${dataset.ref} shard=$shard at offset $offset") - statusActor ! IngestionStarted(dataset.ref, shard, context.parent) + statusActor ! IngestionStarted(dataset.ref, shard, nodeCoord) streamSubscriptions(shard) = memStore.ingestStream(dataset.ref, shard, @@ -209,7 +210,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, .withTag("shard", shard.toString) .withTag("dataset", dataset.ref.toString).start() val stream = ingestionStream.get - statusActor ! RecoveryInProgress(dataset.ref, shard, context.parent, 0) + statusActor ! RecoveryInProgress(dataset.ref, shard, nodeCoord, 0) val shardInstance = memStore.asInstanceOf[TimeSeriesMemStore].getShardE(dataset.ref, shard) val fut = memStore.recoverStream(dataset.ref, shard, stream, checkpoints, interval) @@ -218,7 +219,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, else (off - startOffset) * 100 / (endOffset - startOffset) logger.info(s"Recovery of dataset=${dataset.ref} shard=$shard at " + s"$progressPct % - offset $off (target $endOffset)") - statusActor ! RecoveryInProgress(dataset.ref, shard, context.parent, progressPct.toInt) + statusActor ! RecoveryInProgress(dataset.ref, shard, nodeCoord, progressPct.toInt) off } .until(_ >= endOffset) // TODO: move this code to TimeSeriesShard itself. Shard should control the thread diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala index b846929831..6ea9afa123 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala @@ -449,7 +449,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, // Above condition ensures that we respond to shard events only from the node shard is currently assigned to. // Needed to avoid race conditions where IngestionStopped for an old assignment comes after shard is reassigned. updateFromShardEvent(event) - publishSnapshot(event.ref) + // RecoveryInProgress status results in too many messages that really do not need a publish + if (!event.isInstanceOf[RecoveryInProgress]) publishSnapshot(event.ref) // reassign shard if IngestionError. Exclude previous node since it had error shards. event match { case _: IngestionError => From 0a2402ed72a35c84d1f27117656e660af0fd4572 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Wed, 3 Apr 2019 15:02:17 -0700 Subject: [PATCH 06/74] misc(core): Add zipkin configs to filodb-defaults (#306) --- core/src/main/resources/filodb-defaults.conf | 7 ++++++- core/src/main/scala/filodb.core/zipkin/Zipkin.scala | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 33f753c5c4..3b24108270 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -327,4 +327,9 @@ custom-downing { down-if-in-minority = true shutdown-actor-system-on-resolution = true } -} \ No newline at end of file +} + +kamon.zipkin { + max.requests = 128 + message.max.bytes = 262144 +} diff --git a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala index 3ccc721a1d..b68bf2d85b 100644 --- a/core/src/main/scala/filodb.core/zipkin/Zipkin.scala +++ b/core/src/main/scala/filodb.core/zipkin/Zipkin.scala @@ -139,8 +139,8 @@ class ZipkinReporter extends SpanReporter { OkHttpSender.newBuilder() .encoding(Encoding.JSON) .endpoint(url) - .maxRequests(if (maxRequests > 0) maxRequests else 128) - .messageMaxBytes(if (messageMaxBytes > 0) messageMaxBytes else 1024 * 256) + .maxRequests(maxRequests) + .messageMaxBytes(messageMaxBytes) .build() ) } From b2179a31cbf8628ef4be439ef1795e67b5fc7123 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 4 Apr 2019 10:05:26 -0700 Subject: [PATCH 07/74] feat(coordinator): Resync shard mapping with a full shard state command (#292) Ditch stop/start ingestion messages and rely on resync messages only. --- .../filodb.coordinator/IngestionActor.scala | 191 ++++--- .../NodeCoordinatorActor.scala | 18 +- .../filodb.coordinator/ShardManager.scala | 135 +++-- .../filodb.coordinator/ShardMapper.scala | 5 + .../filodb.coordinator/ShardStatus.scala | 28 +- .../IngestionStreamSpec.scala | 7 +- .../ReassignShardsSpec.scala | 397 +++++++++++-- .../filodb.coordinator/ShardManagerSpec.scala | 525 ++++++++++-------- 8 files changed, 830 insertions(+), 476 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala index 228225249c..37d884b83e 100644 --- a/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/IngestionActor.scala @@ -1,6 +1,6 @@ package filodb.coordinator -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.Future import scala.util.{Failure, Success, Try} import scala.util.control.NonFatal @@ -36,7 +36,7 @@ object IngestionActor { /** * Oversees ingestion and recovery process for a single dataset. The overall process for a single shard: - * 1. StartShardIngestion command is received and start() called + * 1. Shard state command is received and start() called * 2. MemStore.setup() is called for that shard * 3. IF no checkpoint data is found, THEN normal ingestion is started * 4. IF checkpoints are found, then recovery is started from the minimum checkpoint offset @@ -61,6 +61,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, final val streamSubscriptions = new HashMap[Int, CancelableFuture[Unit]] final val streams = new HashMap[Int, IngestionStream] final val nodeCoord = context.parent + var shardStateVersion: Long = 0 // Params for creating the default memStore flush scheduler private final val numGroups = storeConfig.groupsPerShard @@ -83,66 +84,105 @@ private[filodb] final class IngestionActor(dataset: Dataset, override def postStop(): Unit = { super.postStop() // <- logs shutting down logger.info("Cancelling all streams and calling teardown") - streamSubscriptions.keys.foreach(stop(dataset.ref, _, ActorRef.noSender)) + streamSubscriptions.keys.foreach(stopIngestion(_)) } - /** All [[ShardCommand]] tasks are only started if the dataset - * and shard are valid for this ingester. - */ def receive: Receive = LoggingReceive { - case e: StartShardIngestion => start(e, sender()) - case e: IngestRows => ingest(e) - case GetStatus => status(sender()) - case StopShardIngestion(ds, shard) => stop(ds, shard, sender()) + case GetStatus => status(sender()) + case e: IngestRows => ingest(e) + case e: ShardIngestionState => resync(e, sender()) } - /** Guards that only this dataset's commands are acted upon. - * Handles initial memstore setup of dataset to shard. - * Also handles recovery process. + /** + * Compares the given shard mapper snapshot to the current set of shards being ingested and + * reconciles any differences. It does so by stopping ingestion for shards that aren't mapped + * to this node, and it starts ingestion for those that are. */ - private def start(e: StartShardIngestion, origin: ActorRef): Unit = - if (invalid(e.ref)) handleInvalid(e, Some(origin)) else { - try memStore.setup(dataset, e.shard, storeConfig, downsample) catch { - case ShardAlreadySetup(ds, shard) => - logger.warn(s"dataset=$ds shard=$shard already setup, skipping....") - return - } + private def resync(state: ShardIngestionState, origin: ActorRef): Unit = { + if (invalid(state.ref)) { + logger.error(s"$state is invalid for this ingester '${dataset.ref}'.") + return + } - val ingestion = for { - _ <- memStore.recoverIndex(dataset.ref, e.shard) - checkpoints <- memStore.metastore.readCheckpoints(dataset.ref, e.shard) } - yield { - if (checkpoints.isEmpty) { - // Start normal ingestion with no recovery checkpoint and flush group 0 first - normalIngestion(e.shard, None, 0, storeConfig.diskTTLSeconds) - } else { - // Figure out recovery end watermark and intervals. The reportingInterval is the interval at which - // offsets come back from the MemStore for us to report progress. - val startRecoveryWatermark = checkpoints.values.min + 1 - val endRecoveryWatermark = checkpoints.values.max - val lastFlushedGroup = checkpoints.find(_._2 == endRecoveryWatermark).get._1 - val reportingInterval = Math.max((endRecoveryWatermark - startRecoveryWatermark) / 20, 1L) - logger.info(s"Starting recovery for dataset=${dataset.ref} " + - s"shard=${e.shard}: from $startRecoveryWatermark to $endRecoveryWatermark; " + - s"last flushed group $lastFlushedGroup") - logger.info(s"Checkpoints for dataset=${dataset.ref} shard=${e.shard}: $checkpoints") - for { lastOffset <- doRecovery(e.shard, startRecoveryWatermark, endRecoveryWatermark, reportingInterval, - checkpoints) } - yield { - // Start reading past last offset for normal records; start flushes one group past last group - normalIngestion(e.shard, Some(lastOffset + 1), (lastFlushedGroup + 1) % numGroups, - storeConfig.diskTTLSeconds) + if (state.version != 0 && state.version <= shardStateVersion) { + logger.info(s"Ignoring old ShardIngestionState version: ${state.version} <= $shardStateVersion") + return + } + + // Start with the full set of all shards being ingested, and remove shards from this set + // which must continue being ingested. + val shardsToStop = HashSet() ++ streams.keySet + + for (shard <- 0 until state.map.numShards) { + if (state.map.coordForShard(shard) == context.parent) { + if (state.map.isAnIngestionState(shard)) { + if (shardsToStop.contains(shard)) { + // Is aready ingesting, and it must not be stopped. + shardsToStop.remove(shard) + } else { + // Isn't ingesting, so start it. + startIngestion(shard) } + } else { + val status = state.map.statuses(shard) + logger.info(s"Will stop ingestion of shard $shard due to status ${status}") } } - ingestion.recover { - case NonFatal(t) => - logger.error(s"Error occurred during initialization/execution of ingestion for " + - s"dataset=${dataset.ref} shard=${e.shard}", t) - handleError(dataset.ref, e.shard, t) + } + + // Stop ingesting the rest. + for (shard <- shardsToStop) { + stopIngestion(shard) + } + + if (state.version != 0) { + shardStateVersion = state.version + } + } + + private def startIngestion(shard: Int): Unit = { + try memStore.setup(dataset, shard, storeConfig, downsample) catch { + case ShardAlreadySetup(ds, shard) => + logger.warn(s"dataset=$ds shard=$shard already setup, skipping....") + return + } + + val ingestion = for { + _ <- memStore.recoverIndex(dataset.ref, shard) + checkpoints <- memStore.metastore.readCheckpoints(dataset.ref, shard) } + yield { + if (checkpoints.isEmpty) { + // Start normal ingestion with no recovery checkpoint and flush group 0 first + normalIngestion(shard, None, 0, storeConfig.diskTTLSeconds) + } else { + // Figure out recovery end watermark and intervals. The reportingInterval is the interval at which + // offsets come back from the MemStore for us to report progress. + val startRecoveryWatermark = checkpoints.values.min + 1 + val endRecoveryWatermark = checkpoints.values.max + val lastFlushedGroup = checkpoints.find(_._2 == endRecoveryWatermark).get._1 + val reportingInterval = Math.max((endRecoveryWatermark - startRecoveryWatermark) / 20, 1L) + logger.info(s"Starting recovery for dataset=${dataset.ref} " + + s"shard=${shard}: from $startRecoveryWatermark to $endRecoveryWatermark; " + + s"last flushed group $lastFlushedGroup") + logger.info(s"Checkpoints for dataset=${dataset.ref} shard=${shard}: $checkpoints") + for { lastOffset <- doRecovery(shard, startRecoveryWatermark, endRecoveryWatermark, reportingInterval, + checkpoints) } + yield { + // Start reading past last offset for normal records; start flushes one group past last group + normalIngestion(shard, Some(lastOffset + 1), (lastFlushedGroup + 1) % numGroups, + storeConfig.diskTTLSeconds) + } } } + ingestion.recover { + case NonFatal(t) => + logger.error(s"Error occurred during initialization/execution of ingestion for " + + s"dataset=${dataset.ref} shard=${shard}", t) + handleError(dataset.ref, shard, t) + } + } + private def flushStream(startGroupNo: Int = 0): Observable[FlushCommand] = { if (source.config.as[Option[Boolean]]("noflush").getOrElse(false)) { FlushStream.empty @@ -180,7 +220,7 @@ private[filodb] final class IngestionActor(dataset: Dataset, case Failure(x) => handleError(dataset.ref, shard, x) case Success(_) => - // We dont release resources when fitite ingestion ends normally. + // We dont release resources when finite ingestion ends normally. // Kafka ingestion is usually infinite and does not end unless canceled. // Cancel operation is already releasing after cancel is done. // We also have some tests that validate after finite ingestion is complete @@ -276,21 +316,19 @@ private[filodb] final class IngestionActor(dataset: Dataset, private def status(origin: ActorRef): Unit = origin ! IngestionStatus(memStore.numRowsIngested(dataset.ref)) - /** Guards that only this dataset's commands are acted upon. */ - private def stop(ds: DatasetRef, shard: Int, origin: ActorRef): Unit = - if (invalid(ds)) handleInvalid(StopShardIngestion(ds, shard), Some(origin)) else { - streamSubscriptions.get(shard).foreach { s => - s.onComplete { - case Success(_) => - // release resources when stop is invoked explicitly, not when ingestion ends in non-kafka environments - removeAndReleaseResources(ds, shard) - // ingestion stopped event is already handled in the normalIngestion method - logger.info(s"Stopped streaming ingestion for dataset=${dataset.ref} shard=$shard and released resources") - case Failure(_) => - // release of resources on failure is already handled in the normalIngestion method - } + private def stopIngestion(shard: Int): Unit = { + streamSubscriptions.get(shard).foreach { s => + s.onComplete { + case Success(_) => + // release resources when stop is invoked explicitly, not when ingestion ends in non-kafka environments + removeAndReleaseResources(dataset.ref, shard) + // ingestion stopped event is already handled in the normalIngestion method + logger.info(s"Stopped streaming ingestion for dataset=${dataset.ref} shard=$shard and released resources") + case Failure(_) => + // release of resources on failure is already handled in the normalIngestion method } - streamSubscriptions.get(shard).foreach(_.cancel()) + } + streamSubscriptions.get(shard).foreach(_.cancel()) } private def invalid(ref: DatasetRef): Boolean = ref != dataset.ref @@ -303,19 +341,16 @@ private[filodb] final class IngestionActor(dataset: Dataset, logger.error(s"Stopped dataset=${dataset.ref} shard=$shard after error was thrown") } - private def handleInvalid(command: ShardCommand, origin: Option[ActorRef]): Unit = { - logger.error(s"$command is invalid for this ingester '${dataset.ref}'.") - origin foreach(_ ! InvalidIngestionCommand(command.ref, command.shard)) - } - private def removeAndReleaseResources(ref: DatasetRef, shard: Int): Unit = { - // TODO: Wait for all the queries to stop - streamSubscriptions.remove(shard).foreach(_.cancel) - streams.remove(shard).foreach(_.teardown()) - // Release memory for shard in MemStore - memStore.asInstanceOf[TimeSeriesMemStore].getShard(ref, shard) - .foreach { shard => - shard.shutdown() - } + if (streamSubscriptions.contains(shard)) { + // TODO: Wait for all the queries to stop + streamSubscriptions.remove(shard).foreach(_.cancel) + streams.remove(shard).foreach(_.teardown()) + // Release memory for shard in MemStore + memStore.asInstanceOf[TimeSeriesMemStore].getShard(ref, shard) + .foreach { shard => + shard.shutdown() + } + } } -} \ No newline at end of file +} diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index d96e20fe65..f6e2e2a377 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -111,8 +111,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, } /** Creates a new ingestion actor initialized with the shard actor, - * and sends it a sequence of `StartShardIngestion` commands created - * during dataset setup. + * and sends it a shard resync command created. * * Creates a QueryActor, subscribes it to shard events, keeping * it decoupled from the shard actor. The QueryActor will receive an @@ -161,7 +160,6 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, case DatasetSetup(compactDSString, storeConf, source, downsample) => val dataset = Dataset.fromCompactString(compactDSString) if (!(ingesters contains dataset.ref)) { setupDataset(dataset, storeConf, source, downsample, sender()) } - else { logger.warn(s"Getting redundant DatasetSetup for dataset ${dataset.ref}") } case IngestRows(dataset, shard, rows) => withIngester(sender(), dataset) { _ ! IngestionActor.IngestRows(sender(), shard, rows) } @@ -182,7 +180,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def coordinatorReceive: Receive = LoggingReceive { case e: CoordinatorRegistered => registered(e) - case e: ShardCommand => forward(e, sender()) + case e: ShardIngestionState => forward(e, e.ref, sender()) case Terminated(memstoreCoord) => terminated(memstoreCoord) case MiscCommands.GetClusterActor => sender() ! clusterActor case StatusActor.GetCurrentEvents => statusActor.foreach(_.tell(StatusActor.GetCurrentEvents, sender())) @@ -206,15 +204,15 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, } } - /** Forwards shard commands to the ingester for the given dataset. + /** Forwards shard actions to the ingester for the given dataset. * TODO version match if needed, when > 1, currently only 0. */ - private def forward(command: ShardCommand, origin: ActorRef): Unit = - ingesters.get(command.ref) match { + private def forward(action: ShardAction, ref: DatasetRef, origin: ActorRef): Unit = + ingesters.get(ref) match { case Some(actor) => - actor.tell(command, origin) + actor.tell(action, origin) case _ => - logger.warn(s"No IngestionActor for dataset ${command.ref}") + logger.warn(s"No IngestionActor for dataset ${ref}") } private def terminated(ingester: ActorRef): Unit = { @@ -252,4 +250,4 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, // This is a bit heavy handed, it clears out the entire cache, not just for all datasets memStore.reset() } -} \ No newline at end of file +} diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala index 6ea9afa123..8d4c3457b9 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardManager.scala @@ -71,7 +71,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, * INTERNAL API. Idempotent. */ def subscribe(subscriber: ActorRef, dataset: DatasetRef): Unit = - mapperOpt(dataset) match { + mapperCopyOpt(dataset) match { case Some(current) => logger.info(s"Adding $subscriber as a subscriber for dataset=$dataset") _subscriptions = subscriptions.subscribe(subscriber, dataset) @@ -117,8 +117,22 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, * The response is returned directly to the requester. */ def sendSnapshot(ref: DatasetRef, origin: ActorRef): Unit = - origin ! mapperOpt(ref).getOrElse(DatasetUnknown(ref)) + origin ! mapperCopyOpt(ref).getOrElse(DatasetUnknown(ref)) + /** + * Returns a complete copy of the ShardMapper within a CurrentShardSnapshot, if the dataset + * exists. Although a copy of the ShardMapper isn't typically required, it is required for + * the tests to work properly. This is because the TestProbe provides access to the local + * ShardMapper instance, and so any observation of the snapshot would expose the latest + * mappings instead. The complete copy also offers a nice safeguard, in case the ShardMapper + * is concurrently modified before the message is sent. This isn't really expected, however. + */ + private def mapperCopyOpt(ref: DatasetRef): Option[CurrentShardSnapshot] = + _shardMappers.get(ref).map(m => CurrentShardSnapshot(ref, m.copy())) + + /** + * Same as mapperCopyOpt, except it directly references the ShardMapper instance. + */ private def mapperOpt(ref: DatasetRef): Option[CurrentShardSnapshot] = _shardMappers.get(ref).map(m => CurrentShardSnapshot(ref, m)) @@ -134,19 +148,23 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, for ((dataset, resources, mapper) <- datasetShardMaps) { val assignable = strategy.shardAssignments(coordinator, dataset, resources, mapper) - if (assignable.nonEmpty) sendAssignmentMessagesAndEvents(dataset, coordinator, assignable) + if (assignable.nonEmpty) { + doAssignShards(dataset, coordinator, assignable) + publishChanges(dataset) + } } logger.info(s"Completed addMember for coordinator $coordinator") } /** Called on MemberRemoved, new status already updated. */ - def removeMember(address: Address): Option[ActorRef] = + def removeMember(address: Address): Option[ActorRef] = { _coordinators.get(address) map { coordinator => - logger.info(s"Initiated removeMember for coordinator on $address") - _coordinators remove address - removeCoordinator(coordinator) - coordinator + logger.info(s"Initiated removeMember for coordinator on $address") + _coordinators remove address + removeCoordinator(coordinator) + coordinator } + } private def updateShardMetrics(): Unit = { _datasetInfo.foreach { case (dataset, info) => @@ -259,7 +277,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, * * @return - Validates and returns error message on failure and a unit if no validation error */ - def validateRequestAndStopShards(shardStopReq: StopShards, ackTo: ActorRef): Unit Or ErrorResponse = { + private def validateRequestAndStopShards(shardStopReq: StopShards, ackTo: ActorRef): Unit Or ErrorResponse = { for { shardMapper <- validateDataset(shardStopReq.datasetRef) shards <- validateShardsToStop(shardStopReq.unassignmentConfig.shardList, shardMapper) @@ -271,14 +289,14 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, /** * Shutdown shards from the coordinator where it is running */ - def unassignShards(shards: Seq[Int], - dataset: DatasetRef, - shardMapper: ShardMapper): Unit = { + private def unassignShards(shards: Seq[Int], + dataset: DatasetRef, + shardMapper: ShardMapper): Unit = { for { shard <- shards} { val curCoordinator = shardMapper.coordForShard(shard) - sendUnassignmentMessagesAndEvents(dataset, curCoordinator, - shardMapper, Seq(shard)) + doUnassignShards(dataset, curCoordinator, Seq(shard)) } + publishChanges(dataset) } /** @@ -302,9 +320,9 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, * * @return - Validates and returns error message on failure and a unit if no validation error */ - def validateRequestAndStartShards(dataset: DatasetRef, - assignmentConfig: AssignShardConfig, - ackTo: ActorRef): Unit Or ErrorResponse = { + private def validateRequestAndStartShards(dataset: DatasetRef, + assignmentConfig: AssignShardConfig, + ackTo: ActorRef): Unit Or ErrorResponse = { for { shardMapper <- validateDataset(dataset) coordinator <- validateCoordinator(assignmentConfig.address, assignmentConfig.shardList) @@ -312,7 +330,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, _ <- validateNodeCapacity(shards, shardMapper, dataset, _datasetInfo(dataset).resources, coordinator) } yield { - sendAssignmentMessagesAndEvents(dataset, coordinator, shards) + doAssignShards(dataset, coordinator, shards) + publishChanges(dataset) } } @@ -339,9 +358,10 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, private def removeCoordinator(coordinator: ActorRef): Unit = { for ((dataset, resources, mapper) <- datasetShardMaps) { var shardsToDown = mapper.shardsForCoord(coordinator) - sendUnassignmentMessagesAndEvents(dataset, coordinator, mapper, shardsToDown) + doUnassignShards(dataset, coordinator, shardsToDown) // try to reassign shards that were unassigned to other nodes that have room. assignShardsToNodes(dataset, mapper, resources) + publishChanges(dataset) } logger.info(s"Completed removeMember for coordinator $coordinator") } @@ -353,7 +373,6 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, def addDataset(setup: SetupDataset, dataset: Dataset, ackTo: ActorRef): Map[ActorRef, Seq[Int]] = { - logger.info(s"Initiated setup for dataset=${setup.ref}") val answer: Map[ActorRef, Seq[Int]] = mapperOpt(setup.ref) match { case Some(_) => @@ -380,7 +399,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, ackTo ! DatasetVerified assignments } - updateShardMetrics() + publishChanges(setup.ref) answer } @@ -392,7 +411,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, coord <- latestCoords if !excludeCoords.contains(coord) // assign shards on newer nodes first } yield { val assignable = strategy.shardAssignments(coord, dataset, resources, mapper) - if (assignable.nonEmpty) sendAssignmentMessagesAndEvents(dataset, coord, assignable) + if (assignable.nonEmpty) doAssignShards(dataset, coord, assignable) coord -> assignable }).toMap } @@ -403,7 +422,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, (_, coord) <- _coordinators mapper = _shardMappers(dataset) shardsToDown = mapper.shardsForCoord(coord) - } sendUnassignmentMessagesAndEvents(dataset, coord, mapper, shardsToDown) + } doUnassignShards(dataset, coord, shardsToDown) + publishChanges(dataset) _datasetInfo remove dataset _shardMappers remove dataset _subscriptions = _subscriptions - dataset @@ -417,7 +437,7 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, def recoverShards(ref: DatasetRef, map: ShardMapper): Unit = { logger.info(s"Recovering ShardMap for dataset=$ref ; ShardMap contents: $map") _shardMappers(ref) = map - updateShardMetrics() + publishChanges(ref) } def recoverSubscriptions(subs: ShardSubscriptions): Unit = { @@ -449,8 +469,6 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, // Above condition ensures that we respond to shard events only from the node shard is currently assigned to. // Needed to avoid race conditions where IngestionStopped for an old assignment comes after shard is reassigned. updateFromShardEvent(event) - // RecoveryInProgress status results in too many messages that really do not need a publish - if (!event.isInstanceOf[RecoveryInProgress]) publishSnapshot(event.ref) // reassign shard if IngestionError. Exclude previous node since it had error shards. event match { case _: IngestionError => @@ -477,7 +495,8 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, } case _ => } - updateShardMetrics() + // RecoveryInProgress status results in too many messages that really do not need a publish + if (!event.isInstanceOf[RecoveryInProgress]) publishSnapshot(event.ref) } else { logger.warn(s"Ignoring event $event from $sender for dataset=${event.ref} since it does not match current " + s"owner of shard=${event.shard} which is ${mapper.coordForShard(event.shard)}") @@ -510,13 +529,9 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, updateShardMetrics() } - /** - * This method has the shared logic for sending shard assignment messages - * to the coordinator, updating state for the event and broadcast of the state change to subscribers - */ - private def sendAssignmentMessagesAndEvents(dataset: DatasetRef, - coord: ActorRef, - shards: Seq[Int]): Unit = { + private def doAssignShards(dataset: DatasetRef, + coord: ActorRef, + shards: Seq[Int]): Unit = { val state = _datasetInfo(dataset) logger.info(s"Sending setup message for dataset=${state.dataset.ref} to coordinators $coord.") val setupMsg = client.IngestionCommands.DatasetSetup(state.dataset.asCompactString, @@ -527,42 +542,52 @@ private[coordinator] final class ShardManager(settings: FilodbSettings, val event = ShardAssignmentStarted(dataset, shard, coord) updateFromShardEvent(event) } - publishSnapshot(dataset) - /* If no shards are assigned to a coordinator, no commands are sent. */ - logger.info(s"Sending start ingestion message for dataset=$dataset to coordinator $coord for shards $shards") - for {shard <- shards} coord ! StartShardIngestion(dataset, shard, None) } - /** - * This method has the shared logic for sending shard un-assignment messages - * to the coordinator, updating state for the event and broadcast of the state change to subscribers - */ - private def sendUnassignmentMessagesAndEvents(dataset: DatasetRef, - coordinator: ActorRef, - mapper: ShardMapper, - shardsToDown: Seq[Int], - nodeUp: Boolean = true): Unit = { - logger.info(s"Sending stop ingestion message for dataset=$dataset to " + + private def doUnassignShards(dataset: DatasetRef, + coordinator: ActorRef, + shardsToDown: Seq[Int]): Unit = { + logger.info(s"Unassigning shards for dataset=$dataset to " + s"coordinator $coordinator for shards $shardsToDown") for { shard <- shardsToDown } { val event = ShardDown(dataset, shard, coordinator) updateFromShardEvent(event) - if (nodeUp) coordinator ! StopShardIngestion(dataset, shard) } - publishSnapshot(dataset) - logger.debug(s"Unassigned shards $shardsToDown of dataset=$dataset from $coordinator") + } + + /** + * To be called after making a bunch of changes to the ShardMapper for the given dataset. + * Calling this method more often is permitted, but it generates more publish messages + * than is necessary. + */ + private def publishChanges(ref: DatasetRef): Unit = { + publishSnapshot(ref) + updateShardMetrics() } /** Publishes a ShardMapper snapshot of given dataset to all subscribers of that dataset. */ - def publishSnapshot(ref: DatasetRef): Unit = - mapperOpt(ref) match { - case Some(snapshot) => + def publishSnapshot(ref: DatasetRef): Unit = { + mapperCopyOpt(ref) match { + case Some(snapshot) => { for { subscription <- _subscriptions.subscription(ref) } subscription.subscribers foreach (_ ! snapshot) + + // Also send a complete ingestion state command to all ingestion actors. Without this, + // they won't start or stop ingestion. + + // TODO: Need to provide a globally consistent version, incremented when anything + // changes, for any dataset. + val resync = ShardIngestionState(0, snapshot.ref, snapshot.map) + + for (coord <- coordinators) { + coord ! resync + } + } case None => logger.warn(s"Cannot publish snapshot which doesn't exist for ref $ref") } + } private def latestCoords: Seq[ActorRef] = _coordinators.values.foldLeft(List[ActorRef]())((x, y) => y :: x) // reverses the set @@ -583,4 +608,4 @@ private[coordinator] object ShardManager { downsample: DownsampleConfig, storeConfig: StoreConfig, dataset: Dataset) -} \ No newline at end of file +} diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala b/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala index f62ef97486..36ccf6813a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardMapper.scala @@ -164,6 +164,11 @@ class ShardMapper(val numShards: Int) extends Serializable { def numAssignedShards: Int = numShards - unassignedShards.length + def isAnIngestionState(shard: Int): Boolean = statusMap(shard) match { + case ShardStatusStopped | ShardStatusDown => false + case _ => true + } + /** * Find out if a shard is active (Normal or Recovery status) or filter a list of shards */ diff --git a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala index cf3375cffb..77814a91be 100644 --- a/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala +++ b/coordinator/src/main/scala/filodb.coordinator/ShardStatus.scala @@ -12,30 +12,12 @@ final case class CurrentShardSnapshot(ref: DatasetRef, map: ShardMapper) extends ShardAction with Response /** - * These commands are sent by the NodeClusterActor to the right nodes upon events or - * changes to the cluster. For example a new node joins, StartShardIngestion might be sent. - * Should start with a verb, since these are commands. + * Full state of all shards, sent to all ingestion actors. They react by starting/stopping + * ingestion for the shards they own or no longer own. The version is expected to be global + * and monotonically increasing, but if the version is 0, then the actor should skip the + * version check and blindly apply the resync action. */ -sealed trait ShardCommand extends ShardAction { - def ref: DatasetRef - def shard: Int -} - -final case class StartShardIngestion(ref: DatasetRef, shard: Int, offset: Option[Long]) extends ShardCommand - -final case class StopShardIngestion(ref: DatasetRef, shard: Int) extends ShardCommand - -/** Direct result of sending an invalid [[ShardCommand]]. It is acked to the - * sender if the shard command's shard or dataset is not valid based on the - * projection or shard state. It is located with the shard commands because - * this is not a potential result of an Ingestion command and flows through - * a node's coordinator, one of its ingesters, the cluster shard actor and - * its [[filodb.coordinator.ShardAssignmentStrategy]]. - * - * Use cases: result of invalid state and injecting failure to the memstore - * during stream creation in the [[StartShardIngestion]] protocol. - */ -final case class InvalidIngestionCommand(ref: DatasetRef, shard: Int) +final case class ShardIngestionState(version: Long, ref: DatasetRef, map: ShardMapper) extends ShardAction /** * The events are sent by the IngestionActor on a node when the actual ingestion stream diff --git a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala index 0748692a64..88e6511eba 100644 --- a/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/IngestionStreamSpec.scala @@ -116,9 +116,6 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi "if incorrect shard is sent for the creation of the stream") { setup(dataset6.ref, "/GDELT-sample-test.csv", rowsToRead = 5, None) - val invalidShard = -1 - coordinatorActor ! StartShardIngestion(dataset6.ref, invalidShard, None) - // first it becomes active expectMsgPF(within) { case CurrentShardSnapshot(dataset6.ref, mapper) => @@ -191,6 +188,7 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi // expectMsg(IngestionStopped(dataset33.ref, 0)) // Unfortunately since we do not get every message we cannot actually check the progression of recovery + /* RecoveryInProgress events don't publish snapshots because of the high message load. for { i <- 0 until 3} { expectMsgPF(within) { case CurrentShardSnapshot(dataset33.ref, mapper) => @@ -198,6 +196,7 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi mapper.statuses.head.isInstanceOf[ShardStatusRecovery] shouldEqual true } } + */ expectMsgPF(within) { case CurrentShardSnapshot(dataset33.ref, mapper) => @@ -215,4 +214,4 @@ class IngestionStreamSpec extends ActorTest(IngestionStreamSpec.getNewSystem) wi coordinatorActor ! GetIngestionStats(dataset33.ref) expectMsg(IngestionActor.IngestionStatus(85)) // <-- must be rounded to 5, we ingest entire batches } -} \ No newline at end of file +} diff --git a/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala index 12efb0c965..8e4f2b05fd 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ReassignShardsSpec.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import scala.concurrent.duration._ + import akka.actor.{ActorRef, Address} import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory @@ -23,26 +25,45 @@ class ReassignShardsSpec extends AkkaSpec { val settings = new FilodbSettings(ConfigFactory.load("application_test.conf")) protected val shardManager = new ShardManager(settings, DefaultShardAssignmentStrategy) - val coord1 = TestProbe("coordinator1") + private def makeTestProbe(name: String): TestProbe = { + val tp = TestProbe(name) + // Uncomment to ignore messages of a specific type. + //tp.ignoreMsg({case m: Any => m.isInstanceOf[...]}) + tp + } + + val coord1 = makeTestProbe("coordinator1") val coord1Address = uniqueAddress(coord1.ref) - val coord2 = TestProbe("coordinator2") + val coord2 = makeTestProbe("coordinator2") val coord2Address = uniqueAddress(coord2.ref) - val coord3 = TestProbe("coordinator3") + val coord3 = makeTestProbe("coordinator3") val coord3Address = uniqueAddress(coord3.ref) - val coord4 = TestProbe("coordinator4") + val coord4 = makeTestProbe("coordinator4") val coord4Address = uniqueAddress(coord4.ref) - val coordInvalid = TestProbe("coordinatorInvalid") + val coordInvalid = makeTestProbe("coordinatorInvalid") val coordInvalidAddress = uniqueAddress(coordInvalid.ref) - val subscriber = TestProbe("subscriber") + val subscriber = makeTestProbe("subscriber") val noOpSource1 = IngestionSource(classOf[NoOpStreamFactory].getName) val setupDs1 = SetupDataset(dataset1, resources1, noOpSource1, TestData.storeConf) + private def expectDataset(coord: TestProbe, dataset: Dataset): TestProbe = { + coord.expectMsgPF() { case ds: DatasetSetup => + ds.compactDatasetStr shouldEqual dataset.asCompactString + ds.source shouldEqual noOpSource1 + } + coord + } + + private def expectNoMessage(coord: TestProbe): Unit = { + coord.expectNoMessage(100.milliseconds) + } + def uniqueAddress(probe: ActorRef): Address = probe.path.address.copy(system = s"${probe.path.address.system}-${probe.path.name}") @@ -52,17 +73,17 @@ class ReassignShardsSpec extends AkkaSpec { "fail with no datasets" in { shardManager.subscribeAll(subscriber.ref) subscriber.expectMsg(ShardSubscriptions(Set.empty, Set(subscriber.ref))) - subscriber.expectNoMessage() // should not get a CurrentShardSnapshot since there isnt a dataset yet + expectNoMessage(subscriber) // should not get a CurrentShardSnapshot since there isnt a dataset yet shardManager.addMember(coord3Address, coord3.ref) shardManager.coordinators shouldBe Seq(coord3.ref) shardManager.datasetInfo.size shouldBe 0 - coord3.expectNoMessage() // since there are no datasets, there should be no assignments + expectNoMessage(coord3) // since there are no datasets, there should be no assignments shardManager.addMember(coord4Address, coord4.ref) shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref) shardManager.datasetInfo.size shouldBe 0 - coord4.expectNoMessage() // since there are no more shards left to assign + expectNoMessage(coord4) // since there are no more shards left to assign val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(0,1)) shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) @@ -80,50 +101,45 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.datasetInfo.size shouldBe 1 assignments shouldEqual Map(coord4.ref -> Seq(0, 1, 2), coord3.ref -> Seq(3, 4, 5)) expectMsg(DatasetVerified) - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - // assignments first go to the most recently deployed node - coord4.expectMsgAllOf( - StartShardIngestion(dataset1, 0, None), - StartShardIngestion(dataset1, 1, None), - StartShardIngestion(dataset1, 2, None)) - - coord3.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - // assignments first go to the most recently deployed node - coord3.expectMsgAllOf( - StartShardIngestion(dataset1, 3, None), - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) - - // NOTE: because subscriptions do not kick in right away, we don't get new snapshots unitl after - // ShardSubscriptions message + + for (coord <- Seq(coord3, coord4)) { + expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5) + } + expectNoMessage(coord) + } + + // NOTE: because subscriptions do not kick in right away, we don't get new snapshots until + // after ShardSubscriptions message + subscriber.expectMsg(ShardSubscriptions(Set( ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref))) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.ref shouldEqual dataset1 - s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5) -// s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) -// s.map.shardsForCoord(coord1.ref) shouldEqual Seq() + + for (i <- 1 to 2) { + // First is the initial set, the second is generated along with the state. + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + } } - subscriber.expectNoMessage() + expectNoMessage(subscriber) val shardAssign1 = AssignShardConfig(coord4Address.toString, Seq(5)) shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) expectMsgPF() { case s: BadSchema => s.message should startWith(s"Can not start") } - subscriber.expectNoMessage() + expectNoMessage(subscriber) val shardAssign2 = AssignShardConfig(coord2Address.toString, Seq(0)) shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self) expectMsg(BadData(s"${coord2Address.toString} not found")) - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "fail with invalid node" in { @@ -138,13 +154,23 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref, coord2.ref) shardManager.datasetInfo.size shouldBe 1 - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 + expectDataset(coord2, datasetObj1) + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) } - coord2.expectMsgAllOf( - StartShardIngestion(dataset1, 6, None), - StartShardIngestion(dataset1, 7, None)) val assignments = shardManager.shardMappers(dataset1).shardValues assignments shouldEqual Array((coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), @@ -154,14 +180,45 @@ class ReassignShardsSpec extends AkkaSpec { val shardAssign1 = AssignShardConfig(coord2Address.toString, Seq(5)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) expectMsg(Success) + expectDataset(coord2, datasetObj1) + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) } + + expectNoMessage(subscriber) } "not change after adding spare node" in { @@ -169,13 +226,17 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.addMember(coord1Address, coord1.ref) shardManager.coordinators shouldBe Seq(coord3.ref, coord4.ref, coord2.ref, coord1.ref) shardManager.datasetInfo.size shouldBe 1 - coord1.expectNoMessage() // since there are no datasets, there should be no assignments + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + expectNoMessage(coord) // since there are no datasets, there should be no assignments + } val assignments = shardManager.shardMappers(dataset1).shardValues assignments shouldEqual Array((coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), (coord3.ref, ShardStatusAssigned), (coord3.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned)) + expectNoMessage(subscriber) } "fail with invalid datasets" in { @@ -183,11 +244,7 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.startShards(NodeClusterActor.StartShards(shardAssign, dataset2), self) expectMsg(DatasetUnknown(dataset2)) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) - } + expectNoMessage(subscriber) } "fail with invalid shardNum" in { @@ -196,12 +253,7 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) expectMsg(BadSchema(s"Invalid shards found List(8). Valid shards are List()")) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) - } - + expectNoMessage(subscriber) } "fail when assigned to same node" in { @@ -225,9 +277,41 @@ class ReassignShardsSpec extends AkkaSpec { val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(2)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) expectMsg(Success) + expectDataset(coord1, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 1) s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) @@ -238,9 +322,41 @@ class ReassignShardsSpec extends AkkaSpec { val shardAssign2 = AssignShardConfig(coord3Address.toString, Seq(1)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self) expectMsg(Success) + expectDataset(coord3, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0) s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) @@ -254,11 +370,43 @@ class ReassignShardsSpec extends AkkaSpec { val shardAssign2 = AssignShardConfig(coord1Address.toString, Seq(0, 7)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(2) + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self) expectMsg(Success) + expectDataset(coord1, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord4.ref) shouldEqual Seq() + s.map.shardsForCoord(coord4.ref) shouldEqual Nil s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 3, 4) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(5, 6) s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7) @@ -271,6 +419,22 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.coordinators shouldBe Seq(coord4.ref, coord2.ref, coord1.ref) shardManager.datasetInfo.size shouldBe 1 + expectDataset(coord4, datasetObj1) + expectDataset(coord2, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 2, 7) + } + expectNoMessage(coord) + } + + expectNoMessage(coord3) + subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) @@ -284,9 +448,40 @@ class ReassignShardsSpec extends AkkaSpec { val shardAssign2 = AssignShardConfig(coord4Address.toString, Seq(2)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign2.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7) + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign2, dataset1), self) expectMsg(Success) + expectDataset(coord4, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(2, 3, 4) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 7) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.shardsForCoord(coord4.ref) shouldEqual Seq(2, 3, 4) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(1, 5, 6) @@ -299,10 +494,28 @@ class ReassignShardsSpec extends AkkaSpec { shardManager.removeDataset(dataset1) shardManager.datasetInfo.size shouldBe 0 + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + } + val shardAssign1 = AssignShardConfig(coord1Address.toString, Seq(0,1)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self) expectMsg(DatasetUnknown(dataset1)) // since there are no datasets + expectNoMessage(subscriber) } "succeed after adding dataset back" in { @@ -313,9 +526,53 @@ class ReassignShardsSpec extends AkkaSpec { assignments shouldEqual Map(coord1.ref -> Seq(0, 1, 2), coord2.ref -> Seq(3, 4, 5), coord4.ref -> Seq(6, 7)) expectMsg(DatasetVerified) + expectDataset(coord4, datasetObj1) + expectDataset(coord2, datasetObj1) + expectDataset(coord1, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + expectNoMessage(coord) + } + + subscriber.expectMsg(ShardSubscriptions(Set( + ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref))) + + for (i <- 1 to 2) { + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + } + val shardAssign1 = AssignShardConfig(coord4Address.toString, Seq(5)) shardManager.stopShards(NodeClusterActor.StopShards(UnassignShardConfig(shardAssign1.shardList), dataset1), self) expectMsg(Success) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + shardManager.startShards(NodeClusterActor.StartShards(shardAssign1, dataset1), self) expectMsg(Success) @@ -324,6 +581,26 @@ class ReassignShardsSpec extends AkkaSpec { (coord1.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord2.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned), (coord4.ref, ShardStatusAssigned)) + expectDataset(coord4, datasetObj1) + + for (coord <- Seq(coord1, coord2, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + expectNoMessage(coord) + } + + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(5, 6, 7) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 4) + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + } + + expectNoMessage(subscriber) } } -} \ No newline at end of file +} diff --git a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala index 7033d5afb4..c0683b6d93 100644 --- a/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/ShardManagerSpec.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import scala.concurrent.duration._ + import akka.actor.{ActorRef, Address} import akka.testkit.TestProbe import com.typesafe.config.ConfigFactory @@ -9,7 +11,7 @@ import filodb.core.metadata.Dataset class ShardManagerSpec extends AkkaSpec { import NodeClusterActor.{DatasetResourceSpec, DatasetVerified, IngestionSource, SetupDataset} - import client.IngestionCommands.DatasetSetup + import filodb.coordinator.client.IngestionCommands.DatasetSetup protected val dataset1 = DatasetRef("one") protected val datasetObj1 = Dataset(dataset1.dataset, Seq("seg:int"), Seq("timestamp:long")) @@ -22,19 +24,26 @@ class ShardManagerSpec extends AkkaSpec { val settings = new FilodbSettings(ConfigFactory.load("application_test.conf")) protected val shardManager = new ShardManager(settings, DefaultShardAssignmentStrategy) - val coord1 = TestProbe("coordinator1") + private def makeTestProbe(name: String): TestProbe = { + val tp = TestProbe(name) + // Uncomment to ignore messages of a specific type. + //tp.ignoreMsg({case m: Any => m.isInstanceOf[...]}) + tp + } + + val coord1 = makeTestProbe("coordinator1") val coord1Address = uniqueAddress(coord1.ref) - val coord2 = TestProbe("coordinator2") + val coord2 = makeTestProbe("coordinator2") val coord2Address = uniqueAddress(coord2.ref) - val coord3 = TestProbe("coordinator3") + val coord3 = makeTestProbe("coordinator3") val coord3Address = uniqueAddress(coord3.ref) - val coord4 = TestProbe("coordinator4") + val coord4 = makeTestProbe("coordinator4") val coord4Address = uniqueAddress(coord4.ref) - val subscriber = TestProbe("subscriber") + val subscriber = makeTestProbe("subscriber") val noOpSource1 = IngestionSource(classOf[NoOpStreamFactory].getName) val setupDs1 = SetupDataset(dataset1, resources1, noOpSource1, TestData.storeConf) @@ -42,6 +51,17 @@ class ShardManagerSpec extends AkkaSpec { val noOpSource2 = IngestionSource(classOf[NoOpStreamFactory].getName) val setupDs2 = SetupDataset(dataset2, resources2, noOpSource2, TestData.storeConf) + private def expectDataset(coord: TestProbe, dataset: Dataset): TestProbe = { + coord.expectMsgPF() { case ds: DatasetSetup => + ds.compactDatasetStr shouldEqual dataset.asCompactString + ds.source shouldEqual noOpSource1 + } + coord + } + + private def expectNoMessage(coord: TestProbe): Unit = { + coord.expectNoMessage(100.milliseconds) + } def uniqueAddress(probe: ActorRef): Address = probe.path.address.copy(system = s"${probe.path.address.system}-${probe.path.name}") @@ -52,35 +72,35 @@ class ShardManagerSpec extends AkkaSpec { "allow subscription of self for shard events on all datasets" in { shardManager.subscribeAll(subscriber.ref) subscriber.expectMsg(ShardSubscriptions(Set.empty, Set(subscriber.ref))) - subscriber.expectNoMessage() // should not get a CurrentShardSnapshot since there isnt a dataset yet + expectNoMessage(subscriber) // should not get a CurrentShardSnapshot since there isnt a dataset yet } "change state for addition of first coordinator without datasets" in { shardManager.addMember(coord1Address, coord1.ref) shardManager.coordinators shouldBe Seq(coord1.ref) shardManager.datasetInfo.size shouldBe 0 - coord1.expectNoMessage() // since there are no datasets, there should be no assignments + expectNoMessage(coord1) // since there are no datasets, there should be no assignments } "change state for addition of second coordinator without datasets" in { shardManager.addMember(coord2Address, coord2.ref) shardManager.coordinators shouldBe Seq(coord1.ref, coord2.ref) shardManager.datasetInfo.size shouldBe 0 - coord2.expectNoMessage() // since there are no datasets, there should be no assignments + expectNoMessage(coord2) // since there are no datasets, there should be no assignments } "change state for addition of third coordinator without datasets" in { shardManager.addMember(coord3Address, coord3.ref) shardManager.coordinators shouldBe Seq(coord1.ref, coord2.ref, coord3.ref) shardManager.datasetInfo.size shouldBe 0 - coord3.expectNoMessage() // since there are no datasets, there should be no assignments + expectNoMessage(coord3) // since there are no datasets, there should be no assignments } "change state for removal of coordinator without datasets" in { shardManager.removeMember(coord2Address) shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref) shardManager.datasetInfo.size shouldBe 0 - coord2.expectNoMessage() // since there are no datasets, there should be no assignments + expectNoMessage(coord2) // since there are no datasets, there should be no assignments } "change state for addition of new dataset" in { @@ -89,57 +109,57 @@ class ShardManagerSpec extends AkkaSpec { assignments shouldEqual Map(coord3.ref -> Seq(0,1,2), coord1.ref -> Seq(3,4,5)) expectMsg(DatasetVerified) - coord3.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - // assignments first go to the most recently deployed node - coord3.expectMsgAllOf( - StartShardIngestion(dataset1, 0, None), - StartShardIngestion(dataset1, 1, None), - StartShardIngestion(dataset1, 2, None)) - - coord1.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 + + for (coord <- Seq(coord1, coord3)) { + expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + } + expectNoMessage(coord) } - coord1.expectMsgAllOf( - StartShardIngestion(dataset1, 3, None), - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) - // NOTE: because subscriptions do not kick in right away, we don't get new snapshots unitl after - // ShardSubscriptions message + // NOTE: because subscriptions do not kick in right away, we don't get new snapshots until + // after ShardSubscriptions message } "send shard subscribers updates on shard events as a result of dataset addition" in { subscriber.expectMsg(ShardSubscriptions(Set( ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref))) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.ref shouldEqual dataset1 - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5) - } - subscriber.expectNoMessage() + for (i <- 1 to 2) { + // First is the initial set, the second is generated along with the state. + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + } + } + expectNoMessage(subscriber) } "change state for addition of coordinator when there are datasets" in { shardManager.addMember(coord2Address, coord2.ref) shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref, coord2.ref) - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - coord2.expectMsgAllOf( - StartShardIngestion(dataset1, 6, None), - StartShardIngestion(dataset1, 7, None)) - // Should receive new snapshot with shards 6,7 for coord2 subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + } + + expectDataset(coord2, datasetObj1) + + // All should see the changes. + for (coord <- Seq(coord1, coord2, coord3)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(3, 4, 5) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + } + expectNoMessage(coord) } } @@ -147,8 +167,12 @@ class ShardManagerSpec extends AkkaSpec { shardManager.addMember(coord4Address, coord4.ref) shardManager.coordinators shouldBe Seq(coord1.ref, coord3.ref, coord2.ref, coord4.ref) shardManager.datasetInfo.size shouldBe 1 - coord4.expectNoMessage() // since this is a spare node, there should be no assignments - subscriber.expectNoMessage() + expectNoMessage(coord4) // since this is a spare node, there should be no assignments + expectNoMessage(subscriber) + + for (coord <- Seq(coord1, coord2, coord3, coord3)) { + expectNoMessage(coord) + } } "change state for removal of coordinator when there are datasets and spare nodes" in { @@ -156,34 +180,29 @@ class ShardManagerSpec extends AkkaSpec { shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord4.ref) shardManager.datasetInfo.size shouldBe 1 - // first ingestion should be stopped on downed node - coord1.expectMsgAllOf( - StopShardIngestion(dataset1, 3), - StopShardIngestion(dataset1, 4), - StopShardIngestion(dataset1, 5)) + expectDataset(coord4, datasetObj1) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord1.ref) shouldEqual Nil - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + // All coordinators should see a state message, even the removed one. + for (coord <- Seq(coord2, coord3, coord4)) { + //expectDataset(coord, datasetObj1).expectMsgPF() { case s: ShardIngestionState => + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil // coord1 is gone + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) // coord4 takes over + } + expectNoMessage(coord) } - // spare coord4 should take over the shards - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - coord4.expectMsgAllOf( - StartShardIngestion(dataset1, 3, None), - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) } - subscriber.expectNoMessage() + + expectNoMessage(subscriber) } "reassign shards where additional room available on removal of coordinator when there are no spare nodes" in { @@ -191,64 +210,59 @@ class ShardManagerSpec extends AkkaSpec { shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref) shardManager.datasetInfo.size shouldBe 1 - // ingestion should be stopped on downed node - coord4.expectMsgAllOf( - StopShardIngestion(dataset1, 3), - StopShardIngestion(dataset1, 4), - StopShardIngestion(dataset1, 5)) - + expectDataset(coord2, datasetObj1) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) - // s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) - // NOTE: you would expect shard2 to have 6 & 7 only in the first snapshot which is after removing coord4. - // The problem is that ShardMapper is mutable, and since this is a local transfer, the actor message - // points to the ShardMapper object which by this point already has the shard 3 added. :( - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) - s.map.shardsForCoord(coord4.ref) shouldEqual Nil - s.map.numAssignedShards shouldEqual 6 + // Ingestion should be stopped on downed node, and one shard reassigned. + for (coord <- Seq(coord2, coord3)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + expectNoMessage(coord) } - // coord2 has room for one more - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 + for (coord <- Seq(coord1, coord2, coord3, coord3)) { + expectNoMessage(coord) } - coord2.expectMsgAllOf( - StartShardIngestion(dataset1, 3, None)) - - // since there are no spare nodes now, other coords (which are "down") should not get any message - coord1.expectNoMessage() - coord3.expectNoMessage() subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord1.ref) shouldEqual Nil s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) - s.map.unassignedShards shouldEqual Seq(4, 5) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Nil s.map.numAssignedShards shouldEqual 6 } - subscriber.expectNoMessage() + + expectNoMessage(subscriber) } "reassign remaining unassigned shards when a replacement node comes back" in { shardManager.addMember(coord4.ref.path.address, coord4.ref) shardManager.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord4.ref) - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 + expectDataset(coord4, datasetObj1) + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5) + } + expectNoMessage(coord) } - coord4.expectMsgAllOf( - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(0, 1, 2) s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5) s.map.unassignedShards shouldEqual Nil } - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "ingestion error on a shard should reassign shard to another node" in { @@ -256,81 +270,76 @@ class ShardManagerSpec extends AkkaSpec { shardManager.updateFromExternalShardEvent(subscriber.ref, IngestionError(dataset1, 0, new IllegalStateException("simulated"))) - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - coord4.expectMsgAllOf( - StartShardIngestion(dataset1, 0, None)) - - // We are asserting two snapshot messages of same type below. - // The first one should have been one where shard 0 is unassigned and in error state. - // Subsequently, the reassignment should have caused it to change. - // However, we are forced to assert on the final state because the mutation - // of the shard map object has already happened by the time updateFromExternalShardEvent returns - // and we are able to only validate the final state here on both objects. - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) - s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5) - s.map.unassignedShards shouldEqual Nil + expectDataset(coord4, datasetObj1) + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5) + } + expectNoMessage(coord) } subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) s.map.shardsForCoord(coord4.ref) shouldEqual Seq(0, 4, 5) s.map.unassignedShards shouldEqual Nil } - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "continual ingestion error on a shard should not reassign shard to another node" in { shardManager.coordinators shouldEqual Seq(coord3.ref, coord2.ref, coord4.ref) shardManager.updateFromExternalShardEvent(subscriber.ref, IngestionError(dataset1, 0, new IllegalStateException("simulated"))) - coord3.expectNoMessage() - coord2.expectNoMessage() - coord4.expectNoMessage() + + // Shard 0 is "bad" and so it's unassigned now. + + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5) + } + expectNoMessage(coord) + } + subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(1, 2) s.map.shardsForCoord(coord4.ref) shouldEqual Seq(4, 5) s.map.unassignedShards shouldEqual Seq(0) } - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "change state for removal of dataset" in { shardManager.removeDataset(dataset1) shardManager.datasetInfo.size shouldBe 0 - coord3.expectMsgAllOf( - StopShardIngestion(dataset1, 1), - StopShardIngestion(dataset1, 2)) - - coord4.expectMsgAllOf( - StopShardIngestion(dataset1, 4), // shard 0 is in error state - StopShardIngestion(dataset1, 5)) - - coord2.expectMsgAllOf( - StopShardIngestion(dataset1, 3), - StopShardIngestion(dataset1, 6), - StopShardIngestion(dataset1, 7)) + for (coord <- Seq(coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Nil + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + expectNoMessage(coord) + } shardManager.subscriptions.subscriptions.size shouldBe 0 - // 3 snapshots one for each coord - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.unassignedShards shouldEqual (0 to 7) - } subscriber.expectMsgPF() { case s: CurrentShardSnapshot => s.map.unassignedShards shouldEqual (0 to 7) } - subscriber.expectMsgPF() { case s: CurrentShardSnapshot => - s.map.unassignedShards shouldEqual (0 to 7) - } - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "change state for addition of multiple datasets" in { @@ -341,50 +350,41 @@ class ShardManagerSpec extends AkkaSpec { val assignments1 = shardManager.addDataset(setupDs1, datasetObj1, self) shardManager.datasetInfo.size shouldBe 1 assignments1 shouldEqual Map(coord1.ref -> Seq(0, 1, 2), - coord4.ref -> Seq(3, 4, 5), coord2.ref -> Seq(6, 7), - coord3.ref -> Seq()) - - coord1.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - // assignments first go to the most recently deployed node - coord1.expectMsgAllOf( - StartShardIngestion(dataset1, 0, None), - StartShardIngestion(dataset1, 1, None), - StartShardIngestion(dataset1, 2, None)) + coord3.ref -> Seq(), + coord4.ref -> Seq(3, 4, 5)) - // No CurrentShardSnapshot yet. We have to get subscription first. + expectDataset(coord1, datasetObj1) + expectDataset(coord2, datasetObj1) + expectDataset(coord4, datasetObj1) - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) + } + expectNoMessage(coord) } - coord4.expectMsgAllOf( - StartShardIngestion(dataset1, 3, None), - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) - - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - coord2.expectMsgAllOf( - StartShardIngestion(dataset1, 6, None), - StartShardIngestion(dataset1, 7, None)) - - coord3.expectNoMessage() // coord3 is spare node for dataset1 // addition of dataset results in snapshot/subscriptions broadcast subscriber.expectMsg( ShardSubscriptions(Set(ShardSubscription(dataset1, Set(subscriber.ref))), Set(subscriber.ref))) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => - s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) - s.map.unassignedShards shouldEqual Nil + for (i <- 1 to 2) { + // First is the initial set, the second is generated along with the state. + subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(6, 7) + s.map.shardsForCoord(coord4.ref) shouldEqual Seq(3, 4, 5) + s.map.unassignedShards shouldEqual Nil + } } + expectNoMessage(subscriber) + + // Assignments first go to the most recently deployed node, + // coord2 and coord3 are spare nodes for dataset2. val assignments2 = shardManager.addDataset(setupDs2, datasetObj2, self) shardManager.datasetInfo.size shouldBe 2 @@ -394,24 +394,19 @@ class ShardManagerSpec extends AkkaSpec { coord2.ref -> Seq.empty, coord3.ref -> Seq.empty) - coord1.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj2.asCompactString - ds.source shouldEqual noOpSource2 - } - // assignments first go to the most recently deployed node - val msgs1 = coord1.receiveWhile(messages = 8) { case m: StartShardIngestion if m.ref == dataset2 => m } - msgs1 should have length (8) + expectDataset(coord1, datasetObj2) + expectDataset(coord4, datasetObj2) - coord4.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj2.asCompactString - ds.source shouldEqual noOpSource2 + for (coord <- Seq(coord1, coord2, coord3, coord4)) { + coord.expectMsgPF() { case s: ShardIngestionState => + s.ref shouldEqual dataset2 + s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) + s.map.shardsForCoord(coord2.ref) shouldEqual Nil + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16) + } + expectNoMessage(coord) } - val msgs2 = coord4.receiveWhile(messages = 8) { case m: StartShardIngestion if m.ref == dataset2 => m } - msgs2 should have length (8) - - // coord2 and coord3 are spare nodes for dataset2 - coord2.expectNoMessage() - coord3.expectNoMessage() // shard subscriptions should work with multiple datasets subscriber.expectMsg( @@ -419,14 +414,16 @@ class ShardManagerSpec extends AkkaSpec { ShardSubscription(dataset2,Set(subscriber.ref))), Set(subscriber.ref))) - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 => - s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) - s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq.empty - s.map.shardsForCoord(coord3.ref) shouldEqual Seq.empty + for (i <- 1 to 2) { + // First is the initial set, the second is generated along with the state. + subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 => + s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq.empty + s.map.shardsForCoord(coord3.ref) shouldEqual Seq.empty + s.map.shardsForCoord(coord4.ref) shouldEqual Range(8, 16) + } } - - subscriber.expectNoMessage() + expectNoMessage(subscriber) } "recover state on a failed over node " in { @@ -455,19 +452,12 @@ class ShardManagerSpec extends AkkaSpec { shardManager2.coordinators shouldBe Seq(coord3.ref, coord2.ref, coord1.ref) shardManager2.datasetInfo.size shouldBe 2 - // dataset2: reassign shards 8-16 to coord2. Will get two snapshots during move + // dataset2: reassign shards 8-16 to coord2. subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 => s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) - s.map.shardsForCoord(coord4.ref) shouldEqual Nil s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16) s.map.shardsForCoord(coord3.ref) shouldEqual Nil - } - - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset2 => - s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) s.map.shardsForCoord(coord4.ref) shouldEqual Nil - s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16) - s.map.shardsForCoord(coord3.ref) shouldEqual Nil } // dataset1: reassign shards 3,4,5 to coord2 and coord3 @@ -478,40 +468,83 @@ class ShardManagerSpec extends AkkaSpec { s.map.shardsForCoord(coord4.ref) shouldEqual Nil } - subscriber.expectMsgPF() { case s: CurrentShardSnapshot if s.ref == dataset1 => - s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) - s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) - s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5) - s.map.shardsForCoord(coord4.ref) shouldEqual Nil - } + expectNoMessage(subscriber) // ingestion should be stopped on downed node for 8 + 3 shards - coord4.receiveWhile(messages = 11) { case m: StopShardIngestion => m } should have length (11) - - // ingestion should failover to coord2 for dataset2 first, and get 8 StartShardIngestion messages - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj2.asCompactString - ds.source shouldEqual noOpSource2 - } - coord2.receiveWhile(messages = 8) { case m: StartShardIngestion => m } should have length (8) - - // then failover to coord2 for dataset1, with 1 shard - coord2.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - coord2.expectMsg(StartShardIngestion(dataset1, 3, None)) - - // ingestion should failover to coord3 for dataset1 - coord3.expectMsgPF() { case ds: DatasetSetup => - ds.compactDatasetStr shouldEqual datasetObj1.asCompactString - ds.source shouldEqual noOpSource1 - } - // 3 shards for dataset1 - coord3.expectMsgAllOf( - StartShardIngestion(dataset1, 4, None), - StartShardIngestion(dataset1, 5, None)) + { // coord1 + coord1.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset2 + s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) + s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + coord1.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5) + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + expectNoMessage(coord1) + } + + { // coord2 + expectDataset(coord2, datasetObj2) + + coord2.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset2 + s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) + s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + expectDataset(coord2, datasetObj1) + + coord2.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5) + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + expectNoMessage(coord2) + } + + { // coord3 + coord3.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset2 + s.map.shardsForCoord(coord1.ref) shouldEqual Range(0, 8) + s.map.shardsForCoord(coord2.ref) shouldEqual (8 until 16) + s.map.shardsForCoord(coord3.ref) shouldEqual Nil + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + expectDataset(coord3, datasetObj1) + + coord3.expectMsgPF() { + case s: ShardIngestionState => + s.ref shouldEqual dataset1 + s.map.shardsForCoord(coord1.ref) shouldEqual Seq(0, 1, 2) + s.map.shardsForCoord(coord2.ref) shouldEqual Seq(3, 6, 7) + s.map.shardsForCoord(coord3.ref) shouldEqual Seq(4, 5) + s.map.shardsForCoord(coord4.ref) shouldEqual Nil + } + + expectNoMessage(coord3) + } + + expectNoMessage(coord4) } } } From b2f4f76b3335b4ec25b54ac5f45c06910119ac80 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 4 Apr 2019 11:06:50 -0700 Subject: [PATCH 08/74] fix(cli): Make --limit actually behave like a user specified output limit (#307) * fix(cli): Make --limit actually behave like a user specified output limit * Change the docs for CLI --- README.md | 3 ++- cli/src/main/scala/filodb.cli/CliMain.scala | 12 ++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index f74c72f070..4f8b95c488 100644 --- a/README.md +++ b/README.md @@ -590,7 +590,8 @@ The `filo-cli` accepts arguments and options as key-value pairs, specified like | minutes | A shortcut to set the start at N minutes ago, and the stop at current time. Should specify a step also. | | chunks | Either "memory" or "buffers" to select either all the in-memory chunks or the write buffers only. Should specify a step also. | | database | Specifies the "database" the dataset should operate in. For Cassandra, this is the keyspace. If not specified, uses config value. | -| limit | The maximum number of samples per time series | +| limit | Limits the number of time series in the output | +| sampleLimit | Maximum number of output samples in the query result. An exception is thrown if the output returns more results than this. | | shards | (EXPERT) overrides the automatic shard calculation by passing in a comma-separated list of specific shards to query. Very useful to debug sharding issues. | | everyNSeconds | Repeats the query every (argument) seconds | | timeoutSeconds | The number of seconds for the network timeout | diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 790909cfb4..47b153b2f7 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -34,9 +34,9 @@ class Arguments extends FieldArgs { var rowKeys: Seq[String] = Seq("timestamp") var partitionKeys: Seq[String] = Nil var select: Option[Seq[String]] = None - // max # query items (vectors or tuples) returned. Don't make it too high. - var limit: Int = 1000 - var sampleLimit: Int = 200 + // max # of RangeVectors returned. Don't make it too high. + var limit: Int = 200 + var sampleLimit: Int = 1000000 var timeoutSeconds: Int = 60 var outfile: Option[String] = None var delimiter: String = "," @@ -364,7 +364,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste def executeQuery2(client: LocalClient, dataset: String, plan: LogicalPlan, options: QOptions): Unit = { val ref = DatasetRef(dataset) - val qOpts = QueryCommands.QueryOptions(options.spread, options.limit) + val qOpts = QueryCommands.QueryOptions(options.spread, options.sampleLimit) .copy(queryTimeoutSecs = options.timeout.toSeconds.toInt, shardOverrides = options.shardOverrides) println(s"Sending query command to server for $ref with options $qOpts...") @@ -373,7 +373,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste case Some(intervalSecs) => val fut = Observable.intervalAtFixedRate(intervalSecs.seconds).foreach { n => client.logicalPlan2Query(ref, plan, qOpts) match { - case QueryResult(_, schema, result) => result.foreach(rv => println(rv.prettyPrint())) + case QueryResult(_, schema, result) => result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case err: QueryError => throw new ClientException(err) } }.recover { @@ -385,7 +385,7 @@ object CliMain extends ArgMain[Arguments] with CsvImportExport with FilodbCluste try { client.logicalPlan2Query(ref, plan, qOpts) match { case QueryResult(_, schema, result) => println(s"Number of Range Vectors: ${result.size}") - result.foreach(rv => println(rv.prettyPrint())) + result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case QueryError(_,ex) => println(s"QueryError: ${ex.getClass.getSimpleName} ${ex.getMessage}") } } catch { From 9e6dc61b4e37dc0acdb6b314a759891b7fcfa8ba Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 4 Apr 2019 11:21:49 -0700 Subject: [PATCH 09/74] feat(query): aggr-over-vectors: Up to 2x speedup for no grouping (#297) --- .../NodeCoordinatorActorSpec.scala | 10 +-- core/src/main/resources/filodb-defaults.conf | 5 ++ .../scala/filodb.core/query/RangeVector.scala | 3 +- .../scala/filodb.core/query/ResultTypes.scala | 7 +- ...ala => QueryHiCardInMemoryBenchmark.scala} | 20 +++--- .../main/scala/filodb/query/QueryConfig.scala | 1 + .../query/exec/AggrOverRangeVectors.scala | 70 +++++++++++++++++-- .../query/exec/PeriodicSamplesMapper.scala | 14 ++-- .../exec/PeriodicSamplesMapperSpec.scala | 3 + 9 files changed, 103 insertions(+), 30 deletions(-) rename jmh/src/main/scala/filodb.jmh/{QueryLeafScanInMemoryBenchmark.scala => QueryHiCardInMemoryBenchmark.scala} (90%) diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 011ea417d4..8917bd051d 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -206,7 +206,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, q2) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => - schema shouldEqual timeMinSchema + schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2)) vectors should have length (1) vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0) } @@ -219,7 +219,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, q3) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => - schema shouldEqual countSchema + schema shouldEqual countSchema.copy(fixedVectorLen = Some(2)) vectors should have length (1) vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(98.0, 108.0) } @@ -233,7 +233,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, q4) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => - schema shouldEqual timeMinSchema + schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2)) vectors should have length (0) } } @@ -258,7 +258,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew (0 until numQueries).foreach { _ => probe.expectMsgPF() { case QueryResult(_, schema, vectors) => - schema shouldEqual timeMinSchema + schema shouldEqual timeMinSchema.copy(fixedVectorLen = Some(2)) vectors should have length (1) vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(14.0, 24.0) } @@ -385,7 +385,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.expectMsgPF() { case QueryResult(_, schema, vectors) => schema shouldEqual ResultSchema(Seq(ColumnInfo("GLOBALEVENTID", LongColumn), - ColumnInfo("AvgTone", DoubleColumn)), 1) + ColumnInfo("AvgTone", DoubleColumn)), 1, fixedVectorLen = Some(10)) vectors should have length (1) // vectors(0).rows.map(_.getDouble(1)).toSeq shouldEqual Seq(575.24) // TODO: verify if the expected results are right. They are something.... diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 3b24108270..4305b9a9ac 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -41,6 +41,11 @@ filodb { # Parallelism (query threadpool per dataset) ... ceil(available processors * factor) threads-factor = 1.0 + + # Maximum number of steps/windows to use the RangeVectorAggregator.fastReduce aggregators. This aggregator + # uses memory proportional to the # of windows, rather than the # of time series aggregated; it can speed up + # high cardinality aggregations in particular. + fastreduce-max-windows = 50 } shard-manager { diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index 01c5219e8a..00013ab7de 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -80,7 +80,8 @@ object CustomRangeVectorKey { UTF8Str(str) } - val emptyAsZcUtf8 = toZcUtf8(CustomRangeVectorKey(Map.empty)) + val empty = CustomRangeVectorKey(Map.empty) + val emptyAsZcUtf8 = toZcUtf8(empty) } /** diff --git a/core/src/main/scala/filodb.core/query/ResultTypes.scala b/core/src/main/scala/filodb.core/query/ResultTypes.scala index 4663c8d8cd..41e0f6ee0c 100644 --- a/core/src/main/scala/filodb.core/query/ResultTypes.scala +++ b/core/src/main/scala/filodb.core/query/ResultTypes.scala @@ -27,10 +27,13 @@ final case class ColumnInfo(name: String, colType: Column.ColumnType) /** * Describes the full schema of result types, including how many initial columns are for row keys. * The first ColumnInfo in the schema describes the first vector in Vectors and first field in Tuples, etc. - * @param brSchemas if any of the columns is a binary record, thsi + * @param brSchemas if any of the columns is a BinaryRecord: map of colNo -> inner BinaryRecord schema + * @param fixedVectorLen if defined, each vector is guaranteed to have exactly this many output elements. + * See PeriodicSampleMapper for an example of how this is used. */ final case class ResultSchema(columns: Seq[ColumnInfo], numRowKeyColumns: Int, - brSchemas: Map[Int, Seq[ColumnInfo]] = Map.empty) { + brSchemas: Map[Int, Seq[ColumnInfo]] = Map.empty, + fixedVectorLen: Option[Int] = None) { import Column.ColumnType._ def length: Int = columns.length diff --git a/jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala similarity index 90% rename from jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala rename to jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index 026b08897c..0836281ecd 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryLeafScanInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -24,10 +24,11 @@ import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex /** - * Benchmark for scan performed at the lowest leaf of an ExecPlan tree for various queries + * Benchmark for high-cardinality sum() aggregation, each query aggregates 2000 time series. + * Scan is performed at the lowest leaf of an ExecPlan tree for various queries */ @State(Scope.Thread) -class QueryLeafScanInMemoryBenchmark extends StrictLogging { +class QueryHiCardInMemoryBenchmark extends StrictLogging { org.slf4j.LoggerFactory.getLogger("filodb").asInstanceOf[Logger].setLevel(Level.WARN) import filodb.coordinator._ @@ -36,9 +37,9 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging { import NodeClusterActor._ val numShards = 1 - val numSeries = 200 - val samplesDuration = 2.hours - val numSamples = numSeries * (samplesDuration / 10.seconds).toInt + val numSeries = 8000 // NOTE: num of time series queried is this / 4 + val samplesDuration = 15.minutes + val numSamples = (samplesDuration / 10.seconds).toInt // # samples PER TIME SERIES val ingestionStartTime = System.currentTimeMillis - samplesDuration.toMillis val spread = 0 val config = ConfigFactory.load("filodb-defaults.conf") @@ -65,8 +66,8 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging { val storeConf = StoreConfig(ConfigFactory.parseString(""" | flush-interval = 1h - | shard-mem-size = 512MB - | ingestion-buffer-mem-size = 50MB + | shard-mem-size = 2GB + | ingestion-buffer-mem-size = 1GB | groups-per-shard = 4 | demand-paging-enabled = false """.stripMargin)) @@ -79,7 +80,6 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging { Thread sleep 2000 // Give setup command some time to set up dataset shards etc. val (producingFut, containerStream) = TestTimeseriesProducer.metricsToContainerStream(ingestionStartTime, numShards, numSeries, numSamples * numSeries, dataset, shardMapper, spread) - println(s"Ingesting $numSamples samples") val ingestTask = containerStream.groupBy(_._1) // Asynchronously subcribe and ingest each shard .mapAsync(numShards) { groupedStream => @@ -102,7 +102,7 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging { import filodb.coordinator.queryengine2.QueryEngine val engine = new QueryEngine(dataset, shardMapper) - val numQueries = 500 // Please make sure this number matches the OperationsPerInvocation below + val numQueries = 100 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalSec = samplesDuration.toSeconds // # minutes between start and stop val queryStep = 150 // # of seconds between each query sample "step" @@ -125,7 +125,7 @@ class QueryLeafScanInMemoryBenchmark extends StrictLogging { @Benchmark @BenchmarkMode(Array(Mode.AverageTime)) @OutputTimeUnit(TimeUnit.MICROSECONDS) - @OperationsPerInvocation(500) + @OperationsPerInvocation(100) def scanSumOfRateBenchmark(): Unit = { (0 until numQueries).foreach { _ => Await.result(scanSumOfRate.execute(store, dataset, queryConfig).runAsync, 60.seconds) diff --git a/query/src/main/scala/filodb/query/QueryConfig.scala b/query/src/main/scala/filodb/query/QueryConfig.scala index dafc80e685..807a46f141 100644 --- a/query/src/main/scala/filodb/query/QueryConfig.scala +++ b/query/src/main/scala/filodb/query/QueryConfig.scala @@ -13,4 +13,5 @@ class QueryConfig(queryConfig: Config) { lazy val askTimeout = queryConfig.as[FiniteDuration]("ask-timeout") lazy val staleSampleAfterMs = queryConfig.getDuration("stale-sample-after").toMillis lazy val minStepMs = queryConfig.getDuration("min-step").toMillis + lazy val fastReduceMaxWindows = queryConfig.getInt("fastreduce-max-windows") } diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 29e96190e0..dd9a34a5ce 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -7,6 +7,7 @@ import scala.collection.mutable import com.tdunning.math.stats.{ArrayDigest, TDigest} import com.typesafe.scalalogging.StrictLogging import monix.reactive.Observable +import scalaxy.loops._ import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Column.ColumnType @@ -65,6 +66,7 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, sourceSchema: ResultSchema): Observable[RangeVector] = { val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) val aggregator = RowAggregator(aggrOp, aggrParams, valColType) + def grouping(rv: RangeVector): RangeVectorKey = { val groupBy: Map[ZeroCopyUTF8String, ZeroCopyUTF8String] = if (by.nonEmpty) rv.key.labelValues.filter(lv => byLabels.contains(lv._1)) @@ -72,7 +74,17 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, else Map.empty CustomRangeVectorKey(groupBy) } - RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) + + // IF no grouping is done AND prev transformer is Periodic (has fixed length), use optimal path + if (without.isEmpty && by.isEmpty && sourceSchema.fixedVectorLen.isDefined) { + sourceSchema.fixedVectorLen.filter(_ <= queryConfig.fastReduceMaxWindows).map { numWindows => + RangeVectorAggregator.fastReduce(aggregator, false, source, numWindows) + }.getOrElse { + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) + } + } else { + RangeVectorAggregator.mapReduce(aggregator, skipMapPhase = false, source, grouping) + } } override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = { @@ -152,12 +164,11 @@ object RangeVectorAggregator extends StrictLogging { val mapInto = rowAgg.newRowToMapInto rvs.groupBy(grouping).mapValues { rvs => new Iterator[rowAgg.AggHolderType] { - val rowIterators = rvs.map(_.rows) - val rvKeys = rvs.map(_.key) - def hasNext: Boolean = rowIterators.forall(_.hasNext) + val itsAndKeys = rvs.map { rv => (rv.rows, rv.key) } + def hasNext: Boolean = itsAndKeys.forall(_._1.hasNext) def next(): rowAgg.AggHolderType = { acc.resetToZero() - rowIterators.zip(rvKeys).foreach { case (rowIter, rvk) => + itsAndKeys.foreach { case (rowIter, rvk) => val mapped = if (skipMapPhase) rowIter.next() else rowAgg.map(rvk, rowIter.next(), mapInto) acc = if (skipMapPhase) rowAgg.reduceAggregate(acc, mapped) else rowAgg.reduceMappedRow(acc, mapped) } @@ -166,6 +177,54 @@ object RangeVectorAggregator extends StrictLogging { } } } + + /** + * A fast reduce method intended specifically for the case when no grouping needs to be done AND + * the previous transformer is a PeriodicSampleMapper with fixed output lengths. + * It's much faster than mapReduce() since it iterates through each vector first and then from vector to vector. + * Time wise first iteration also uses less memory for high-cardinality use cases and reduces the + * time window of holding chunk map locks to each time series, instead of the entire query. + */ + def fastReduce(rowAgg: RowAggregator, + skipMapPhase: Boolean, + source: Observable[RangeVector], + outputLen: Int): Observable[RangeVector] = { + // Can't use an Array here because rowAgg.AggHolderType does not have a ClassTag + val accs = collection.mutable.ArrayBuffer.fill(outputLen)(rowAgg.zero) + var count = 0 + + // FoldLeft means we create the source PeriodicMapper etc and process immediately. We can release locks right away + // NOTE: ChunkedWindowIterator automatically releases locks after last window. So it should all just work. :) + val aggObs = if (skipMapPhase) { + source.foldLeftF(accs) { case (_, rv) => + count += 1 + val rowIter = rv.rows + for { i <- 0 until outputLen optimized } { + accs(i) = rowAgg.reduceAggregate(accs(i), rowIter.next) + } + accs + } + } else { + val mapIntos = Array.fill(outputLen)(rowAgg.newRowToMapInto) + source.foldLeftF(accs) { case (_, rv) => + count += 1 + val rowIter = rv.rows + for { i <- 0 until outputLen optimized } { + val mapped = rowAgg.map(rv.key, rowIter.next, mapIntos(i)) + accs(i) = rowAgg.reduceMappedRow(accs(i), mapped) + } + accs + } + } + + aggObs.flatMap { _ => + if (count > 0) { + Observable.now(new IteratorBackedRangeVector(CustomRangeVectorKey.empty, accs.toIterator.map(_.toRowReader))) + } else { + Observable.empty + } + } + } } trait AggregateHolder { @@ -199,6 +258,7 @@ trait RowAggregator { * Note that one object is used per aggregation. The returned object * is reused to aggregate each row-key of each RangeVector by resetting * before aggregation of next row-key. + * Should return a new AggHolder. */ def zero: AggHolderType diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index de0c9cae83..7153a14e12 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -88,13 +88,13 @@ final case class PeriodicSamplesMapper(start: Long, // Transform source double or long to double schema override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = source.copy(columns = source.columns.zipWithIndex.map { - // Transform if its not a row key column - case (ColumnInfo(name, ColumnType.LongColumn), i) if i >= source.numRowKeyColumns => - ColumnInfo(name, ColumnType.DoubleColumn) - case (ColumnInfo(name, ColumnType.IntColumn), i) if i >= source.numRowKeyColumns => - ColumnInfo(name, ColumnType.DoubleColumn) - case (c: ColumnInfo, _) => c - }) + // Transform if its not a row key column + case (ColumnInfo(name, ColumnType.LongColumn), i) if i >= source.numRowKeyColumns => + ColumnInfo(name, ColumnType.DoubleColumn) + case (ColumnInfo(name, ColumnType.IntColumn), i) if i >= source.numRowKeyColumns => + ColumnInfo(name, ColumnType.DoubleColumn) + case (c: ColumnInfo, _) => c + }, fixedVectorLen = Some(((end - start)/step).toInt + 1)) } /** diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala index d412c544d0..03ec1a07f8 100644 --- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -37,5 +37,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures resultRows.foreach(_.toList shouldEqual expectedResults) + val outSchema = periodicSamplesVectorFnMapper.schema(MetricsTestData.timeseriesDataset, resultSchema) + outSchema.columns shouldEqual resultSchema.columns + outSchema.fixedVectorLen shouldEqual Some(6) } } From 8fb0417c0f0d3601b381c2f948849a14b9681f76 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 4 Apr 2019 15:46:57 -0700 Subject: [PATCH 10/74] fix(memory, query): Fix exceptions in Histogram summing (#308) They were caused by some type misalignment. Now everything in aggregations is a HistogramWithBuckets. --- .../filodb.memory/format/vectors/Histogram.scala | 13 ++----------- .../format/vectors/HistogramVector.scala | 4 ++-- .../filodb/query/exec/AggrOverRangeVectors.scala | 9 ++++++--- .../main/scala/filodb/query/exec/TransientRow.scala | 4 ++-- .../query/exec/rangefn/AggrOverTimeFunctions.scala | 2 +- .../filodb/query/exec/rangefn/RangeFunction.scala | 2 +- .../query/exec/AggrOverRangeVectorsSpec.scala | 12 +++++++++++- 7 files changed, 25 insertions(+), 21 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala index c4cd7e0197..ee8ac39214 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -116,16 +116,7 @@ trait Histogram extends Ordered[Histogram] { } object Histogram { - val empty = new Histogram { - final def numBuckets: Int = 0 - final def bucketTop(no: Int): Double = ??? - final def bucketValue(no: Int): Double = ??? - final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { - val buf = intoBuf.getOrElse(BinaryHistogram.histBuf) - BinaryHistogram.writeNonIncreasing(HistogramBuckets.emptyBuckets, Array[Long](), buf) - buf - } - } + val empty = MutableHistogram(HistogramBuckets.emptyBuckets, Array.empty) } trait HistogramWithBuckets extends Histogram { @@ -176,7 +167,7 @@ final case class MutableHistogram(buckets: HistogramBuckets, values: Array[Doubl /** * Copies this histogram as a new copy so it can be used for aggregation or mutation. Allocates new storage. */ - final def copy: Histogram = MutableHistogram(buckets, values.clone) + final def copy: MutableHistogram = MutableHistogram(buckets, values.clone) /** * Adds the values from another Histogram. diff --git a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala index cc23454556..933d4e8bbb 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -279,7 +279,7 @@ class AppendableHistogramVector(factory: MemFactory, trait HistogramReader extends VectorDataReader { def buckets: HistogramBuckets - def apply(index: Int): Histogram + def apply(index: Int): HistogramWithBuckets def sum(start: Int, end: Int): MutableHistogram } @@ -372,7 +372,7 @@ class RowHistogramReader(histVect: Ptr.U8) extends HistogramReader { def length(addr: BinaryVectorPtr): Int = length // WARNING: histogram returned is shared between calls, do not reuse! - final def apply(index: Int): Histogram = { + final def apply(index: Int): HistogramWithBuckets = { require(length > 0) val histPtr = locate(index) val histLen = histPtr.asU16.getU16 diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index dd9a34a5ce..073e313e93 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -382,7 +382,8 @@ object SumRowAggregator extends RowAggregator { object HistSumRowAggregator extends RowAggregator { import filodb.memory.format.{vectors => bv} - class HistSumHolder(var timestamp: Long = 0L, var h: bv.Histogram = bv.Histogram.empty) extends AggregateHolder { + class HistSumHolder(var timestamp: Long = 0L, + var h: bv.MutableHistogram = bv.Histogram.empty) extends AggregateHolder { val row = new TransientHistRow() def toRowReader: MutableRowReader = { row.setValues(timestamp, h); row } def resetToZero(): Unit = h = bv.Histogram.empty @@ -393,10 +394,12 @@ object HistSumRowAggregator extends RowAggregator { def map(rvk: RangeVectorKey, item: RowReader, mapInto: MutableRowReader): RowReader = item def reduceAggregate(acc: HistSumHolder, aggRes: RowReader): HistSumHolder = { acc.timestamp = aggRes.getLong(0) + val newHist = aggRes.getHistogram(1) acc.h match { // sum is mutable histogram, copy to be sure it's our own copy - case hist if hist.numBuckets == 0 => acc.h = bv.MutableHistogram(aggRes.getHistogram(1)) - case hist: bv.MutableHistogram => hist.add(aggRes.getHistogram(1).asInstanceOf[bv.HistogramWithBuckets]) + case hist if hist.numBuckets == 0 => acc.h = bv.MutableHistogram(newHist) + case h if newHist.numBuckets > 0 => acc.h.add(newHist.asInstanceOf[bv.HistogramWithBuckets]) + case h => } acc } diff --git a/query/src/main/scala/filodb/query/exec/TransientRow.scala b/query/src/main/scala/filodb/query/exec/TransientRow.scala index adf3bd7da5..0a8aeb9b7b 100644 --- a/query/src/main/scala/filodb/query/exec/TransientRow.scala +++ b/query/src/main/scala/filodb/query/exec/TransientRow.scala @@ -56,8 +56,8 @@ final class TransientRow(var timestamp: Long, var value: Double) extends Mutable } final class TransientHistRow(var timestamp: Long = 0L, - var value: bv.Histogram = bv.Histogram.empty) extends MutableRowReader { - def setValues(ts: Long, hist: bv.Histogram): Unit = { + var value: bv.HistogramWithBuckets = bv.Histogram.empty) extends MutableRowReader { + def setValues(ts: Long, hist: bv.HistogramWithBuckets): Unit = { timestamp = ts value = hist } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala index 27750e67a9..2d71c8f0c2 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -166,7 +166,7 @@ class SumOverTimeChunkedFunctionL extends SumOverTimeChunkedFunction() with Chun } } -class SumOverTimeChunkedFunctionH(var h: bv.Histogram = bv.Histogram.empty) +class SumOverTimeChunkedFunctionH(var h: bv.MutableHistogram = bv.Histogram.empty) extends ChunkedRangeFunction[TransientHistRow] { override final def reset(): Unit = { h = bv.Histogram.empty } final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala index a767c35ea4..2520f5ff1d 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -315,7 +315,7 @@ extends LastSampleChunkedFunction[TransientRow] { } // LastSample function for Histogram columns -class LastSampleChunkedFunctionH(var value: bv.Histogram = bv.Histogram.empty) +class LastSampleChunkedFunctionH(var value: bv.HistogramWithBuckets = bv.Histogram.empty) extends LastSampleChunkedFunction[TransientHistRow] { override final def reset(): Unit = { timestamp = -1L; value = bv.Histogram.empty } final def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = { diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 65c6ee5cae..f926d00118 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -344,7 +344,9 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val samples: Array[RangeVector] = Array(rv1, rv2) val agg1 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.HistogramColumn) - val resultObs = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) + val resultObs1 = RangeVectorAggregator.mapReduce(agg1, false, Observable.fromIterable(samples), noGrouping) + val resultObs = RangeVectorAggregator.mapReduce(agg1, true, resultObs1, rv=>rv.key) + val result = resultObs.toListL.runAsync.futureValue result.size shouldEqual 1 result(0).key shouldEqual noKey @@ -356,6 +358,14 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { }.toList result(0).rows.map(_.getHistogram(1)).toList shouldEqual sums + + // Test mapReduce of empty histogram sums + val agg2 = RowAggregator(AggregationOperator.Sum, Nil, ColumnType.HistogramColumn) + val emptyObs = RangeVectorAggregator.mapReduce(agg2, false, Observable.empty, noGrouping) + val resultObs2 = RangeVectorAggregator.mapReduce(agg2, true, emptyObs ++ resultObs1, rv=>rv.key) + val result2 = resultObs2.toListL.runAsync.futureValue + result2.size shouldEqual 1 + result2(0).key shouldEqual noKey } @tailrec From b086617d21edfe0af97b7bad3b22391054e0c132 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 5 Apr 2019 13:06:04 -0700 Subject: [PATCH 11/74] bug(coordinator): Fix thread safety issue with sharing of ShardMapper instances. (#311) --- .../scala/filodb.coordinator/NodeCoordinatorActor.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index f6e2e2a377..448ee33e1a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -1,5 +1,7 @@ package filodb.coordinator +import java.util.concurrent.ConcurrentHashMap + import scala.collection.mutable.HashMap import scala.concurrent.duration._ @@ -57,7 +59,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, val ingesters = new HashMap[DatasetRef, ActorRef] val queryActors = new HashMap[DatasetRef, ActorRef] var clusterActor: Option[ActorRef] = None - val shardMaps = new HashMap[DatasetRef, ShardMapper] + val shardMaps = new ConcurrentHashMap[DatasetRef, ShardMapper] var statusActor: Option[ActorRef] = None private val statusAckTimeout = config.as[FiniteDuration]("tasks.timeouts.status-ack-timeout") @@ -136,7 +138,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, ingesters(ref) = ingester logger.info(s"Creating QueryActor for dataset $ref") - val queryRef = context.actorOf(QueryActor.props(memStore, dataset, shardMaps(ref)), s"$Query-$ref") + val queryRef = context.actorOf(QueryActor.props(memStore, dataset, shardMaps.get(ref)), s"$Query-$ref") nca.tell(SubscribeShardUpdates(ref), self) queryActors(ref) = queryRef @@ -188,7 +190,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, case NodeProtocol.ResetState => reset(sender()) case CurrentShardSnapshot(ds, mapper) => logger.debug(s"Received ShardSnapshot $mapper") - shardMaps(ds) = mapper + shardMaps.put(ds, mapper) // NOTE: QueryActor has AtomicRef so no need to forward message to it } From b4745870a69e66311c05007e64ec3ff0eb100db3 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Mon, 8 Apr 2019 11:53:12 -0700 Subject: [PATCH 12/74] feat(coordinator): Publish shard mapping snapshots once a minute, to cope with dropped messages. (#310) --- .../main/scala/filodb.coordinator/NodeClusterActor.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala index 137850bffb..2464eb6bea 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala @@ -206,7 +206,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, val shardManager = new ShardManager(settings, assignmentStrategy) val localRemoteAddr = RemoteAddressExtension(context.system).address var everybodyLeftSender: Option[ActorRef] = None - val shardUpdates = new MutableHashSet[DatasetRef] // Counter is incremented each time shardmapper snapshot is published. // value > 0 implies that the node is a ShardManager. For rest of the nodes metric will not be reported. @@ -350,7 +349,7 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, // handleEventEnvelope() currently acks right away, so there is a chance that this actor dies between receiving // a new event and the new snapshot is published. private def scheduleSnapshotPublishes() = { - pubTask = Some(context.system.scheduler.schedule(1.second, publishInterval, self, PublishSnapshot)) + pubTask = Some(context.system.scheduler.schedule(1.minute, publishInterval, self, PublishSnapshot)) } def shardMapHandler: Receive = LoggingReceive { @@ -364,11 +363,10 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, def subscriptionHandler: Receive = LoggingReceive { case e: ShardEvent => handleShardEvent(e) case e: StatusActor.EventEnvelope => handleEventEnvelope(e, sender()) - case PublishSnapshot => shardUpdates.foreach(shardManager.publishSnapshot) + case PublishSnapshot => datasets.keys.foreach(shardManager.publishSnapshot) //This counter gets published from ShardManager, // > 0 means this node is shardmanager iamShardManager.increment() - shardUpdates.clear() case e: SubscribeShardUpdates => subscribe(e.ref, sender()) case SubscribeAll => subscribeAll(sender()) case Terminated(subscriber) => context unwatch subscriber @@ -376,7 +374,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, private def handleShardEvent(e: ShardEvent) = { logger.debug(s"Received ShardEvent $e from $sender") - shardUpdates += e.ref shardManager.updateFromExternalShardEvent(sender(), e) } @@ -466,7 +463,6 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, logger.info("Resetting all dataset state except membership.") datasets.clear() sources.clear() - shardUpdates.clear() implicit val timeout: Timeout = DefaultTaskTimeout shardManager.reset() From 2764a0aec4fd206ea05507aa0d614012a00e64e8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 8 Apr 2019 11:54:13 -0700 Subject: [PATCH 13/74] fix(core): Switch to thread-safe Queue in WriteBufferPool (#309) --- .../scala/filodb.core/memstore/WriteBufferPool.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala index 67ce93b065..17b00e1dbf 100644 --- a/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala +++ b/core/src/main/scala/filodb.core/memstore/WriteBufferPool.scala @@ -1,6 +1,7 @@ package filodb.core.memstore import com.typesafe.scalalogging.StrictLogging +import org.jctools.queues.MpscUnboundedArrayQueue import scalaxy.loops._ import filodb.core.metadata.Dataset @@ -29,7 +30,7 @@ class WriteBufferPool(memFactory: MemFactory, storeConf: StoreConfig) extends StrictLogging { import TimeSeriesPartition._ - val queue = new collection.mutable.Queue[(NativePointer, AppenderArray)] + val queue = new MpscUnboundedArrayQueue[(NativePointer, AppenderArray)](storeConf.allocStepSize * 10) private def allocateBuffers(): Unit = { logger.debug(s"Allocating ${storeConf.allocStepSize} WriteBuffers....") @@ -41,14 +42,14 @@ class WriteBufferPool(memFactory: MemFactory, for { colNo <- 0 until dataset.numDataColumns optimized } { ChunkSetInfo.setVectorPtr(info.infoAddr, colNo, builders(colNo).addr) } - queue.enqueue((info.infoAddr, builders)) + queue.add((info.infoAddr, builders)) } } /** * Returns the number of allocatable sets of buffers in the pool */ - def poolSize: Int = queue.length + def poolSize: Int = queue.size /** * Obtains a new set of AppendableVectors from the pool, creating additional buffers if there is memory available. @@ -59,7 +60,7 @@ class WriteBufferPool(memFactory: MemFactory, def obtain(): (NativePointer, AppenderArray) = { // If queue is empty, try and allocate more buffers depending on if memFactory has more memory if (queue.isEmpty) allocateBuffers() - queue.dequeue + queue.remove() } /** @@ -71,7 +72,7 @@ class WriteBufferPool(memFactory: MemFactory, // (in case some reader is still hanging on to this old info) ChunkSetInfo.resetNumRows(metaAddr) appenders.foreach(_.reset()) - queue.enqueue((metaAddr, appenders)) + queue.add((metaAddr, appenders)) // TODO: check number of buffers in queue, and release baack to free memory. // NOTE: no point to this until the pool shares a single MemFactory amongst multiple shards. In that case // we have to decide (w/ concurrency a concern): share a single MemFactory or a single WriteBufferPool? From 72c995147bdd1ed0fb71b570a2f27d42e3465af2 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 9 Apr 2019 10:12:27 -0700 Subject: [PATCH 14/74] feat(query): Allow metric name as tag in promQL; Allow escape chars in tag values (#313) * Now allow metric name to be provided as a tag in promQL. Queries such as {__name__="foo", baz="boo" } are supported. * Allow special characters in tag values. This allows possibility of special characters in metric names too. --- .../filodb/prometheus/ast/Functions.scala | 8 +-- .../scala/filodb/prometheus/ast/Vectors.scala | 62 ++++++++++--------- .../filodb/prometheus/parse/Parser.scala | 24 +++++-- .../filodb/prometheus/parse/ParserSpec.scala | 12 +++- 4 files changed, 65 insertions(+), 41 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala index 5270bd1228..0db0ae778e 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala @@ -17,13 +17,11 @@ trait Functions extends Base with Operators with Vectors { def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { val seriesParam = allParams.filter(_.isInstanceOf[Series]).head.asInstanceOf[Series] - val otherParams = allParams.filter(!_.equals(seriesParam)).map(_ match { + val otherParams = allParams.filter(!_.equals(seriesParam)).map { case num: ScalarExpression => num.toScalar - case s: InstantExpression => s.metricName.replaceAll("^\"|\"$", "") - case _ => - throw new IllegalArgumentException("Parameters can be a string or number") + case s: InstantExpression => s.realMetricName.replaceAll("^\"|\"$", "") + case _ => throw new IllegalArgumentException("Parameters can be a string or number") } - ) val instantFunctionIdOpt = InstantFunctionId.withNameInsensitiveOption(name) val filoFunctionIdOpt = FiloFunctionId.withNameInsensitiveOption(name) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala index 374c43aa44..ff77af19f2 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Vectors.scala @@ -84,13 +84,30 @@ trait Vectors extends Scalars with TimeUnits with Base { } sealed trait Vector extends Expression { + + def metricName: Option[String] + def labelSelection: Seq[LabelMatch] + + def realMetricName: String = { + val nameLabelValue = labelSelection.find(_.label == PromMetricLabel).map(_.value) + if (nameLabelValue.nonEmpty && metricName.nonEmpty) { + throw new IllegalArgumentException("Metric name should not be set twice") + } + metricName.orElse(nameLabelValue) + .getOrElse(throw new IllegalArgumentException("Metric name is not present")) + } + protected def labelMatchesToFilters(labels: Seq[LabelMatch]) = labels.map { labelMatch => - labelMatch.labelMatchOp match { - case EqualMatch => ColumnFilter(labelMatch.label, query.Filter.Equals(labelMatch.value)) - case NotRegexMatch => ColumnFilter(labelMatch.label, query.Filter.NotEqualsRegex(labelMatch.value)) - case RegexMatch => ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelMatch.value)) - case NotEqual(false) => ColumnFilter(labelMatch.label, query.Filter.NotEquals(labelMatch.value)) + val labelValue = labelMatch.value.replace("\\\\", "\\") + .replace("\\\"", "\"") + .replace("\\n", "\n") + .replace("\\t", "\t") + labelMatch.labelMatchOp match { + case EqualMatch => ColumnFilter(labelMatch.label, query.Filter.Equals(labelValue)) + case NotRegexMatch => ColumnFilter(labelMatch.label, query.Filter.NotEqualsRegex(labelValue)) + case RegexMatch => ColumnFilter(labelMatch.label, query.Filter.EqualsRegex(labelValue)) + case NotEqual(false) => ColumnFilter(labelMatch.label, query.Filter.NotEquals(labelValue)) case other: Any => throw new IllegalArgumentException(s"Unknown match operator $other") } // Remove the column selector as that is not a real time series filter @@ -114,21 +131,16 @@ trait Vectors extends Scalars with TimeUnits with Base { * appending a set of labels to match in curly braces ({}). */ - case class InstantExpression(metricName: String, - labelSelection: Seq[LabelMatch], + case class InstantExpression(override val metricName: Option[String], + override val labelSelection: Seq[LabelMatch], offset: Option[Duration]) extends Vector with PeriodicSeries { - val staleDataLookbackSeconds = 5 * 60 // 5 minutes - - private val nameLabels = labelSelection.filter(_.label == PromMetricLabel) - if (nameLabels.nonEmpty && !nameLabels.head.label.equals(metricName)) { - throw new IllegalArgumentException("Metric name should not be set twice") - } + val staleDataLookbackSeconds = 5 * 60 // 5 minutes private[prometheus] val columnFilters = labelMatchesToFilters(labelSelection) private[prometheus] val columns = labelMatchesToColumnName(labelSelection) - private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(metricName)) - def getColFilters: Seq[ColumnFilter] = columnFilters :+ nameFilter + private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(realMetricName)) + def getColFilters: Seq[ColumnFilter] = if (metricName.isDefined) columnFilters :+ nameFilter else columnFilters def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { @@ -140,12 +152,9 @@ trait Vectors extends Scalars with TimeUnits with Base { timeParams.start * 1000, timeParams.step * 1000, timeParams.end * 1000 ) } - } - case class MetadataExpression(instantExpression: InstantExpression) extends Vector with Metadata { - - override def toMetadataQueryPlan(timeParams: TimeRangeParams): MetadataQueryPlan = { - SeriesKeysByFilters(instantExpression.getColFilters, timeParams.start * 1000, timeParams.end * 1000) + def toMetadataPlan(timeParams: TimeRangeParams): SeriesKeysByFilters = { + SeriesKeysByFilters(getColFilters, timeParams.start * 1000, timeParams.end * 1000) } } @@ -156,21 +165,16 @@ trait Vectors extends Scalars with TimeUnits with Base { * at the end of a vector selector to specify how far back in time values * should be fetched for each resulting range vector element. */ - case class RangeExpression(metricName: String, - labelSelection: Seq[LabelMatch], + case class RangeExpression(override val metricName: Option[String], + override val labelSelection: Seq[LabelMatch], window: Duration, offset: Option[Duration]) extends Vector with SimpleSeries { - private val nameLabels = labelSelection.filter(_.label == PromMetricLabel) - - if (nameLabels.nonEmpty && !nameLabels.head.label.equals(metricName)) { - throw new IllegalArgumentException("Metric name should not be set twice") - } private[prometheus] val columnFilters = labelMatchesToFilters(labelSelection) private[prometheus] val columns = labelMatchesToColumnName(labelSelection) - private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(metricName)) + private[prometheus] val nameFilter = ColumnFilter(PromMetricLabel, query.Filter.Equals(realMetricName)) - val allFilters: Seq[ColumnFilter] = columnFilters :+ nameFilter + val allFilters: Seq[ColumnFilter] = if (metricName.isDefined) columnFilters :+ nameFilter else columnFilters def toRawSeriesPlan(timeParams: TimeRangeParams, isRoot: Boolean): RawSeriesPlan = { if (isRoot && timeParams.start != timeParams.end) { diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 1bfaff3537..7a110b6ae7 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -194,22 +194,36 @@ trait Join extends Numeric { ////////////////////// SELECTORS /////////////////////////////////////////// trait Selector extends Operator with Unit with BaseParser { protected lazy val simpleSeries: PackratParser[InstantExpression] = - "([\"'])(?:\\\\\\1|.)*?\\1".r ^^ { str => InstantExpression(str, Seq.empty, None) } + "([\"'])(?:\\\\\\1|.)*?\\1".r ^^ { str => InstantExpression(Some(str), Seq.empty, None) } lazy val instantVectorSelector: PackratParser[InstantExpression] = labelIdentifier ~ labelSelection.? ~ offset.? ^^ { case metricName ~ ls ~ opt => - InstantExpression(metricName.str, ls.getOrElse(Seq.empty), opt.map(_.duration)) + InstantExpression(Some(metricName.str), ls.getOrElse(Seq.empty), opt.map(_.duration)) + } + + lazy val instantVectorSelector2: PackratParser[InstantExpression] + = labelSelection ~ offset.? ^^ { + case ls ~ opt => + InstantExpression(None, ls, opt.map(_.duration)) } lazy val rangeVectorSelector: PackratParser[RangeExpression] = labelIdentifier ~ labelSelection.? ~ "[" ~ duration ~ "]" ~ offset.? ^^ { case metricName ~ ls ~ leftBracket ~ td ~ rightBracket ~ opt => - RangeExpression(metricName.str, ls.getOrElse(Seq.empty), td, opt.map(_.duration)) + RangeExpression(Some(metricName.str), ls.getOrElse(Seq.empty), td, opt.map(_.duration)) } - lazy val vector: PackratParser[Vector] = rangeVectorSelector | instantVectorSelector + lazy val rangeVectorSelector2: PackratParser[RangeExpression] = + labelSelection ~ "[" ~ duration ~ "]" ~ offset.? ^^ { + case ls ~ leftBracket ~ td ~ rightBracket ~ opt => + RangeExpression(None, ls, td, opt.map(_.duration)) + } + + lazy val vector: PackratParser[Vector] = + rangeVectorSelector2 | rangeVectorSelector | instantVectorSelector2 | instantVectorSelector + } ////////////////////// END SELECTORS /////////////////////////////////////////// @@ -319,7 +333,7 @@ object Parser extends Expression { def metadataQueryToLogicalPlan(query: String, timeParams: TimeRangeParams): LogicalPlan = { val expression = parseQuery(query) expression match { - case p: InstantExpression => MetadataExpression(p).toMetadataQueryPlan(timeParams) + case p: InstantExpression => p.toMetadataPlan(timeParams) case _ => throw new UnsupportedOperationException() } } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index ad3ecfb7f6..54b78d7886 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -265,7 +265,7 @@ class ParserSpec extends FunSpec with Matchers { "sum(http_requests_total) without (instance)" -> "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List(instance))", "count_values(\"version\", build_version)" -> - "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(\"version\"),List(),List())", + "Aggregate(CountValues,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(build_version))),List()),1524855988000,1000000,1524855988000),List(Some(\"version\")),List(),List())", "label_replace(up{job=\"api-server\",service=\"a:c\"}, \"foo\", \"$1\", \"service\", \"(.*):.*\")" -> "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(service,Equals(a:c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelReplace,List(foo, $1, service, (.*):.*))", "sum(http_requests_total)" -> @@ -321,7 +321,15 @@ class ParserSpec extends FunSpec with Matchers { "absent(nonexistent{job=\"myjob\",instance=~\".*\"})" -> "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(instance,EqualsRegex(.*)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),Absent,List())", "absent(sum(nonexistent{job=\"myjob\"}))" -> - "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),Absent,List())" + "ApplyInstantFunction(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),Absent,List())", + """{__name__="foo\\\"\n\t",job="myjob"}[5m]""" -> + "RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo\\\"\n\t)), ColumnFilter(job,Equals(myjob))),List())", + "{__name__=\"foo\",job=\"myjob\"}" -> + "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List()),1524855988000,1000000,1524855988000)", + "{__name__=\"foo\",job=\"myjob\"}[5m]" -> + "RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List())", + "sum({__name__=\"foo\",job=\"myjob\"})" -> + "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo)), ColumnFilter(job,Equals(myjob))),List()),1524855988000,1000000,1524855988000),List(),List(),List())" ) val qts: Long = 1524855988L From 18f3bda463037423a48dc1d08352475b925eb091 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 10 Apr 2019 15:37:56 -0700 Subject: [PATCH 15/74] bug(query): binary join with ignoring and on bug fix (#315) Extract "ignoring" and "on" labels from expression when specified and use it to create BinaryJoin logical plan --- .../filodb/prometheus/ast/Expressions.scala | 9 +- .../filodb/prometheus/parse/ParserSpec.scala | 13 ++- .../query/exec/BinaryJoinExecSpec.scala | 83 +++++++++++++++++++ 3 files changed, 100 insertions(+), 5 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala index 6199fc6891..965e810d13 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Expressions.scala @@ -47,7 +47,14 @@ trait Expressions extends Aggregates with Functions { val seriesPlanLhs = series.toPeriodicSeriesPlan(timeParams) val seriesPlanRhs = rhs.asInstanceOf[PeriodicSeries].toPeriodicSeriesPlan(timeParams) val cardinality = vectorMatch.map(_.cardinality.cardinality).getOrElse(Cardinality.OneToOne) - BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs) + + val matcher = vectorMatch.flatMap(_.matching) + val onLabels = matcher.filter(_.isInstanceOf[On]).map(_.labels) + val ignoringLabels = matcher.filter(_.isInstanceOf[Ignoring]).map(_.labels) + + BinaryJoin(seriesPlanLhs, operator.getPlanOperator, cardinality, seriesPlanRhs, + onLabels.getOrElse(Nil), ignoringLabels.getOrElse(Nil)) + } } } diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 54b78d7886..b65b964eb3 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -4,7 +4,6 @@ import org.scalatest.{FunSpec, Matchers} import filodb.prometheus.ast.TimeStepParams - //noinspection ScalaStyle // scalastyle:off class ParserSpec extends FunSpec with Matchers { @@ -276,8 +275,12 @@ class ParserSpec extends FunSpec with Matchers { "ApplyInstantFunction(Aggregate(Sum,PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),List(),List(le),List()),HistogramQuantile,List(0.9))", "delta(cpu_temp_celsius{host=\"zeus\"}[2h])" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524848788000,1524855988000),List(ColumnFilter(host,Equals(zeus)), ColumnFilter(__name__,Equals(cpu_temp_celsius))),List()),1524855988000,1000000,1524855988000,7200000,Delta,List())", - "method_code:http_errors:rate5m{code=\"500\"} / ignoring(code) method:http_requests:rate5m" -> + "method_code:http_errors:rate5m{code=\"500\"} / method:http_requests:rate5m" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List())", + "method_code:http_errors:rate5m{code=\"500\"} / ignoring(code) method:http_requests:rate5m" -> + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List(code))", + "method_code:http_errors:rate5m{code=\"500\"} / on(method) method:http_requests:rate5m" -> + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(code,Equals(500)), ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(method),List())", "histogram_quantile(0.9, rate(http_request_duration_seconds_bucket[10m]))" -> "ApplyInstantFunction(PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855388000,1524855988000),List(ColumnFilter(__name__,Equals(http_request_duration_seconds_bucket))),List()),1524855988000,1000000,1524855988000,600000,Rate,List()),HistogramQuantile,List(0.9))", "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (job, le))" -> @@ -312,8 +315,10 @@ class ParserSpec extends FunSpec with Matchers { "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", "http_requests_total{environment=~\"staging|testing|development\",method!=\"GET\"}" -> "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(environment,EqualsRegex(staging|testing|development)), ColumnFilter(method,NotEquals(GET)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", + "method_code:http_errors:rate5m / ignoring(code) group_left method:http_requests:rate5m" -> - "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List())", + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method_code:http_errors:rate5m))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(method:http_requests:rate5m))),List()),1524855988000,1000000,1524855988000),List(),List(code))", + "increase(http_requests_total{job=\"api-server\"}[5m])" -> "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Increase,List())", "sum(http_requests_total{method=\"GET\"} offset 5m)" -> @@ -339,7 +344,7 @@ class ParserSpec extends FunSpec with Matchers { lp.toString shouldEqual (e) } } - + private def parseSuccessfully(query: String) = { Parser.parseQuery(query) } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index 3f458f50b3..e791fc90b8 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -53,6 +53,27 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { } } + val samplesLhsGrouping: Array[RangeVector] = Array.tabulate(2) { i => + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricLhs".utf8, + "tag1".utf8 -> s"tag1-$i".utf8, + "tag2".utf8 -> s"tag2-1".utf8, + "job".utf8 -> s"somejob".utf8)) + val rows: Iterator[RowReader] = data(i).iterator + } + } + + val samplesRhsGrouping: Array[RangeVector] = Array.tabulate(2) { i => + new RangeVector { + val key: RangeVectorKey = CustomRangeVectorKey( + Map("__name__".utf8 -> s"someMetricRhs".utf8, + "tag1".utf8 -> s"tag1-$i".utf8, + "job".utf8 -> s"somejob".utf8)) + val rows: Iterator[RowReader] = data(i).iterator + } + } + it("should join one-to-one without on or ignoring") { val samplesRhs2 = scala.util.Random.shuffle(samplesRhs.toList) // they may come out of order @@ -185,4 +206,66 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { e shouldBe a[BadQueryException] } } + it("should join one-to-one with ignoring") { + + val execPlan = BinaryJoinExec("someID", dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToOne, + Nil, Seq("tag2")) + + val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + + // scalastyle:off + val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializableRangeVector(rv, schema))) + // val lhs = QueryResult("someId", null, samplesLhs.filter(rv => rv.key.labelValues.get(ZeroCopyUTF8String("tag2")).get.equals("tag1-1")).map(rv => SerializableRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializableRangeVector(rv, schema))) + // scalastyle:on + // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account + val result = execPlan.compose(dataset, Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig) + .toListL.runAsync.futureValue + + result.foreach { rv => + rv.key.labelValues.contains("__name__".utf8) shouldEqual false + rv.key.labelValues.contains("tag1".utf8) shouldEqual true + rv.key.labelValues.contains("tag2".utf8) shouldEqual false + val i = rv.key.labelValues("tag1".utf8).asNewString.split("-")(1) + rv.rows.map(_.getDouble(1)).foreach(_ shouldEqual i.toDouble * 2) + } + + result.map(_.key).toSet.size shouldEqual 2 + } + + it("should join one-to-one with on") { + + val execPlan = BinaryJoinExec("someID", dummyDispatcher, + Array(dummyPlan), // cannot be empty as some compose's rely on the schema + new Array[ExecPlan](1), // empty since we test compose, not execute or doExecute + BinaryOperator.ADD, + Cardinality.OneToOne, + Seq("tag1", "job"), Nil) + + val schema = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + + // scalastyle:off + val lhs = QueryResult("someId", null, samplesLhsGrouping.map(rv => SerializableRangeVector(rv, schema))) + val rhs = QueryResult("someId", null, samplesRhsGrouping.map(rv => SerializableRangeVector(rv, schema))) + // scalastyle:on + // note below that order of lhs and rhs is reversed, but index is right. Join should take that into account + val result = execPlan.compose(dataset, Observable.fromIterable(Seq((rhs, 1), (lhs, 0))), queryConfig) + .toListL.runAsync.futureValue + + result.foreach { rv => + rv.key.labelValues.contains("__name__".utf8) shouldEqual false + rv.key.labelValues.contains("tag1".utf8) shouldEqual true + rv.key.labelValues.contains("tag2".utf8) shouldEqual false + val i = rv.key.labelValues("tag1".utf8).asNewString.split("-")(1) + rv.rows.map(_.getDouble(1)).foreach(_ shouldEqual i.toDouble * 2) + } + + result.map(_.key).toSet.size shouldEqual 2 + } } From c4f8124177a9ff111a0454bb865056fcd28202d7 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 12 Apr 2019 11:06:38 -0700 Subject: [PATCH 16/74] bug(core): startTimes Lucene query needs batching (#317) There is a limit to number of terms in a query, so we need batching to prevent exception when startTime for too many partIds needs to be fetched. --- .../filodb.core/memstore/PartKeyLuceneIndex.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 282c62b315..bff65b9505 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -326,11 +326,13 @@ class PartKeyLuceneIndex(dataset: Dataset, */ def startTimeFromPartIds(partIds: Iterator[Int]): debox.Map[Int, Long] = { val collector = new PartIdStartTimeCollector() - val booleanQuery = new BooleanQuery.Builder - partIds.foreach { pId => - booleanQuery.add(new TermQuery(new Term(PART_ID, pId.toString)), Occur.SHOULD) + partIds.grouped(512).map { batch => // limit on query clause count is 1024, hence batch + val booleanQuery = new BooleanQuery.Builder + batch.foreach { pId => + booleanQuery.add(new TermQuery(new Term(PART_ID, pId.toString)), Occur.SHOULD) + } + searcherManager.acquire().search(booleanQuery.build(), collector) } - searcherManager.acquire().search(booleanQuery.build(), collector) collector.startTimes } From a607c6b4e971b3de2269d1b5b4d92dc8fe08a650 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Fri, 12 Apr 2019 11:12:54 -0700 Subject: [PATCH 17/74] misc(core): Remove code for old chunk id encoding format. (#314) --- .../memstore/TimeSeriesPartition.scala | 2 +- .../filodb.core/store/ChunkSetInfo.scala | 2 +- .../scala/filodb.core/store/package.scala | 33 ++----------------- 3 files changed, 5 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala index 34bd7ce270..bf93e7027c 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesPartition.scala @@ -121,7 +121,7 @@ extends ChunkMap(memFactory, initMapSize) with ReadablePartition { if (newChunk) { // First row of a chunk, set the start time to it val (infoAddr, newAppenders) = bufferPool.obtain() - val currentChunkID = newChunkID(ts) + val currentChunkID = chunkID(ts) ChunkSetInfo.setChunkID(infoAddr, currentChunkID) ChunkSetInfo.resetNumRows(infoAddr) // Must reset # rows otherwise it keeps increasing! ChunkSetInfo.setStartTime(infoAddr, ts) diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index 3de4c53bfa..d76bf9397b 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -41,7 +41,7 @@ object ChunkSet { def apply(dataset: Dataset, part: PartitionKey, rows: Seq[RowReader], factory: MemFactory): ChunkSet = { require(rows.nonEmpty) val startTime = dataset.timestamp(rows.head) - val info = ChunkSetInfo(factory, dataset, newChunkID(startTime), rows.length, + val info = ChunkSetInfo(factory, dataset, chunkID(startTime), rows.length, startTime, dataset.timestamp(rows.last)) val filoSchema = Column.toFiloSchema(dataset.dataColumns) diff --git a/core/src/main/scala/filodb.core/store/package.scala b/core/src/main/scala/filodb.core/store/package.scala index 4b89bb782c..432cf3eef3 100644 --- a/core/src/main/scala/filodb.core/store/package.scala +++ b/core/src/main/scala/filodb.core/store/package.scala @@ -2,26 +2,18 @@ package filodb.core import java.nio.ByteBuffer -import com.github.rholder.fauxflake.IdGenerators import net.jpountz.lz4.{LZ4Compressor, LZ4Factory, LZ4FastDecompressor} import filodb.core.Types._ import filodb.core.metadata.Dataset -import filodb.core.SingleKeyTypes.Long64HighBit import filodb.memory.format.{RowReader, UnsafeUtils} package object store { val compressor = new ThreadLocal[LZ4Compressor]() val decompressor = new ThreadLocal[LZ4FastDecompressor]() - val machineIdLong = IdGenerators.newSnowflakeIdGenerator.generateId(1) - val machineId1024 = (machineIdLong.asLong >> 12) & (0x03ff) val msBitOffset = 21 - val machIdBitOffset = 11 - val baseNsBitOffset = 9 // 2 ** 9 = 512 - val nanoBitMask = Math.pow(2, machIdBitOffset).toInt - 1 - val lowerBitsMask = Math.pow(2, msBitOffset).toInt - 1 - val baseTimeMillis = org.joda.time.DateTime.parse("2016-01-01T00Z").getMillis + val lowerBitsMask = Math.pow(2, msBitOffset).toInt - 1 // Assume LZ4 compressor has state and is not thread safe. Use ThreadLocals. private def getCompressor: LZ4Compressor = { @@ -95,32 +87,13 @@ package object store { } /** - * 64-bit TimeUUID function designed specifically for generating unique ChunkIDs. Chunks take a while - * to encode so rarely would you be generating more than a few thousand chunks per second. Format: - * bits 63-21 (43 bits): milliseconds since Jan 1, 2016 - enough for 278.7 years or through 2294 - * bits 20-11 (10 bits): SnowFlake-style machine ID from FauxFlake library - * bits 10-0 (11 bits): nanosecond time in 512-ns increments. - * - * Bit 63 is inverted to allow for easy comparisons using standard signed Long math. - * - * The TimeUUID function generally increases in time but successive calls are not guaranteed to be strictly - * increasing, but if called greater than 512ns apart should be unique. - */ - def timeUUID64: Long = { - ((System.currentTimeMillis - baseTimeMillis) << msBitOffset) | - (machineId1024 << machIdBitOffset) | - ((System.nanoTime >> baseNsBitOffset) & nanoBitMask) ^ - Long64HighBit - } - - /** - * New formulation for chunkID based on a combo of the start time for a chunk and the current time in the lower + * Formulation for chunkID based on a combo of the start time for a chunk and the current time in the lower * bits to disambiguate two chunks which have the same start time. * * bits 63-21 (43 bits): milliseconds since Unix Epoch (1/1/1970) - enough for 278.7 years or through 2248 * bits 20-0 (21 bits): The lower 21 bits of nanotime for disambiguation */ - @inline final def newChunkID(startTime: Long): Long = chunkID(startTime, System.nanoTime) + @inline final def chunkID(startTime: Long): Long = chunkID(startTime, System.nanoTime) @inline final def chunkID(startTime: Long, currentTime: Long): Long = (startTime << msBitOffset) | (currentTime & lowerBitsMask) From 3489d3844b9ef25277f6533ee2415fff513f2006 Mon Sep 17 00:00:00 2001 From: Mahaaveer Date: Tue, 16 Apr 2019 09:58:20 -0700 Subject: [PATCH 18/74] misc(query): make spreadFunc serializable (#304) Making `spreadFunc` be derived using a static provider allows it to be serializable. --- .github/COMMIT_TEMPLATE | 26 +++++++++++++++++++ .../client/QueryCommands.scala | 25 ++++++++++++++++-- .../queryengine/Utils.scala | 3 ++- .../queryengine2/QueryEngine.scala | 4 +-- .../queryengine2/QueryEngineSpec.scala | 12 ++++----- .../filodb/http/PrometheusApiRoute.scala | 11 +++++--- 6 files changed, 66 insertions(+), 15 deletions(-) create mode 100644 .github/COMMIT_TEMPLATE diff --git a/.github/COMMIT_TEMPLATE b/.github/COMMIT_TEMPLATE new file mode 100644 index 0000000000..54a8fc5b53 --- /dev/null +++ b/.github/COMMIT_TEMPLATE @@ -0,0 +1,26 @@ + +## Use the following template for commits +## tip: the following command adds this template and helps whenever you do a `git commit` +## `git config --replace-all commit.template .github/COMMIT_TEMPLATE` +## +## type: defines the type of change. can be one of: +### - fix: for general fixes +### - bug: for bug fixes +### - feat: for features +### - misc: for miscellaneous items +## +## scope: defines the scope of the change. e.g.: +### - query +### - coordinator +### - memory +### - core +### - coord +## +## subject: a short description of the change +(): + +## explain the change in greater detail. + + +## any footer notes (optional) +