From f41d932818e9189d1c7897b88a718f27988c4303 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2019 16:08:05 -0700 Subject: [PATCH 01/28] feat(core): MultiSchema prep: Allow PartitionSet to be used with multiple schema/Dataset definitions (#449) --- .../filodb.core/memstore/PartitionSet.scala | 42 +++++++++---------- .../memstore/TimeSeriesShard.scala | 8 ++-- .../memstore/PartitionSetSpec.scala | 20 ++++----- .../memstore/TimeSeriesMemStoreSpec.scala | 4 +- 4 files changed, 37 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/PartitionSet.scala b/core/src/main/scala/filodb.core/memstore/PartitionSet.scala index 09f2f372b4..65c38c8177 100644 --- a/core/src/main/scala/filodb.core/memstore/PartitionSet.scala +++ b/core/src/main/scala/filodb.core/memstore/PartitionSet.scala @@ -23,7 +23,7 @@ import scala.{specialized => sp} import spire.syntax.all._ -import filodb.core.binaryrecord2.{RecordComparator, RecordSchema} +import filodb.core.metadata.Dataset import filodb.core.store.FiloPartition import filodb.memory.BinaryRegionLarge @@ -37,8 +37,7 @@ import filodb.memory.BinaryRegionLarge * When the type A is known (or the caller is specialized on A), * Set[A] will store the values in an unboxed array. */ -final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: Int, - ingestSchema: RecordSchema, comp: RecordComparator) extends Serializable { lhs => +final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: Int) extends Serializable { lhs => // set machinery var items: Array[FiloPartition] = as // slots for items var buckets: Array[Byte] = bs // buckets track defined/used slots @@ -163,9 +162,9 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * @param ingestOffset the offset/address of the ingestion BinaryRecord * @param addFunc a no-arg function to create the FiloPartition if it cannot be found */ - final def getOrAddWithIngestBR(ingestBase: Any, ingestOffset: Long, + final def getOrAddWithIngestBR(ingestBase: Any, ingestOffset: Long, ds: Dataset, addFunc: => FiloPartition): FiloPartition = { - getWithIngestBR(ingestBase, ingestOffset) match { + getWithIngestBR(ingestBase, ingestOffset, ds) match { case null => val newItem = addFunc if (newItem != null) add(newItem) @@ -177,19 +176,20 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I /** * Returns the partition that matches the partition key in an ingest record, or NULL if it doesn't exist */ - final def getWithIngestBR(ingestBase: Any, ingestOffset: Long): FiloPartition = { + final def getWithIngestBR(ingestBase: Any, ingestOffset: Long, ds: Dataset): FiloPartition = { @inline @tailrec def loop(i: Int, perturbation: Int): FiloPartition = { val j = i & mask val status = buckets(j) if (status == 0) { null - } else if (status == 3 && comp.partitionMatch(ingestBase, ingestOffset, null, items(j).partKeyOffset)) { + } else if (status == 3 && + ds.comparator.partitionMatch(ingestBase, ingestOffset, null, items(j).partKeyOffset)) { items(j) } else { loop((i << 2) + i + perturbation + 1, perturbation >> 5) } } - val i = ingestSchema.partitionHash(ingestBase, ingestOffset) & 0x7fffffff + val i = ds.ingestionSchema.partitionHash(ingestBase, ingestOffset) & 0x7fffffff loop(i, i) } @@ -197,7 +197,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * Searches for and returns Some(tsPartition) if a partition exists with a key matching the passed in * partition key BinaryRecord. Otherwise, None is returned. */ - final def getWithPartKeyBR(partBase: Any, partOffset: Long): Option[FiloPartition] = { + final def getWithPartKeyBR(partBase: Any, partOffset: Long, ds: Dataset): Option[FiloPartition] = { @inline @tailrec def loop(i: Int, perturbation: Int): Option[FiloPartition] = { val j = i & mask val status = buckets(j) @@ -210,7 +210,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I loop((i << 2) + i + perturbation + 1, perturbation >> 5) } } - val i = comp.partitionKeySchema.partitionHash(partBase, partOffset) & 0x7fffffff + val i = ds.comparator.partitionKeySchema.partitionHash(partBase, partOffset) & 0x7fffffff loop(i, i) } @@ -222,7 +222,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * * This is an O(n) operation. */ - final def copy(): PartitionSet = new PartitionSet(items.clone, buckets.clone, len, used, ingestSchema, comp) + final def copy(): PartitionSet = new PartitionSet(items.clone, buckets.clone, len, used) /** * Clears the set's internal state. @@ -238,7 +238,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * This is an O(1) operation, but may generate a lot of garbage if * the set was previously large. */ - final def clear(): Unit = { absorb(PartitionSet.empty(ingestSchema, comp)) } + final def clear(): Unit = { absorb(PartitionSet.empty()) } /** * Aborb the given set's contents into this set. @@ -422,7 +422,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I */ final def grow(): Unit = { val next = buckets.length * (if (buckets.length < 10000) 4 else 2) - val set = PartitionSet.ofAllocatedSize(next, ingestSchema, comp) + val set = PartitionSet.ofAllocatedSize(next) cfor(0)(_ < buckets.length, _ + 1) { i => if (buckets(i) == 3) set += items(i) } @@ -443,7 +443,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * This is an O(n) operation, where n it the set's size. */ final def compact(): Unit = { - val set = PartitionSet.ofSize(len, ingestSchema, comp) + val set = PartitionSet.ofSize(len) cfor(0)(_ < buckets.length, _ + 1) { i => if (buckets(i) == 3) set += items(i) } @@ -651,7 +651,7 @@ final class PartitionSet(as: Array[FiloPartition], bs: Array[Byte], n: Int, u: I * This is an O(n) operation, where n is the size of the set. */ def findAll(p: FiloPartition => Boolean): PartitionSet = { - val out = PartitionSet.empty(lhs.ingestSchema, lhs.comp) + val out = PartitionSet.empty() cfor(0)(_ < buckets.length, _ + 1) { i => if (buckets(i) == 3 && p(items(i))) out += items(i) } @@ -744,8 +744,8 @@ object PartitionSet { /** * Allocate an empty Set. */ - def empty(ingestSchema: RecordSchema, comp: RecordComparator): PartitionSet = - new PartitionSet(new Array[FiloPartition](8), new Array[Byte](8), 0, 0, ingestSchema, comp) + def empty(): PartitionSet = + new PartitionSet(new Array[FiloPartition](8), new Array[Byte](8), 0, 0) /** * Allocate an empty Set, capable of holding n items without @@ -754,8 +754,8 @@ object PartitionSet { * This method is useful if you know you'll be adding a large number * of elements in advance and you want to save a few resizes. */ - def ofSize(n: Int, ingestSchema: RecordSchema, comp: RecordComparator): PartitionSet = - ofAllocatedSize(n / 2 * 3, ingestSchema, comp) + def ofSize(n: Int): PartitionSet = + ofAllocatedSize(n / 2 * 3) /** * Allocate an empty Set, with underlying storage of size n. @@ -764,12 +764,12 @@ object PartitionSet { * underlying array to be. In most cases ofSize() is probably what * you want instead. */ - private def ofAllocatedSize(n: Int, ingestSchema: RecordSchema, comp: RecordComparator): PartitionSet = { + private def ofAllocatedSize(n: Int): PartitionSet = { val sz = debox.Util.nextPowerOfTwo(n) match { case n if n < 0 => throw debox.DeboxOverflowError(n) case 0 => 8 case n => n } - new PartitionSet(new Array[FiloPartition](sz), new Array[Byte](sz), 0, 0, ingestSchema, comp) + new PartitionSet(new Array[FiloPartition](sz), new Array[Byte](sz), 0, 0) } } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 01d724fa85..67b9051577 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -252,7 +252,7 @@ class TimeSeriesShard(val dataset: Dataset, /** * PartitionSet - access TSPartition using ingest record partition key in O(1) time. */ - private[memstore] final val partSet = PartitionSet.ofSize(InitialNumPartitions, ingestSchema, recordComp) + private[memstore] final val partSet = PartitionSet.ofSize(InitialNumPartitions) // Use a StampedLock because it supports optimistic read locking. This means that no blocking // occurs in the common case, when there isn't any contention reading from partSet. private[memstore] final val partSetLock = new StampedLock @@ -1026,7 +1026,7 @@ class TimeSeriesShard(val dataset: Dataset, private[filodb] def getOrAddPartitionForIngestion(recordBase: Any, recordOff: Long, group: Int, ingestOffset: Long) = { assertThreadName(IngestSchedName) - var part = partSet.getWithIngestBR(recordBase, recordOff) + var part = partSet.getWithIngestBR(recordBase, recordOff, dataset) if (part == null) { part = addPartitionForIngestion(recordBase, recordOff, group) } @@ -1353,7 +1353,7 @@ class TimeSeriesShard(val dataset: Dataset, // nothing changed in the set, and the partition object is the correct one. var stamp = partSetLock.tryOptimisticRead() if (stamp != 0) { - part = partSet.getWithPartKeyBR(partKey, UnsafeUtils.arayOffset) + part = partSet.getWithPartKeyBR(partKey, UnsafeUtils.arayOffset, dataset) } if (!partSetLock.validate(stamp)) { // Because the stamp changed, the write lock was acquired and the set likely changed. @@ -1362,7 +1362,7 @@ class TimeSeriesShard(val dataset: Dataset, // the correct partition is returned. stamp = partSetLock.readLock() try { - part = partSet.getWithPartKeyBR(partKey, UnsafeUtils.arayOffset) + part = partSet.getWithPartKeyBR(partKey, UnsafeUtils.arayOffset, dataset) } finally { partSetLock.unlockRead(stamp) } diff --git a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala index 66ee3b5a06..f458b393f6 100644 --- a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala @@ -35,7 +35,7 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.blockMetaSize, true) val builder = new RecordBuilder(memFactory, dataset2.ingestionSchema) - val partSet = PartitionSet.empty(dataset2.ingestionSchema, dataset2.comparator) + val partSet = PartitionSet.empty() before { partSet.clear() @@ -78,34 +78,34 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { partSet += part partSet.size shouldEqual 1 - val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), { throw new RuntimeException("error")} ) + val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), dataset2, { throw new RuntimeException("error")} ) got shouldEqual part partSet.size shouldEqual 1 } it("should add new TSPartition if one doesnt exist with getOrAddWithIngestBR") { partSet.isEmpty shouldEqual true - partSet.getWithPartKeyBR(null, partKeyAddrs(0)) shouldEqual None - partSet.getWithIngestBR(null, ingestRecordAddrs(0)) shouldEqual null + partSet.getWithPartKeyBR(null, partKeyAddrs(0), dataset2) shouldEqual None + partSet.getWithIngestBR(null, ingestRecordAddrs(0), dataset2) shouldEqual null val part = makePart(0, dataset2, partKeyAddrs(0), bufferPool) - val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), part) + val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), dataset2, part) partSet.size shouldEqual 1 partSet.isEmpty shouldEqual false got shouldEqual part - partSet.getWithPartKeyBR(null, partKeyAddrs(0)) shouldEqual Some(part) - partSet.getWithIngestBR(null, ingestRecordAddrs(0)) shouldEqual part + partSet.getWithPartKeyBR(null, partKeyAddrs(0), dataset2) shouldEqual Some(part) + partSet.getWithIngestBR(null, ingestRecordAddrs(0), dataset2) shouldEqual part } it("should not add new TSPartition if function returns null") { partSet.isEmpty shouldEqual true - partSet.getWithPartKeyBR(null, partKeyAddrs(0)) shouldEqual None + partSet.getWithPartKeyBR(null, partKeyAddrs(0), dataset2) shouldEqual None - val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), null) + val got = partSet.getOrAddWithIngestBR(null, ingestRecordAddrs(0), dataset2, null) got shouldEqual null partSet.isEmpty shouldEqual true - partSet.getWithPartKeyBR(null, partKeyAddrs(0)) shouldEqual None + partSet.getWithPartKeyBR(null, partKeyAddrs(0), dataset2) shouldEqual None } it("should remove TSPartitions correctly") { diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index 1ae2680e2f..fefd96cc60 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -306,11 +306,11 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val expectedPartKey = dataset1.partKeySchema.asByteArray(UnsafeUtils.ZeroPointer, off) 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.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off, dataset).get.partID shouldEqual i tsShard.partitions.containsKey(i) shouldEqual true // since partition is ingesting } else { - tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off) shouldEqual None + tsShard.partSet.getWithPartKeyBR(UnsafeUtils.ZeroPointer, off, dataset) shouldEqual None tsShard.partitions.containsKey(i) shouldEqual false // since partition is not ingesting } } From 0bb54ff9ba7de896cc58f3b89fceca1e09555cea Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 25 Jul 2019 14:34:57 -0700 Subject: [PATCH 02/28] feat(core, *): PartitionSchema and DataSchemas: multi-schema part I (#440) BREAKING CONFIG CHANGES --- README.md | 8 +- cli/src/main/scala/filodb.cli/CliMain.scala | 21 +- conf/timeseries-128shards-source.conf | 17 +- conf/timeseries-dev-source.conf | 38 +-- conf/timeseries-ds-1m-dev-source.conf | 17 +- conf/timeseries-standalonetest-source.conf | 17 +- .../filodb.coordinator/FilodbSettings.scala | 23 +- .../filodb.coordinator/NodeClusterActor.scala | 2 +- .../NodeCoordinatorActor.scala | 4 +- .../queryengine/Utils.scala | 14 +- .../queryengine2/QueryEngine.scala | 10 +- .../ClusterRecoverySpec.scala | 22 +- .../queryengine2/QueryEngineSpec.scala | 6 +- core/src/main/resources/filodb-defaults.conf | 43 +++ .../downsample/ShardDownsampler.scala | 8 +- .../scala/filodb.core/metadata/Dataset.scala | 134 +++------- .../scala/filodb.core/metadata/Schemas.scala | 242 +++++++++++++++++ .../src/test/scala/filodb.core/TestData.scala | 64 ++--- .../binaryrecord2/BinaryRecordSpec.scala | 3 +- .../downsample/ShardDownsamplerSpec.scala | 13 +- .../filodb.core/metadata/DatasetSpec.scala | 175 +----------- .../filodb.core/metadata/SchemasSpec.scala | 249 ++++++++++++++++++ .../filodb.core/query/KeyFilterSpec.scala | 10 +- .../filodb.core/store/ColumnStoreSpec.scala | 19 -- .../filodb.core/store/MetaStoreSpec.scala | 32 +-- doc/ingestion.md | 38 +-- .../scala/filodb/gateway/GatewayServer.scala | 8 +- .../filodb/http/PrometheusApiRouteSpec.scala | 8 +- .../filodb.jmh/IntSumReadBenchmark.scala | 3 +- .../scala/filodb/kafka/TestConsumer.scala | 3 +- project/FiloBuild.scala | 2 + .../filodb/prometheus/FormatConversion.scala | 8 +- .../query/exec/SelectRawPartitionsExec.scala | 8 +- 33 files changed, 737 insertions(+), 532 deletions(-) create mode 100644 core/src/main/scala/filodb.core/metadata/Schemas.scala create mode 100644 core/src/test/scala/filodb.core/metadata/SchemasSpec.scala diff --git a/README.md b/README.md index 25d69b9239..afa4deb031 100644 --- a/README.md +++ b/README.md @@ -327,10 +327,16 @@ The **partition key** differentiates time series and also controls distribution The data points use a configurable schema consisting of multiple columns. Each column definition consists of `name:columntype`, with optional parameters. For examples, see the examples below, or see the introductory walk-through above where two datasets are created. +A single partition key schema is used for a running FiloDB cluster, though multiple data schemas may be supported. These schemas are defined in the config file - see the `partition-schema` and `schemas` sections of `filodb-defaults.conf`. The CLI command `validateSchemas` may be run to verify schemas defined in config files, as follows: + + ./filo-cli -Dconfig.file=conf/timeseries-filodb-server.conf --command validateSchemas + ### Dataset Configuration THIS IS IMPORTANT TO READ AND UNDERSTAND. +Each "dataset" ingests one stream or Kafka topic of raw time series data, and is also the unit of isolation. Each dataset contains its own offheap memory, and can have independent data retention and ingestion properties. + Datasets are setup and loaded into the server via configuration files referred to by application.conf loaded by the server. See `conf/timeseries-dev-source.conf` for an example. It is important to note that some aspects of the dataset, like its column definition are immutable. This is primarily because the data columns are used to populate persistent @@ -344,7 +350,6 @@ that part of the cluster could be with the old config and the rest could have ne ### Prometheus FiloDB Schema for Operational Metrics * Partition key = `tags:map` -* Row key = `timestamp` * Columns: `timestamp:ts,value:double:detectDrops=true` The above is the classic Prometheus-compatible schema. It supports indexing on any tag. Thus standard Prometheus queries that filter by a tag such as `hostname` or `datacenter` for example would work fine. Note that the Prometheus metric name is encoded as a key `__name__`, which is the Prometheus standard when exporting tags. @@ -358,7 +363,6 @@ NOTE: `detectDrops=true` allows for proper and efficient rate calculation on Pro Let's say that one had a metrics client, such as CodaHale metrics, which pre-aggregates percentiles and sends them along with the metric. If we used the Prometheus schema, each percentile would wind up in its own time series. This is fine, but incurs significant overhead as the partition key has to then be sent with each percentile over the wire. Instead we can have a schema which includes all the percentiles together when sending the data: * Partition key = `metricName:string,tags:map` -* Row key = `timestamp` * Columns: `timestamp:ts,min:double,max:double,p50:double,p90:double,p95:double,p99:double,p999:double` ### Data Modelling and Performance Considerations diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 08fa757d7c..45880ffd9c 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -9,12 +9,13 @@ import scala.util.Try import com.opencsv.CSVWriter import com.quantifind.sumac.{ArgMain, FieldArgs} import monix.reactive.Observable +import org.scalactic._ import filodb.coordinator._ import filodb.coordinator.client._ import filodb.coordinator.client.QueryCommands.{SpreadChange, SpreadProvider, StaticSpreadProvider} import filodb.core._ -import filodb.core.metadata.Column +import filodb.core.metadata.{Column, Schemas} import filodb.memory.format.RowReader import filodb.prometheus.ast.{InMemoryParam, TimeRangeParams, TimeStepParams, WriteBuffersParam} import filodb.prometheus.parse.Parser @@ -144,6 +145,8 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { val (remote, ref) = getClientAndRef(args) dumpShardStatus(remote, ref) + case Some("validateSchemas") => validateSchemas() + case Some("timeseriesMetadata") => require(args.host.nonEmpty && args.dataset.nonEmpty && args.matcher.nonEmpty, "--host, --dataset and --matcher must be defined") val remote = Client.standaloneClient(system, args.host.get, args.port) @@ -202,6 +205,22 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { } } + def validateSchemas(): Unit = { + Schemas.fromConfig(config) match { + case Good(Schemas(partSchema, data, _)) => + println("Schema validated.\nPartition schema:") + partSchema.columns.foreach(c => println(s"\t$c")) + data.foreach { case (schemaName, sch) => + println(s"Schema $schemaName:") + sch.columns.foreach(c => println(s"\t$c")) + } + case Bad(errors) => + println(s"Schema validation errors:") + errors.foreach { case (name, err) => println(s"$name\t$err")} + exitCode = 1 + } + } + final case class QOptions(limit: Int, sampleLimit: Int, everyN: Option[Int], timeout: FiniteDuration, shardOverrides: Option[Seq[Int]], diff --git a/conf/timeseries-128shards-source.conf b/conf/timeseries-128shards-source.conf index 3415a25338..8e003126b5 100644 --- a/conf/timeseries-128shards-source.conf +++ b/conf/timeseries-128shards-source.conf @@ -1,20 +1,5 @@ dataset = "prometheus" - - definition { - partition-columns = ["tags:map"] - data-columns = ["timestamp:ts", "value:double:detectDrops=true"] - row-key-columns = [ "timestamp" ] - downsamplers = [ ] - } - - options { - shardKeyColumns = [ "__name__", "_ns" ] - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - valueColumn = "value" - metricColumn = "__name__" - ignoreTagsOnPartitionKeyHash = [ "le" ] - copyTags = { } - } + schema = "prometheus" num-shards = 128 min-num-nodes = 2 diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index e0159d13da..776e6e06a7 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -1,45 +1,11 @@ dataset = "prometheus" - # Schema used for defining the BinaryRecord used in ingestion and persistence. - # - # Should not change once dataset has been set up on the server and data has been ingested into kafka - # or written to cassandra - definition { - # defines the unique identifier for partition - partition-columns = ["tags:map"] - # Schema of all of the values stored against the partition key. This includes the row-keys as well - data-columns = ["timestamp:ts", "value:double:detectDrops=true"] - # Clustering key for each row. Together with partition key, they form the primary key. - row-key-columns = [ "timestamp" ] - # List of downsamplers for the data columns - downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)" ] - } - - # Dataset Options - # - # Should not change once dataset has been set up on the server and data has been ingested into kafka - # or written to cassandra - options { - # These column values are used to identify the shard group for the partition - shardKeyColumns = [ "__name__", "_ns" ] - # suffixes from map values are stripped before hashing - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - # default data column name to be used referencing the value column - valueColumn = "value" - # column name to use when referencing the name column - metricColumn = "__name__" - # these columns are ignored in calculation of full partition key hash - ignoreTagsOnPartitionKeyHash = [ "le" ] - - # These key-names will be replaced in the key map during ingestion/query. - # Temporary workaround. Will be deprecated. - copyTags = { } - } + # Name of schema used for this dataset stream. See filodb.schemas in filodb-defaults or any other server conf + schema = "prometheus" # Should not change once dataset has been set up on the server and data has been persisted to cassandra num-shards = 4 - min-num-nodes = 2 # Length of chunks to be written, roughly sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" diff --git a/conf/timeseries-ds-1m-dev-source.conf b/conf/timeseries-ds-1m-dev-source.conf index 9c6d7903e7..33b428d0d3 100644 --- a/conf/timeseries-ds-1m-dev-source.conf +++ b/conf/timeseries-ds-1m-dev-source.conf @@ -1,20 +1,5 @@ dataset = "prometheus_ds_1m" - - definition { - partition-columns = ["tags:map"] - data-columns = [ "timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double" ] - row-key-columns = [ "timestamp" ] - downsamplers = [ ] - } - - options { - shardKeyColumns = [ "__name__", "_ns" ] - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - valueColumn = "avg" - metricColumn = "__name__" - ignoreTagsOnPartitionKeyHash = [ "le" ] - copyTags = { } - } + schema = "prom-ds-gauge" num-shards = 4 min-num-nodes = 2 diff --git a/conf/timeseries-standalonetest-source.conf b/conf/timeseries-standalonetest-source.conf index 1ac94dfeb0..4a0baa0fb2 100644 --- a/conf/timeseries-standalonetest-source.conf +++ b/conf/timeseries-standalonetest-source.conf @@ -1,20 +1,5 @@ dataset = "prometheus" - - definition { - partition-columns = ["tags:map"] - data-columns = ["timestamp:ts", "value:double:detectDrops=true"] - row-key-columns = [ "timestamp" ] - downsamplers = [ ] - } - - options { - shardKeyColumns = [ "__name__", "_ns" ] - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - valueColumn = "value" - metricColumn = "__name__" - ignoreTagsOnPartitionKeyHash = [ "le" ] - copyTags = { } - } + schema = "prometheus" num-shards = 4 min-num-nodes = 2 diff --git a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala index c77611f8e5..b5ad3b01ce 100755 --- a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala +++ b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala @@ -6,14 +6,15 @@ import scala.concurrent.duration.FiniteDuration import akka.actor.{ActorPath, Address, RootActorPath} import com.typesafe.config.{Config, ConfigFactory} import net.ceedubs.ficus.Ficus._ +import org.scalactic._ +import filodb.core.metadata.{Dataset, Schemas} import filodb.core.GlobalConfig /** Settings for the FiloCluster Akka Extension which gets * config from `GlobalConfig`. Uses Ficus. */ final class FilodbSettings(val conf: Config) { - def this() = this(ConfigFactory.empty) ConfigFactory.invalidateCaches() @@ -44,18 +45,32 @@ final class FilodbSettings(val conf: Config) { /** The timeout to use to resolve an actor ref for new nodes. */ val ResolveActorTimeout = config.as[FiniteDuration]("tasks.timeouts.resolve-actor") - val datasets = config.as[Seq[String]]("dataset-configs") + val datasetConfPaths = config.as[Seq[String]]("dataset-configs") /** * Returns IngestionConfig/dataset configuration from parsing dataset-configs file paths. * If those are empty, then parse the "streams" config key for inline configs. */ val streamConfigs: Seq[Config] = - if (datasets.nonEmpty) { - datasets.map { d => ConfigFactory.parseFile(new java.io.File(d)) } + if (datasetConfPaths.nonEmpty) { + datasetConfPaths.map { d => ConfigFactory.parseFile(new java.io.File(d)) } } else { config.as[Seq[Config]]("inline-dataset-configs") } + + val schemas = Schemas.fromConfig(config) match { + case Good(sch) => sch + case Bad(errs) => throw new RuntimeException("Errors parsing schemas:\n" + + errs.map { case (ds, err) => s"Schema $ds\t$err" }.mkString("\n")) + } + + // Creates a Dataset from a stream config. NOTE: this is a temporary thing to keep old code using Dataset + // compatible and minimize changes. The Dataset name is taken from the dataset/stream config, but the schema + // underneath points to one of the schemas above and schema may have a different name. The approach below + // allows one schema (with one schema name) to be shared amongst datasets using different names. + def datasetFromStream(streamConf: Config): Dataset = + Dataset(streamConf.getString("dataset"), + schemas.schemas(streamConf.getString("schema"))) } /** Consistent naming: allows other actors to accurately filter diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala index 66d64ba0f2..b4644d74f0 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala @@ -226,7 +226,7 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, // shard and dataset state can be recovered correctly. First all the datasets are set up. // Then shard state is recovered, and finally cluster membership events are replayed. settings.streamConfigs.foreach { config => - val dataset = Dataset.fromConfig(config) + val dataset = settings.datasetFromStream(config) val ingestion = IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get initializeDataset(dataset, ingestion, None) } diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index 46cf5d67a4..24e8646261 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -157,7 +157,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def datasetHandlers: Receive = LoggingReceive { case CreateDataset(datasetObj, db) => // used only for unit testing now - createDataset(sender(), datasetObj.copy(database = db)) + createDataset(sender(), datasetObj) case TruncateDataset(ref) => truncateDataset(sender(), ref) @@ -211,7 +211,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, if (!datasetsInitialized) { logger.debug(s"Initializing stream configs: ${settings.streamConfigs}") settings.streamConfigs.foreach { config => - val dataset = Dataset.fromConfig(config) + val dataset = settings.datasetFromStream(config) val ingestion = IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get initializeDataset(dataset, ingestion) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 5687cde7e3..7087984e2d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -12,7 +12,7 @@ import monix.reactive.Observable import org.scalactic._ import filodb.coordinator.ShardMapper -import filodb.core.{ErrorResponse, Types} +import filodb.core.ErrorResponse import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} @@ -28,18 +28,6 @@ final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) exten object Utils extends StrictLogging { import filodb.coordinator.client.QueryCommands._ import TrySugar._ - import filodb.coordinator.client.QueryCommands._ - - /** - * Convert column name strings into columnIDs. NOTE: column names should not include row key columns - * as those are automatically prepended. - */ - def getColumnIDs(dataset: Dataset, colStrs: Seq[String]): Seq[Types.ColumnId] Or ErrorResponse = - dataset.colIDs(colStrs: _*).badMap(missing => UndefinedColumns(missing.toSet)) - .map { ids => // avoid duplication if first ids are already row keys - if (ids.take(dataset.rowKeyIDs.length) == dataset.rowKeyIDs) { ids } - else { dataset.rowKeyIDs ++ ids } - } /** * Validates a PartitionQuery, returning a set of PartitionScanMethods with shard numbers. diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 126c9cbbd3..9ce1771a96 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -328,7 +328,7 @@ class QueryEngine(dataset: Dataset, options: QueryOptions, lp: RawChunkMeta, spreadProvider : SpreadProvider): PlanResult = { // Translate column name to ID and validate here - val colName = if (lp.column.isEmpty) dataset.options.valueColumn else lp.column + val colName = if (lp.column.isEmpty) dataset.schema.data.valueColName else lp.column val colID = dataset.colIDs(colName).get.head val renamedFilters = renameMetricFilter(lp.filters) val metaExec = shardsFromFilters(renamedFilters, options, spreadProvider).map { shard => @@ -368,13 +368,13 @@ class QueryEngine(dataset: Dataset, * as those are automatically prepended. */ private def getColumnIDs(dataset: Dataset, cols: Seq[String]): Seq[Types.ColumnId] = { - val realCols = if (cols.isEmpty) Seq(dataset.options.valueColumn) else cols + val realCols = if (cols.isEmpty) Seq(dataset.schema.data.valueColName) else cols val ids = dataset.colIDs(realCols: _*) .recover(missing => throw new BadQueryException(s"Undefined columns $missing")) .get - // avoid duplication if first ids are already row keys - if (ids.take(dataset.rowKeyIDs.length) == dataset.rowKeyIDs) { ids } - else { dataset.rowKeyIDs ++ ids } + // avoid duplication if first id is a timestamp, otherwise add it + if (ids.take(1) == Dataset.rowKeyIDs) { ids } + else { Dataset.rowKeyIDs ++ ids } } private def toChunkScanMethod(rangeSelector: RangeSelector): ChunkScanMethod = { diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index 74ff8634c9..b8826f8b3c 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -24,17 +24,23 @@ object ClusterRecoverySpecConfig extends MultiNodeConfig { val ourConf = s""" filodb { memstore.groups-per-shard = 4 + partition-schema { + columns = ["Actor2Code:string", "Actor2Name:string"] + predefined-keys = [] + ${GdeltTestData.datasetOptionConfig} + } + schemas { + gdelt { + columns = ["GLOBALEVENTID:long", "SQLDATE:long", "MonthYear:int", + "Year:int", "NumArticles:int", "AvgTone:double"] + value-column = "AvgTone" + downsamplers = [] + } + } inline-dataset-configs = [ { dataset = "gdelt" - definition { - partition-columns = ["Actor2Code:string", "Actor2Name:string"] - data-columns = ["GLOBALEVENTID:long", "SQLDATE:long", "MonthYear:int", - "Year:int", "NumArticles:int", "AvgTone:double"] - row-key-columns = [ "GLOBALEVENTID" ] - downsamplers = [] - } - ${GdeltTestData.datasetOptionConfig} + schema = "gdelt" num-shards = 2 min-num-nodes = 2 sourcefactory = "${classOf[sources.CsvStreamFactory].getName}" diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index 0bf0abdf06..0f8f093406 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -126,10 +126,12 @@ class QueryEngineSpec extends FunSpec with Matchers { } } + import com.softwaremill.quicklens._ + it("should rename Prom __name__ filters if dataset has different metric column") { // Custom QueryEngine with different dataset with different metric name - val dataset2 = dataset.copy(options = dataset.options.copy( - metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job"))) + val datasetOpts = dataset.options.copy(metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job")) + val dataset2 = dataset.modify(_.schema.partition.options).setTo(datasetOpts) val engine2 = new QueryEngine(dataset2, mapperRef) // materialized exec plan diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index db392592ad..86e48c54a8 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -14,6 +14,49 @@ filodb { # ] inline-dataset-configs = [] + # Definition of cluster-wide partition key scheme. The partition key defines each unique time series, + # such as labels or tags, and is used for sharding time series across the cluster. + # The below definition is standard for Prometheus schema + partition-schema { + # Typical column types used: map, string. Also possible: ts,long,double + columns = ["tags:map"] + + # Predefined keys allow space to be saved for over the wire tags with the given keys + predefined-keys = ["_ns", "app", "__name__", "instance", "dc", "le"] + + options { + copyTags = {} + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + ignoreTagsOnPartitionKeyHash = ["le"] + metricColumn = "__name__" + shardKeyColumns = [ "__name__", "_ns" ] + } + } + + # Definitions of possible data schemas to be used in all datasets + # Each one must have a unique name and column schema. + # FiloDB will refuse to start if the schema definitions have errors. Use the validateSchemas CLI command to check. + schemas { + prometheus { + # Each column def is of name:type format. Type may be ts,long,double,string,int + # The first column must be ts or long + columns = ["timestamp:ts", "value:double:detectDrops=true"] + + # Default column to query using PromQL + value-column = "value" + + # Downsampling configuration. See doc/downsampling.md + downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)" ] + } + + # Used for downsampled gauge data + prom-ds-gauge { + columns = [ "timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double" ] + value-column = "avg" + downsamplers = [] + } + } + tasks { # Frequency with which new shard maps are published shardmap-publish-frequency = 5s diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index 9af51c7586..4146db844e 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -25,9 +25,11 @@ class ShardDownsampler(dataset: Dataset, resolutions: Seq[Int], publisher: DownsamplePublisher, stats: TimeSeriesShardStats) extends StrictLogging { + private val downsamplers = dataset.schema.data.downsamplers + if (enabled) { logger.info(s"Downsampling enabled for dataset=${dataset.ref} shard=$shardNum with " + - s"following downsamplers: ${dataset.downsamplers.map(_.encoded)} at resolutions: $resolutions") + s"following downsamplers: ${downsamplers.map(_.encoded)} at resolutions: $resolutions") } else { logger.info(s"Downsampling disabled for dataset=${dataset.ref} shard=$shardNum") } @@ -58,7 +60,7 @@ class ShardDownsampler(dataset: Dataset, */ private[downsample] def downsampleIngestSchema(): RecordSchema = { // The name of the column in downsample record does not matter at the ingestion side. Type does matter. - val downsampleCols = dataset.downsamplers.map { d => ColumnInfo(s"${d.name.entryName}", d.colType) } + val downsampleCols = downsamplers.map { d => ColumnInfo(s"${d.name.entryName}", d.colType) } new RecordSchema(downsampleCols ++ dataset.partKeySchema.columns, Some(downsampleCols.size), dataset.ingestionSchema.predefinedKeys) } @@ -94,7 +96,7 @@ class ShardDownsampler(dataset: Dataset, val endRowNum = Math.min(tsReader.ceilingIndex(vecPtr, pEnd), chunkset.numRows - 1) builder.startNewRecord() // for each downsampler, add downsample column value - dataset.downsamplers.foreach { + downsamplers.foreach { case d: TimeChunkDownsampler => builder.addLong(d.downsampleChunk(part, chunkset, startRowNum, endRowNum)) case d: DoubleChunkDownsampler => diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index 575111132e..83c66b0ec8 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -7,12 +7,11 @@ import net.ceedubs.ficus.Ficus._ import org.scalactic._ import filodb.core._ -import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2, _} +import filodb.core.binaryrecord2._ 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, ZeroCopyUTF8String => ZCUTF8} +import filodb.memory.format.{RowReader, TypedIterator, ZeroCopyUTF8String => ZCUTF8} /** * A dataset describes the schema (column name & type) and distribution for a stream/set of data. @@ -26,31 +25,30 @@ import filodb.memory.format.{BinaryVector, RowReader, TypedIterator, ZeroCopyUTF * partition columns: metricName:string, tags:map * data columns: timestamp:long, value:double * + * NOTE: this data structure will be deprecated slowly in favor of PartitionSchema/DataSchema. + * NOTE2: name is used for ingestion stream name, which is separate from the name of the schema. + * * The Column IDs (used for querying) for data columns are numbered starting with 0, and for partition * columns are numbered starting with PartColStartIndex. This means position is the same or easily derived * * The rowKeyIDs are the dataColumns IDs/positions for the "row key", typically a timestamp column but * something which makes a value unique within a partition and describes a range of data in a chunk. */ -final case class Dataset(name: String, - partitionColumns: Seq[Column], - dataColumns: Seq[Column], - rowKeyIDs: Seq[Int], - downsamplers: Seq[ChunkDownsampler], - database: Option[String] = None, - options: DatasetOptions = DatasetOptions.DefaultOptions) { - require(rowKeyIDs.nonEmpty) - val ref = DatasetRef(name, database) - val rowKeyColumns = rowKeyIDs.map(dataColumns) - val rowKeyRouting = rowKeyIDs.toArray - - val ingestionSchema = RecordSchema2.ingestion(this) // TODO: add predefined keys yo - val comparator = new RecordComparator(ingestionSchema) - val partKeySchema = comparator.partitionKeySchema +final case class Dataset(name: String, schema: Schema) { + val options = schema.partition.options + val dataColumns = schema.data.columns + val partitionColumns = schema.partition.columns + + val ref = DatasetRef(name, None) + val rowKeyColumns = schema.data.columns take 1 + + val ingestionSchema = schema.ingestionSchema + val comparator = schema.comparator + val partKeySchema = schema.partKeySchema // Used to create a `VectorDataReader` of correct type for a given data column ID; type PtrToDataReader - val dataReaders = dataColumns.map(col => BinaryVector.defaultPtrToReader(col.columnType.clazz)).toArray - val numDataColumns = dataColumns.length + val dataReaders = schema.data.readers + val numDataColumns = schema.data.columns.length // Used for ChunkSetReader.binarySearchKeyChunks val rowKeyOrdering = CompositeReaderOrdering(rowKeyColumns.map(_.columnType.keyType)) @@ -59,8 +57,8 @@ final case class Dataset(name: String, val timestampColID = timestampColumn.id // The number of bytes of chunkset metadata including vector pointers in memory - val chunkSetInfoSize = ChunkSetInfo.chunkSetInfoSize(dataColumns.length) - val blockMetaSize = chunkSetInfoSize + 4 + val chunkSetInfoSize = schema.data.chunkSetInfoSize + val blockMetaSize = schema.data.blockMetaSize private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, partKeySchema, 10240) @@ -81,8 +79,8 @@ final case class Dataset(name: String, */ def partColIterator(columnID: Int, base: Any, offset: Long): TypedIterator = { val partColPos = columnID - Dataset.PartColStartIndex - require(Dataset.isPartitionID(columnID) && partColPos < partitionColumns.length) - partitionColumns(partColPos).columnType match { + require(Dataset.isPartitionID(columnID) && partColPos < schema.partition.columns.length) + schema.partition.columns(partColPos).columnType match { case StringColumn => new PartKeyUTF8Iterator(partKeySchema, base, offset, partColPos) case LongColumn => new PartKeyLongIterator(partKeySchema, base, offset, partColPos) case TimestampColumn => new PartKeyLongIterator(partKeySchema, base, offset, partColPos) @@ -93,7 +91,7 @@ final case class Dataset(name: String, /** * Extracts a timestamp out of a RowReader, assuming data columns are first (ingestion order) */ - def timestamp(dataRowReader: RowReader): Long = dataRowReader.getLong(rowKeyIDs.head) + final def timestamp(dataRowReader: RowReader): Long = dataRowReader.getLong(0) import Accumulation._ import OptionSugar._ @@ -101,8 +99,8 @@ final case class Dataset(name: String, * Returns the column IDs for the named columns or the missing column names */ def colIDs(colNames: String*): Seq[Int] Or Seq[String] = - colNames.map { n => dataColumns.find(_.name == n).map(_.id) - .orElse { partitionColumns.find(_.name == n).map(_.id) } + colNames.map { n => schema.data.columns.find(_.name == n).map(_.id) + .orElse { schema.partition.columns.find(_.name == n).map(_.id) } .toOr(One(n)) } .combined.badMap(_.toSeq) @@ -112,38 +110,23 @@ final case class Dataset(name: String, * over the input RowReader to return data columns corresponding to dataset definition. */ def dataRouting(colNames: Seq[String]): Array[Int] = - dataColumns.map { c => colNames.indexOf(c.name) }.toArray + schema.data.columns.map { c => colNames.indexOf(c.name) }.toArray /** * Returns a routing from data + partition columns (as required for ingestion BinaryRecords) to * the input RowReader columns whose names are passed in. */ def ingestRouting(colNames: Seq[String]): Array[Int] = - dataRouting(colNames) ++ partitionColumns.map { c => colNames.indexOf(c.name) } + dataRouting(colNames) ++ schema.partition.columns.map { c => colNames.indexOf(c.name) } /** Returns the Column instance given the ID */ def columnFromID(columnID: Int): Column = - if (Dataset.isPartitionID(columnID)) { partitionColumns(columnID - Dataset.PartColStartIndex) } - else { dataColumns(columnID) } + if (Dataset.isPartitionID(columnID)) { schema.partition.columns(columnID - Dataset.PartColStartIndex) } + else { schema.data.columns(columnID) } /** Returns ColumnInfos from a set of column IDs. Throws exception if ID is invalid */ def infosFromIDs(ids: Seq[Types.ColumnId]): Seq[ColumnInfo] = ids.map(columnFromID).map { c => ColumnInfo(c.name, c.columnType) } - - def toConfig: Config = { - - val c = Map[String, Any] ( - "dataset" -> name, - "definition.partition-columns" -> partitionColumns.map(_.toStringNotation).asJava, - "definition.data-columns" -> dataColumns.map(_.toStringNotation).asJava, - "definition.row-key-columns" -> rowKeyIDs.map(dataColumns(_).name).asJava, - "definition.downsamplers" -> downsamplers.map(_.encoded).asJava - ).asJava - - ConfigFactory.parseMap(c).withFallback(ConfigFactory.parseString( - s""" options ${options} """)) - - } } /** @@ -152,7 +135,6 @@ final case class Dataset(name: String, */ case class DatasetOptions(shardKeyColumns: Seq[String], metricColumn: String, - valueColumn: String, // TODO: deprecate these options once we move all input to Telegraf/Influx // They are needed only to differentiate raw Prometheus-sourced data ignoreShardKeyColumnSuffixes: Map[String, Seq[String]] = Map.empty, @@ -167,7 +149,6 @@ case class DatasetOptions(shardKeyColumns: Seq[String], val map: Map[String, Any] = Map( "shardKeyColumns" -> shardKeyColumns.asJava, "metricColumn" -> metricColumn, - "valueColumn" -> valueColumn, "ignoreShardKeyColumnSuffixes" -> ignoreShardKeyColumnSuffixes.mapValues(_.asJava).asJava, "ignoreTagsOnPartitionKeyHash" -> ignoreTagsOnPartitionKeyHash.asJava, @@ -188,7 +169,6 @@ case class DatasetOptions(shardKeyColumns: Seq[String], object DatasetOptions { val DefaultOptions = DatasetOptions(shardKeyColumns = Nil, metricColumn = "__name__", - valueColumn = "value", // defaults that work well for Prometheus ignoreShardKeyColumnSuffixes = Map("__name__" -> Seq("_bucket", "_count", "_sum")), @@ -201,7 +181,6 @@ object DatasetOptions { def fromConfig(config: Config): DatasetOptions = DatasetOptions(shardKeyColumns = config.as[Seq[String]]("shardKeyColumns"), metricColumn = config.getString("metricColumn"), - valueColumn = config.getString("valueColumn"), ignoreShardKeyColumnSuffixes = config.as[Map[String, Seq[String]]]("ignoreShardKeyColumnSuffixes"), ignoreTagsOnPartitionKeyHash = config.as[Seq[String]]("ignoreTagsOnPartitionKeyHash"), @@ -212,18 +191,7 @@ object DatasetOptions { * Contains many helper functions especially pertaining to Dataset creation and validation. */ object Dataset { - def fromConfig(config: Config): Dataset = { - val dataset = config.getString("dataset") - val defn = config.getConfig("definition") - val options = config.getConfig("options") - - val partitionCols = defn.as[Seq[String]]("partition-columns") - val dataCols = defn.as[Seq[String]]("data-columns") - val downsamplers = defn.as[Seq[String]]("downsamplers") - val rowKeyColumns = defn.as[Seq[String]]("row-key-columns") - - Dataset.make(dataset, partitionCols, dataCols, rowKeyColumns, downsamplers, DatasetOptions.fromConfig(options)).get - } + val rowKeyIDs = Seq(0) // First or timestamp column is always the row keys /** * Creates a new Dataset with various options @@ -231,32 +199,30 @@ object Dataset { * @param name The name of the dataset * @param partitionColumns list of partition columns in name:type form * @param dataColumns list of data columns in name:type form - * @param keyColumns the key column names, no :type * @return a Dataset, or throws an exception if a dataset cannot be created */ def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], keyColumns: Seq[String]): Dataset = - apply(name, partitionColumns, dataColumns, keyColumns, Nil, DatasetOptions.DefaultOptions) + apply(name, partitionColumns, dataColumns, Nil, DatasetOptions.DefaultOptions) def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], - keyColumns: Seq[String], downsamplers: Seq[String], options : DatasetOptions): Dataset = - make(name, partitionColumns, dataColumns, keyColumns, downsamplers, options).badMap(BadSchemaError).toTry.get + make(name, partitionColumns, dataColumns, downsamplers, options).badMap(BadSchemaError).toTry.get def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], - keyColumn: String, options: DatasetOptions): Dataset = - apply(name, partitionColumns, dataColumns, Seq(keyColumn), Nil, options) + options: DatasetOptions): Dataset = + apply(name, partitionColumns, dataColumns, Nil, options) def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String]): Dataset = - apply(name, partitionColumns, dataColumns, "timestamp", DatasetOptions.DefaultOptions) + apply(name, partitionColumns, dataColumns, DatasetOptions.DefaultOptions) sealed trait BadSchema case class BadDownsampler(msg: String) extends BadSchema @@ -268,6 +234,7 @@ object Dataset { case class UnknownRowKeyColumn(keyColumn: String) extends BadSchema case class IllegalMapColumn(reason: String) extends BadSchema case class NoTimestampRowKey(colName: String, colType: String) extends BadSchema + case class HashConflict(detail: String) extends BadSchema case class BadSchemaError(badSchema: BadSchema) extends Exception(badSchema.toString) @@ -304,14 +271,6 @@ object Dataset { nothing2 <- validatePartMapCol() } yield () } - def getRowKeyIDs(dataColumns: Seq[Column], rowKeyColNames: Seq[String]): Seq[Int] Or BadSchema = { - val indices = rowKeyColNames.map { rowKeyCol => dataColumns.indexWhere(_.name == rowKeyCol) } - indices.zip(rowKeyColNames).find(_._1 < 0) match { - case Some((_, col)) => Bad(UnknownRowKeyColumn(col)) - case None => Good(indices) - } - } - def validateTimeSeries(dataColumns: Seq[Column], rowKeyIDs: Seq[Int]): Unit Or BadSchema = dataColumns(rowKeyIDs.head).columnType match { case Column.ColumnType.LongColumn => Good(()) @@ -344,20 +303,13 @@ object Dataset { def make(name: String, partitionColNameTypes: Seq[String], dataColNameTypes: Seq[String], - keyColumnNames: Seq[String], downsamplerNames: Seq[String] = Seq.empty, - options: DatasetOptions = DatasetOptions.DefaultOptions): Dataset Or BadSchema = { - - for {partColumns <- Column.makeColumnsFromNameTypeList(partitionColNameTypes, PartColStartIndex) - dataColumns <- Column.makeColumnsFromNameTypeList(dataColNameTypes) - _ <- validateMapColumn(partColumns, dataColumns) - rowKeyIDs <- getRowKeyIDs(dataColumns, keyColumnNames) - downsamplers <- validateDownsamplers(downsamplerNames) - _ <- validateTimeSeries(dataColumns, rowKeyIDs)} - yield { - Dataset(name, partColumns, dataColumns, rowKeyIDs, downsamplers, None, options) - } + options: DatasetOptions = DatasetOptions.DefaultOptions, + valueColumn: Option[String] = None): Dataset Or BadSchema = { + // Default value column is the last data column name + val valueCol = valueColumn.getOrElse(dataColNameTypes.last.split(":").head) + for { partSchema <- PartitionSchema.make(partitionColNameTypes, options) + dataSchema <- DataSchema.make(name, dataColNameTypes, downsamplerNames, valueCol) } + yield { Dataset(name, Schema(partSchema, dataSchema)) } } - - } diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala new file mode 100644 index 0000000000..87eefa4c93 --- /dev/null +++ b/core/src/main/scala/filodb.core/metadata/Schemas.scala @@ -0,0 +1,242 @@ +package filodb.core.metadata + +import com.typesafe.config.Config +import net.ceedubs.ficus.Ficus._ +import org.scalactic._ + +import filodb.core.binaryrecord2.{RecordBuilder, RecordComparator, RecordSchema} +import filodb.core.downsample.ChunkDownsampler +import filodb.core.query.ColumnInfo +import filodb.core.store.ChunkSetInfo +import filodb.core.Types._ +import filodb.memory.{BinaryRegion, MemFactory} +import filodb.memory.format.BinaryVector + +/** + * A DataSchema describes the data columns within a time series - the actual data that would vary from sample to + * sample and is encoded. It has a unique hash code for each unique DataSchema. + * One Dataset in FiloDB can comprise multiple DataSchemas. + * One DataSchema should be used for each type of metric or data, such as gauge, histogram, etc. + * The "timestamp" or rowkey is the first column and must be either a LongColumn or TimestampColumn. + * DataSchemas are intended to be built from config through Schemas. + */ +final case class DataSchema private(name: String, + columns: Seq[Column], + downsamplers: Seq[ChunkDownsampler], + hash: Int, + valueColumn: ColumnId) { + val timestampColumn = columns.head + val timestampColID = 0 + + // Used to create a `VectorDataReader` of correct type for a given data column ID; type PtrToDataReader + val readers = columns.map(col => BinaryVector.defaultPtrToReader(col.columnType.clazz)).toArray + + // The number of bytes of chunkset metadata including vector pointers in memory + val chunkSetInfoSize = ChunkSetInfo.chunkSetInfoSize(columns.length) + val blockMetaSize = chunkSetInfoSize + 4 + + def valueColName: String = columns(valueColumn).name +} + +/** + * A PartitionSchema is the schema describing the unique "key" of each time series, such as labels. + * The columns inside PartitionSchema are used for distribution and sharding, as well as filtering and searching + * for time series during querying. + * There should only be ONE PartitionSchema across the entire Database. + */ +final case class PartitionSchema(columns: Seq[Column], + predefinedKeys: Seq[String], + options: DatasetOptions) { + import PartitionSchema._ + + val binSchema = new RecordSchema(columns.map(c => ColumnInfo(c.name, c.columnType)), Some(0), predefinedKeys) + + private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, binSchema, DefaultContainerSize) + + /** + * Creates a PartitionKey (BinaryRecord v2) from individual parts. Horribly slow, use for testing only. + */ + def partKey(parts: Any*): Array[Byte] = { + val offset = partKeyBuilder.addFromObjects(parts: _*) + val bytes = binSchema.asByteArray(partKeyBuilder.allContainers.head.base, offset) + partKeyBuilder.reset() + bytes + } +} + +object DataSchema { + import Dataset._ + import java.nio.charset.StandardCharsets.UTF_8 + + def validateValueColumn(dataColumns: Seq[Column], valueColName: String): ColumnId Or BadSchema = { + val index = dataColumns.indexWhere(_.name == valueColName) + if (index < 0) Bad(BadColumnName(valueColName, s"$valueColName not a valid data column")) + else Good(index) + } + + /** + * Generates a unique 16-bit hash from the column names and types. Sensitive to order. + */ + def genHash(columns: Seq[Column]): Int = { + var hash = 7 + for { col <- columns } { + // Use XXHash to get high quality hash for column name. String.hashCode is _horrible_ + hash = 31 * hash + (BinaryRegion.hash32(col.name.getBytes(UTF_8)) * col.columnType.hashCode) + } + hash & 0x0ffff + } + + /** + * Creates and validates a new DataSchema + * @param name The name of the schema + * @param dataColNameTypes list of data columns in name:type[:params] form + * @return Good(Dataset) or Bad(BadSchema) + */ + def make(name: String, + dataColNameTypes: Seq[String], + downsamplerNames: Seq[String] = Seq.empty, + valueColumn: String): DataSchema Or BadSchema = { + + for { dataColumns <- Column.makeColumnsFromNameTypeList(dataColNameTypes) + downsamplers <- validateDownsamplers(downsamplerNames) + valueColID <- validateValueColumn(dataColumns, valueColumn) + _ <- validateTimeSeries(dataColumns, Seq(0)) } + yield { + DataSchema(name, dataColumns, downsamplers, genHash(dataColumns), valueColID) + } + } + + /** + * Parses a DataSchema from config object, like this: + * {{{ + * { + * prometheus { + * columns = ["timestamp:ts", "value:double:detectDrops=true"] + * value-column = "value" + * downsamplers = [] + * } + * } + * }}} + * + * From the example above, pass in "prometheus" as the schemaName. + * It is advisable to parse the outer config of all schemas using `.as[Map[String, Config]]` + */ + def fromConfig(schemaName: String, conf: Config): DataSchema Or BadSchema = + make(schemaName, + conf.as[Seq[String]]("columns"), + conf.as[Seq[String]]("downsamplers"), + conf.getString("value-column")) +} + +object PartitionSchema { + import Dataset._ + + val DefaultContainerSize = 10240 + + /** + * Creates and validates a new PartitionSchema + * @param partColNameTypes list of partition columns in name:type[:params] form + * @param options + * @param predefinedKeys + * @return Good(Dataset) or Bad(BadSchema) + */ + def make(partColNameTypes: Seq[String], + options: DatasetOptions, + predefinedKeys: Seq[String] = Seq.empty): PartitionSchema Or BadSchema = { + + for { partColumns <- Column.makeColumnsFromNameTypeList(partColNameTypes, PartColStartIndex) + _ <- validateMapColumn(partColumns, Nil) } + yield { + PartitionSchema(partColumns, predefinedKeys, options) + } + } + + /** + * Parses a PartitionSchema from config. Format: + * {{{ + * columns = ["tags:map"] + * predefined-keys = ["_ns", "app", "__name__", "instance", "dc"] + * options { + * ... # See DatasetOptions parsing format + * } + * }}} + */ + def fromConfig(partConfig: Config): PartitionSchema Or BadSchema = + make(partConfig.as[Seq[String]]("columns"), + DatasetOptions.fromConfig(partConfig.getConfig("options")), + partConfig.as[Option[Seq[String]]]("predefined-keys").getOrElse(Nil)) +} + +/** + * A Schema combines a PartitionSchema with a DataSchema, forming all the columns of a single ingestion record. + */ +final case class Schema(partition: PartitionSchema, data: DataSchema) { + val allColumns = data.columns ++ partition.columns + val ingestionSchema = new RecordSchema(allColumns.map(c => ColumnInfo(c.name, c.columnType)), + Some(data.columns.length), + partition.predefinedKeys) + + val comparator = new RecordComparator(ingestionSchema) + val partKeySchema = comparator.partitionKeySchema +} + +final case class Schemas(part: PartitionSchema, + data: Map[String, DataSchema], + schemas: Map[String, Schema]) + +/** + * Singleton with code to load all schemas from config, verify no conflicts, and ensure there is only + * one PartitionSchema. Config schema: + * {{{ + * filodb { + * partition-schema { + * columns = ["tags:map"] + * } + * schemas { + * prometheus { ... } + * # etc + * } + * } + * }}} + */ +object Schemas { + import Dataset._ + import Accumulation._ + + // Validates all the data schemas from config, including checking hash conflicts, and returns all errors found + def validateDataSchemas(schemas: Map[String, Config]): Seq[DataSchema] Or Seq[(String, BadSchema)] = { + // get all data schemas parsed, combining errors + val parsed = schemas.toSeq.map { case (schemaName, schemaConf) => + DataSchema.fromConfig(schemaName, schemaConf) + .badMap(err => One((schemaName, err))) + }.combined.badMap(_.toSeq) + + // Check for no hash conflicts + parsed.filter { schemas => + val uniqueHashes = schemas.map(_.hash).toSet + if (uniqueHashes.size == schemas.length) Pass + else Fail(Seq(("", HashConflict(s"${schemas.length - uniqueHashes.size + 1} schemas have the same hash")))) + } + } + + /** + * Parse and initialize all the data schemas and single partition schema from config. + * Verifies that all of the schemas are conflict-free (no conflicting hash) and config parses correctly. + * @param config a Config object at the filodb config level, ie "partition-schema" is an entry + */ + def fromConfig(config: Config): Schemas Or Seq[(String, BadSchema)] = { + for { + partSchema <- PartitionSchema.fromConfig(config.getConfig("partition-schema")) + .badMap(e => Seq(("", e))) + dataSchemas <- validateDataSchemas(config.as[Map[String, Config]]("schemas")) + } yield { + val data = new collection.mutable.HashMap[String, DataSchema] + val schemas = new collection.mutable.HashMap[String, Schema] + dataSchemas.foreach { schema => + data(schema.name) = schema + schemas(schema.name) = Schema(partSchema, schema) + } + Schemas(partSchema, data.toMap, schemas.toMap) + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 445059442d..c18b3e1736 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -69,24 +69,24 @@ object TestData { object NamesTestData { def mapper(rows: Seq[Product]): Seq[RowReader] = rows.map(TupleRowReader) - val dataColSpecs = Seq("first:string", "last:string", "age:long:interval=10") - val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, "age", DatasetOptions.DefaultOptions) + val dataColSpecs = Seq("age:long:interval=10", "first:string", "last:string") + val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, DatasetOptions.DefaultOptions) // NOTE: first 3 columns are the data columns, thus names could be used for either complete record // or the data column rowReader - val names = Seq((Some("Khalil"), Some("Mack"), Some(24L), Some(0)), - (Some("Ndamukong"), Some("Suh"), Some(28L), Some(0)), - (Some("Rodney"), Some("Hudson"), Some(25L), Some(0)), - (Some("Jerry"), None, Some(40L), Some(0)), - (Some("Peyton"), Some("Manning"), Some(39L), Some(0)), - (Some("Terrance"), Some("Knighton"), Some(29L), Some(0))) - - val altNames = Seq((Some("Stacy"), Some("McGee"), Some(24L), Some(0)), - (Some("Bruce"), Some("Irvin"), Some(28L), Some(0)), - (Some("Amari"), Some("Cooper"), Some(25L), Some(0)), - (Some("Jerry"), None, Some(40L), Some(0)), - (Some("Derek"), Some("Carr"), Some(39L), Some(0)), - (Some("Karl"), Some("Joseph"), Some(29L), Some(0))) + val names = Seq((Some(24L), Some("Khalil"), Some("Mack"), Some(0)), + (Some(28L), Some("Ndamukong"), Some("Suh"), Some(0)), + (Some(25L), Some("Rodney"), Some("Hudson"), Some(0)), + (Some(40L), Some("Jerry"), None, Some(0)), + (Some(39L), Some("Peyton"), Some("Manning"), Some(0)), + (Some(29L), Some("Terrance"), Some("Knighton"), Some(0))) + + val altNames = Seq((Some(24L), Some("Stacy"), Some("McGee"), Some(0)), + (Some(28L), Some("Bruce"), Some("Irvin"), Some(0)), + (Some(25L), Some("Amari"), Some("Cooper"), Some(0)), + (Some(40L), Some("Jerry"), None, Some(0)), + (Some(39L), Some("Derek"), Some("Carr"), Some(0)), + (Some(29L), Some("Karl"), Some("Joseph"), Some(0))) val firstKey = dataset.timestamp(mapper(names).head) val lastKey = dataset.timestamp(mapper(names).last) @@ -98,12 +98,12 @@ object NamesTestData { def chunkSetStream(data: Seq[Product] = names): Observable[ChunkSet] = TestData.toChunkSetStream(dataset, defaultPartKey, mapper(data)) - val firstNames = names.map(_._1.get) + val firstNames = names.map(_._2.get) val utf8FirstNames = firstNames.map(_.utf8) val sortedFirstNames = Seq("Khalil", "Rodney", "Ndamukong", "Terrance", "Peyton", "Jerry") val sortedUtf8Firsts = sortedFirstNames.map(_.utf8) - val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) + val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, DatasetOptions.DefaultOptions) val lotLotNames = { for { league <- Seq("nfc", "afc") @@ -111,8 +111,8 @@ object NamesTestData { chunk <- 0 to numChunks startRowNo = numChunks * 10000 + chunk * 100 rowNo <- startRowNo to (startRowNo + 99) } - yield { (names(rowNo % 6)._1, names(rowNo % 6)._2, - Some(rowNo.toLong), // the unique row key + yield { (Some(rowNo.toLong), // the unique row key + names(rowNo % 6)._1, names(rowNo % 6)._2, Some(rowNo / 10000 * 10000), // the segment key Some(league)) } // partition key } @@ -181,25 +181,21 @@ object GdeltTestData { } val seqReaders = records.map { record => SeqRowReader(record.productIterator.toList) } - // Dataset1: Partition keys (Actor2Code, Year) / Row key GLOBALEVENTID - val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), "GLOBALEVENTID", DatasetOptions.DefaultOptions) + // NOTE: For all datasets the row key is GLOBALEVENTID + // Dataset1: Partition keys (Actor2Code, Year) + val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), DatasetOptions.DefaultOptions) - // Dataset2: Partition key (MonthYear) / Row keys (GLOBALEVENTID, Actor2Code) - val dataset2 = Dataset("gdelt", Seq(schema(2)), schema.patch(2, Nil, 1), Seq("GLOBALEVENTID", "Actor2Code")) + // Dataset2: Partition key (MonthYear) + val dataset2 = Dataset("gdelt", Seq(schema(2)), schema.patch(2, Nil, 1)) val partBuilder2 = new RecordBuilder(TestData.nativeMem, dataset2.partKeySchema, 10240) // Dataset3: same as Dataset1 for now val dataset3 = dataset1 - // Dataset4: One big partition (Year) with (Actor2Code, GLOBALEVENTID) rowkey - // to easily test row key scans - // val dataset4 = Dataset("gdelt", Seq(schema(3)), schema.patch(3, Nil, 1), Seq("Actor2Code", "GLOBALEVENTID")) - // val partBuilder4 = new RecordBuilder(TestData.nativeMem, dataset4.partKeySchema, 10240) - // Proj 6: partition Actor2Code,Actor2Name to test partition key bitmap indexing val datasetOptions = DatasetOptions.DefaultOptions.copy( shardKeyColumns = Seq( "__name__","_ns")) - val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), "GLOBALEVENTID", datasetOptions) + val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), datasetOptions) val datasetOptionConfig = """ options { @@ -397,7 +393,6 @@ object MachineMetricsData { // A simulation of custom machine metrics data - for testing extractTimeBucket object CustomMetricsData { - val columns = Seq("timestamp:ts", "min:double", "avg:double", "max:double", "count:long") //Partition Key with multiple string columns @@ -405,9 +400,8 @@ object CustomMetricsData { val metricdataset = Dataset.make("tsdbdata", partitionColumns, columns, - Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("metric", "_ns"), "metric", "count")).get + DatasetOptions(Seq("metric", "_ns"), "metric")).get val partKeyBuilder = new RecordBuilder(TestData.nativeMem, metricdataset.partKeySchema, 2048) val defaultPartKey = partKeyBuilder.addFromObjects("metric1", "app1") @@ -416,9 +410,8 @@ object CustomMetricsData { val metricdataset2 = Dataset.make("tsdbdata", partitionColumns2, columns, - Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__"), "__name__", "count")).get + DatasetOptions(Seq("__name__"), "__name__")).get val partKeyBuilder2 = new RecordBuilder(TestData.nativeMem, metricdataset2.partKeySchema, 2048) val defaultPartKey2 = partKeyBuilder2.addFromObjects(Map(ZeroCopyUTF8String("abc") -> ZeroCopyUTF8String("cba"))) @@ -428,9 +421,8 @@ object MetricsTestData { val timeseriesDataset = Dataset.make("timeseries", Seq("tags:map"), Seq("timestamp:ts", "value:double:detectDrops=true"), - Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + DatasetOptions(Seq("__name__", "job"), "__name__")).get val builder = new RecordBuilder(MemFactory.onHeapFactory, timeseriesDataset.ingestionSchema) diff --git a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala index 021cadc36e..dfb3e92ca7 100644 --- a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala +++ b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala @@ -604,9 +604,8 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be val timeseriesDataset = Dataset.make("timeseries", Seq("tags:map"), Seq("timestamp:ts", "value:double"), - Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__", "value", Map("dummy" -> Seq("_bucket")))).get + DatasetOptions(Seq("__name__", "job"), "__name__", Map("dummy" -> Seq("_bucket")))).get val metricName4 = RecordBuilder.trimShardColumn(timeseriesDataset, "__name__", "heap_usage_bucket") metricName4 shouldEqual "heap_usage_bucket" diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index da76c1e532..83299ff140 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -20,16 +20,14 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll val promDataset = Dataset.make("custom1", Seq("someStr:string", "tags:map"), Seq("timestamp:ts", "value:double"), - Seq("timestamp"), Seq("tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)"), - DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + DatasetOptions(Seq("__name__", "job"), "__name__")).get val customDataset = Dataset.make("custom2", Seq("name:string", "namespace:string", "instance:string"), Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist:counter=false"), - Seq("timestamp"), Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)", "hSum(6)"), - DatasetOptions(Seq("name", "namespace"), "name", "total")).get + DatasetOptions(Seq("name", "namespace"), "name")).get private val blockStore = MMD.blockStore protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) @@ -201,12 +199,13 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll // avg val expectedAvgs2 = expectedSums2.zip(expectedCounts2).map { case (sum,count) => sum/count } downsampledData2.map(_._6) shouldEqual expectedAvgs2 - } + import com.softwaremill.quicklens._ + val histDSDownsamplers = Seq("tTime(0)", "tTime(1)", "tTime(2)", "hSum(3)") - val histDSDataset = MMD.histDataset.copy( - downsamplers = Dataset.validateDownsamplers(histDSDownsamplers).get) + val histDSDataset = modify(MMD.histDataset)(_.schema.data.downsamplers) + .setTo(Dataset.validateDownsamplers(histDSDownsamplers).get) // Create downsampleOps for histogram dataset. Samples every 10s, downsample freq 60s/1min val downsampleOpsH = new ShardDownsampler(histDSDataset, 0, true, Seq(60000), NoOpDownsamplePublisher, diff --git a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala index 9225c7a477..b3f6dfe615 100644 --- a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala +++ b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala @@ -1,183 +1,20 @@ package filodb.core.metadata -import com.typesafe.config.ConfigFactory import org.scalatest.{FunSpec, Matchers} import filodb.core._ import filodb.core.query.ColumnInfo +// DEPRECATED: remove soon class DatasetSpec extends FunSpec with Matchers { import Column.ColumnType._ import Dataset._ import NamesTestData._ - describe("Dataset creation") { - it("should load/write dataset from/to config") { - val config = ConfigFactory.parseString( - """ - | dataset = "prometheus" - | - | definition { - | partition-columns = ["tags:map"] - | data-columns = ["timestamp:ts", "value:double"] - | row-key-columns = [ "timestamp" ] - | downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)"] - | } - | - | options { - | shardKeyColumns = [ "__name__", "_ns" ] - | ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - | valueColumn = "value" - | metricColumn = "__name__" - | ignoreTagsOnPartitionKeyHash = [ "le" ] - | copyTags = { } - | } - """.stripMargin) - val dataset = Dataset.fromConfig(config) - - val config2 = dataset.toConfig - val dataset2 = Dataset.fromConfig(config2) - - dataset shouldEqual dataset2 - } - } - describe("Dataset validation") { - it("should return NotNameColonType if column specifiers not name:type format") { - val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "column2", Seq("age")) - resp1.isBad shouldEqual true - resp1.swap.get shouldEqual ColumnErrors(Seq(NotNameColonType("column2"))) - - intercept[BadSchemaError] { - Dataset("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", "age", DatasetOptions.DefaultOptions) - } - } - - it("should return BadColumnParams if name:type:params portion not valid key=value pairs") { - val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", Seq("age")) - resp1.isBad shouldEqual true - resp1.swap.get shouldBe a[ColumnErrors] - val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs - errors should have length 1 - errors.head shouldBe a[BadColumnParams] - } - - it("should return BadColumnParams if required param config not specified") { - val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:foo=bar", Seq("age")) - resp1.isBad shouldEqual true - resp1.swap.get shouldBe a[ColumnErrors] - val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs - errors should have length 1 - errors.head shouldBe a[BadColumnParams] - - val resp2 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:counter=bar", Seq("age")) - resp2.isBad shouldEqual true - resp2.swap.get shouldBe a[ColumnErrors] - val errors2 = resp2.swap.get.asInstanceOf[ColumnErrors].errs - errors2 should have length 1 - errors2.head shouldBe a[BadColumnParams] - } - - it("should return BadColumnName if illegal chars in column name") { - val resp1 = Dataset.make("dataset", Seq("part:string"), Seq("col, umn1:string"), Seq("age")) - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (1) - errors.head shouldBe a[BadColumnName] - } - - it("should return BadColumnType if unsupported type specified in column spec") { - val resp1 = Dataset.make("dataset", Seq("part:linkedlist"), dataColSpecs, Seq("age")) - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (1) - errors.head shouldEqual BadColumnType("linkedlist") - } - - it("should return multiple column spec errors") { - val resp1 = Dataset.make("dataset", Seq("part:string"), - Seq("first:str", "age:long", "la(st):int"), Seq("age")) - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (2) - errors.head shouldEqual BadColumnType("str") - } - - it("should return UnknownRowKeyColumn if row key column(s) not in data columns") { - val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("column2")) - resp1.isBad shouldEqual true - resp1.swap.get shouldEqual UnknownRowKeyColumn("column2") - - val resp2 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("age", "column9")) - resp2.isBad shouldEqual true - resp2.swap.get shouldEqual UnknownRowKeyColumn("column9") - } - - it("should allow MapColumns only in last position of partition key") { - val mapCol = "tags:map" - - // OK: only partition column is map - val ds1 = Dataset("dataset", Seq(mapCol), dataColSpecs, "age", DatasetOptions.DefaultOptions) - ds1.partitionColumns.map(_.name) should equal (Seq("tags")) - - // OK: last partition column is map - val ds2 = Dataset("dataset", Seq("first:string", mapCol), dataColSpecs drop 1, "age", DatasetOptions.DefaultOptions) - ds2.partitionColumns.map(_.name) should equal (Seq("first", "tags")) - - // Not OK: first partition column is map - val resp3 = Dataset.make("dataset", Seq(mapCol, "first:string"), dataColSpecs drop 1, Seq("age")) - resp3.isBad shouldEqual true - resp3.swap.get shouldBe an[IllegalMapColumn] - - // Not OK: map in data columns, not partition column - intercept[BadSchemaError] { - Dataset("dataset", Seq("seg:int"), dataColSpecs :+ mapCol, Seq("age")) } - } - - it("should return NoTimestampRowKey if non timestamp used for row key") { - val ds1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("first")) - ds1.isBad shouldEqual true - ds1.swap.get shouldBe a[NoTimestampRowKey] - } - - it("should return a valid Dataset when a good specification passed") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) - ds.rowKeyIDs shouldEqual Seq(2) - ds.dataColumns should have length (3) - ds.dataColumns.map(_.id) shouldEqual Seq(0, 1, 2) - ds.dataColumns.map(_.columnType) shouldEqual Seq(StringColumn, StringColumn, LongColumn) - ds.partitionColumns should have length (1) - ds.partitionColumns.map(_.columnType) shouldEqual Seq(StringColumn) - ds.partitionColumns.map(_.id) shouldEqual Seq(PartColStartIndex) - Dataset.isPartitionID(ds.partitionColumns.head.id) shouldEqual true - ds.timestampColumn.name shouldEqual "age" - ds.rowKeyRouting shouldEqual Array(2) - } - - it("should return valid Dataset when multiple row key columns specified") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, Seq("age", "first")) - ds.rowKeyIDs shouldEqual Seq(2, 0) - ds.dataColumns should have length (3) - ds.dataColumns.map(_.id) shouldEqual Seq(0, 1, 2) - ds.dataColumns.map(_.columnType) shouldEqual Seq(StringColumn, StringColumn, LongColumn) - ds.partitionColumns should have length (1) - ds.partitionColumns.map(_.columnType) shouldEqual Seq(StringColumn) - ds.timestampColumn.name shouldEqual "age" - ds.rowKeyRouting shouldEqual Array(2, 0) - } - it("should return IDs for column names or seq of missing names") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) - ds.colIDs("first", "age").get shouldEqual Seq(0, 2) + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, DatasetOptions.DefaultOptions) + ds.colIDs("first", "age").get shouldEqual Seq(1, 0) ds.colIDs("part").get shouldEqual Seq(Dataset.PartColStartIndex) @@ -187,11 +24,11 @@ class DatasetSpec extends FunSpec with Matchers { } it("should return ColumnInfos for colIDs") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) - val infos = ds.infosFromIDs(Seq(0, 2)) + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, DatasetOptions.DefaultOptions) + val infos = ds.infosFromIDs(Seq(1, 0)) infos shouldEqual Seq(ColumnInfo("first", StringColumn), ColumnInfo("age", LongColumn)) - val infos2 = ds.infosFromIDs(Seq(PartColStartIndex, 1)) + val infos2 = ds.infosFromIDs(Seq(PartColStartIndex, 2)) infos2 shouldEqual Seq(ColumnInfo("part", StringColumn), ColumnInfo("last", StringColumn)) } diff --git a/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala b/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala new file mode 100644 index 0000000000..8c70410ad3 --- /dev/null +++ b/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala @@ -0,0 +1,249 @@ +package filodb.core.metadata + +import com.typesafe.config.ConfigFactory +import org.scalatest.{FunSpec, Matchers} + +import filodb.core._ + +class SchemasSpec extends FunSpec with Matchers { + import Column.ColumnType._ + import Dataset._ + import NamesTestData._ + + describe("DataSchema") { + it("should return NotNameColonType if column specifiers not name:type format") { + val resp1 = DataSchema.make("dataset", dataColSpecs :+ "column2", Nil, "first") + resp1.isBad shouldEqual true + resp1.swap.get shouldEqual ColumnErrors(Seq(NotNameColonType("column2"))) + } + + it("should return BadColumnParams if name:type:params portion not valid key=value pairs") { + val resp1 = DataSchema.make("dataset", dataColSpecs :+ "column2:a:b", Nil, "first") + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + } + + it("should return BadColumnParams if required param config not specified") { + val resp1 = DataSchema.make("dataset", dataColSpecs :+ "h:hist:foo=bar", Nil, "first") + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + + val resp2 = DataSchema.make("dataset", dataColSpecs :+ "h:hist:counter=bar", Nil, "first") + resp2.isBad shouldEqual true + resp2.swap.get shouldBe a[ColumnErrors] + val errors2 = resp2.swap.get.asInstanceOf[ColumnErrors].errs + errors2 should have length 1 + errors2.head shouldBe a[BadColumnParams] + } + + it("should return BadColumnName if illegal chars in column name") { + val resp1 = DataSchema.make("dataset", Seq("col, umn1:string"), Nil, "first") + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (1) + errors.head shouldBe a[BadColumnName] + } + + it("should return BadColumnType if unsupported type specified in column spec") { + val resp1 = DataSchema.make("dataset", dataColSpecs :+ "part:linkedlist", Nil, "first") + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (1) + errors.head shouldEqual BadColumnType("linkedlist") + } + + it("should return BadColumnName if value column not one of other columns") { + val conf2 = ConfigFactory.parseString(""" + { + columns = ["first:string", "last:string", "age:long"] + value-column = "first2" + downsamplers = [] + }""") + val resp = DataSchema.fromConfig("dataset", conf2) + resp.isBad shouldEqual true + resp.swap.get shouldBe a[BadColumnName] + } + + it("should return multiple column spec errors") { + val resp1 = DataSchema.make("dataset", Seq("first:str", "age:long", "la(st):int"), Nil, "first") + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (2) + errors.head shouldEqual BadColumnType("str") + } + + it("should return NoTimestampRowKey if non timestamp used for row key / first column") { + val ds1 = DataSchema.make("dataset", Seq("first:string", "age:long"), Nil, "first") + ds1.isBad shouldEqual true + ds1.swap.get shouldBe a[NoTimestampRowKey] + } + + it("should return a valid Dataset when a good specification passed") { + val conf2 = ConfigFactory.parseString(""" + { + columns = ["timestamp:ts", "code:long", "event:string"] + value-column = "event" + downsamplers = [] + }""") + val schema = DataSchema.fromConfig("dataset", conf2).get + schema.columns should have length (3) + schema.columns.map(_.id) shouldEqual Seq(0, 1, 2) + schema.columns.map(_.columnType) shouldEqual Seq(TimestampColumn, LongColumn, StringColumn) + schema.timestampColumn.name shouldEqual "timestamp" + } + } + + val partSchemaStr = """{ + columns = ["tags:map"] + predefined-keys = ["_ns", "app", "__name__", "instance", "dc"] + options { + copyTags = {} + ignoreShardKeyColumnSuffixes = {} + ignoreTagsOnPartitionKeyHash = ["le"] + metricColumn = "__name__" + shardKeyColumns = ["__name__", "_ns"] + } + }""" + + describe("PartitionSchema") { + it("should allow MapColumns only in last position of partition key") { + val mapCol = "tags:map" + + // OK: only partition column is map + val ds1 = PartitionSchema.make(Seq(mapCol), DatasetOptions.DefaultOptions).get + ds1.columns.map(_.name) should equal (Seq("tags")) + + // OK: last partition column is map + val ds2 = PartitionSchema.make(Seq("first:string", mapCol), DatasetOptions.DefaultOptions).get + ds2.columns.map(_.name) should equal (Seq("first", "tags")) + + // Not OK: first partition column is map + val resp3 = PartitionSchema.make(Seq(mapCol, "first:string"), DatasetOptions.DefaultOptions) + resp3.isBad shouldEqual true + resp3.swap.get shouldBe an[IllegalMapColumn] + } + + it("should return BadColumnType if unsupported type specified in column spec") { + val resp1 = PartitionSchema.make(Seq("first:strolo"), DatasetOptions.DefaultOptions) + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (1) + errors.head shouldEqual BadColumnType("strolo") + } + + it("should parse config with options") { + val conf2 = ConfigFactory.parseString(partSchemaStr) + val schema = PartitionSchema.fromConfig(conf2).get + + schema.columns.map(_.columnType) shouldEqual Seq(MapColumn) + schema.predefinedKeys shouldEqual Seq("_ns", "app", "__name__", "instance", "dc") + } + } + + describe("Schemas") { + it("should return all errors from every data schema") { + val conf2 = ConfigFactory.parseString(s""" + { + partition-schema $partSchemaStr + schemas { + prom1 { + columns = ["timestamp:tsa", "code:long", "event:string"] + value-column = "event" + downsamplers = [] + } + prom2 { + columns = ["timestamp:ts", "code:long", "ev. ent:string"] + value-column = "foo" + downsamplers = [] + } + prom3 { + columns = ["timestamp:ts", "code:long", "event:string"] + value-column = "event" + downsamplers = [] + } + } + }""") + val resp = Schemas.fromConfig(conf2) + resp.isBad shouldEqual true + val errors = resp.swap.get + errors should have length (2) + errors.map(_._1).toSet shouldEqual Set("prom1", "prom2") + errors.map(_._2.getClass).toSet shouldEqual Set(classOf[ColumnErrors]) + } + + it("should detect and report hash conflicts") { + val conf2 = ConfigFactory.parseString(s""" + { + partition-schema $partSchemaStr + schemas { + prom { + columns = ["timestamp:ts", "value:double"] + value-column = "value" + downsamplers = [] + } + prom2 { + columns = ["timestamp:ts", "value:double"] + value-column = "timestamp" + downsamplers = [] + } + } + }""") + val resp = Schemas.fromConfig(conf2) + resp.isBad shouldEqual true + val errors = resp.swap.get + errors.map(_._2.getClass) shouldEqual Seq(classOf[HashConflict]) + } + + it("should return Schemas instance with every schema parsed") { + val conf2 = ConfigFactory.parseString(s""" + { + partition-schema $partSchemaStr + schemas { + prom { + columns = ["timestamp:ts", "value:double"] + value-column = "value" + downsamplers = [] + } + hist { + columns = ["timestamp:ts", "count:long", "sum:long", "h:hist:counter=true"] + value-column = "h" + downsamplers = [] + } + } + }""") + val schemas = Schemas.fromConfig(conf2).get + + schemas.part.columns.map(_.columnType) shouldEqual Seq(MapColumn) + schemas.part.columns.map(_.id) shouldEqual Seq(PartColStartIndex) + schemas.part.predefinedKeys shouldEqual Seq("_ns", "app", "__name__", "instance", "dc") + Dataset.isPartitionID(schemas.part.columns.head.id) shouldEqual true + + schemas.data.keySet shouldEqual Set("prom", "hist") + schemas.schemas.keySet shouldEqual Set("prom", "hist") + schemas.data("prom").columns.map(_.columnType) shouldEqual Seq(TimestampColumn, DoubleColumn) + schemas.data("prom").columns.map(_.id) shouldEqual Seq(0, 1) + schemas.data("prom").timestampColumn.name shouldEqual "timestamp" + schemas.data("hist").columns.map(_.columnType) shouldEqual + Seq(TimestampColumn, LongColumn, LongColumn, HistogramColumn) + // println(schemas.data.values.map(_.hash)) + } + } +} diff --git a/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala b/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala index 2bff5d1782..7437c24618 100644 --- a/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala +++ b/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala @@ -12,15 +12,15 @@ class KeyFilterSpec extends FunSpec with Matchers { import Filter._ it("should parse values for regular KeyTypes") { - KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc") should equal ("abc".utf8) - KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc".utf8) should equal ("abc".utf8) + KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc") should equal ("abc".utf8) + KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc".utf8) should equal ("abc".utf8) KeyFilter.parseSingleValue(dataset.partitionColumns.head, -15) should equal (-15) - KeyFilter.parseValues(dataset.dataColumns.head, Set("abc", "def")) should equal (Set("abc".utf8, "def".utf8)) + KeyFilter.parseValues(dataset.dataColumns(1), Set("abc", "def")) should equal (Set("abc".utf8, "def".utf8)) } it("should validate equalsFunc for string and other types") { - val eqFunc1 = Equals(KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc")).filterFunc + val eqFunc1 = Equals(KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc")).filterFunc eqFunc1("abc".utf8) should equal (true) eqFunc1("abc") should equal (false) eqFunc1(15) should equal (false) @@ -30,7 +30,7 @@ class KeyFilterSpec extends FunSpec with Matchers { } it("should validate inFunc for string and other types") { - val inFunc1 = In(KeyFilter.parseValues(dataset.dataColumns.head, Set("abc", "def")).toSet).filterFunc + val inFunc1 = In(KeyFilter.parseValues(dataset.dataColumns(1), Set("abc", "def")).toSet).filterFunc inFunc1("abc".utf8) should equal (true) inFunc1("aaa".utf8) should equal (false) inFunc1(15) should equal (false) diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index ea6c060c70..71d2391828 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -28,21 +28,17 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { val policy = new FixedMaxPartitionsEvictionPolicy(100) // Since 99 GDELT rows, this will never evict val memStore = new TimeSeriesMemStore(config, colStore, metaStore, Some(policy)) - val datasetDb2 = dataset.copy(database = Some("unittest2")) - // First create the tables in C* override def beforeAll(): Unit = { super.beforeAll() metaStore.initialize().futureValue colStore.initialize(dataset.ref).futureValue colStore.initialize(GdeltTestData.dataset2.ref).futureValue - colStore.initialize(datasetDb2.ref).futureValue } before { colStore.truncate(dataset.ref).futureValue colStore.truncate(GdeltTestData.dataset2.ref).futureValue - colStore.truncate(datasetDb2.ref).futureValue memStore.reset() metaStore.clearAllData() } @@ -107,21 +103,6 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { rowIt2.map(_.getLong(0)).toSeq should equal (Seq(24L, 28L, 25L, 40L, 39L, 29L)) } - it should "read back rows written in another database" ignore { - whenReady(colStore.write(datasetDb2, chunkSetStream())) { response => - response should equal (Success) - } - - val paramSet = colStore.getScanSplits(dataset.ref, 1) - paramSet should have length (1) - - val rowIt = memStore.scanRows(datasetDb2, Seq(0, 1, 2), FilteredPartitionScan(paramSet.head)) - rowIt.map(_.getLong(2)).toSeq should equal (Seq(24L, 28L, 25L, 40L, 39L, 29L)) - - // Check that original keyspace/database has no data - memStore.scanRows(dataset, Seq(0), partScan).toSeq should have length (0) - } - it should "read back rows written with multi-column row keys" ignore { import GdeltTestData._ val stream = toChunkSetStream(dataset2, partBuilder2.addFromObjects(197901), dataRows(dataset2)) diff --git a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala index 74f3b83ad5..9f817db1fc 100644 --- a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala @@ -19,35 +19,35 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { metaStore.initialize().futureValue } - val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), "timestamp", + val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), DatasetOptions.DefaultOptions) before { metaStore.clearAllData().futureValue } describe("checkpoint api") { it("should allow reading and writing checkpoints for shard") { - val ds = dataset.copy(name = "gdelt-" + UUID.randomUUID()) // Add uuid so tests can be rerun + val ref = DatasetRef("gdelt-" + UUID.randomUUID()) // Add uuid so tests can be rerun // when there is no data for a shard, then return Long.MinValue as the checkpoint - metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual Long.MinValue - metaStore.readCheckpoints(ds.ref, 2).futureValue.isEmpty shouldBe true + metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual Long.MinValue + metaStore.readCheckpoints(ref, 2).futureValue.isEmpty shouldBe true // should be able to insert checkpoints for new groups - metaStore.writeCheckpoint(ds.ref, 2, 1, 10).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 2, 9).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 3, 13).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 4, 5).futureValue shouldEqual Success - metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual 5 + metaStore.writeCheckpoint(ref, 2, 1, 10).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 2, 9).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 3, 13).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 4, 5).futureValue shouldEqual Success + metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual 5 // should be able to update checkpoints for existing groups - metaStore.writeCheckpoint(ds.ref, 2, 1, 12).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 2, 15).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 3, 17).futureValue shouldEqual Success - metaStore.writeCheckpoint(ds.ref, 2, 4, 7).futureValue shouldEqual Success - metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual 7 + metaStore.writeCheckpoint(ref, 2, 1, 12).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 2, 15).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 3, 17).futureValue shouldEqual Success + metaStore.writeCheckpoint(ref, 2, 4, 7).futureValue shouldEqual Success + metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual 7 // should be able to retrieve raw offsets as well - val offsets = metaStore.readCheckpoints(ds.ref, 2).futureValue + val offsets = metaStore.readCheckpoints(ref, 2).futureValue offsets.size shouldEqual 4 offsets(1) shouldEqual 12 offsets(2) shouldEqual 15 @@ -56,7 +56,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { // should be able to clear the table metaStore.clearAllData().futureValue - metaStore.readCheckpoints(ds.ref, 2).futureValue.isEmpty shouldBe true + metaStore.readCheckpoints(ref, 2).futureValue.isEmpty shouldBe true } } diff --git a/doc/ingestion.md b/doc/ingestion.md index a916f714f6..8b30acafcb 100644 --- a/doc/ingestion.md +++ b/doc/ingestion.md @@ -33,43 +33,7 @@ NOTE: for the latest and most up to date, see [timeseries-dev-source.conf](../co ```yaml dataset = "example" - -# Schema used for defining the BinaryRecord used in ingestion and persistence. -# -# Should not change once dataset has been set up on the server and data has been ingested into kafka -# or written to cassandra -definition { - # defines the unique identifier for partition - partition-columns = ["tags:map"] - # Schema of all of the values stored against the partition key. This includes the row-keys as well - data-columns = ["timestamp:ts", "value:double:detectDrops=true"] - # Clustering key for each row. Together with partition key, they form the primary key. - row-key-columns = [ "timestamp" ] - # List of downsamplers for the data columns - downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)"] -} - -# Dataset Options -# -# Should not change once dataset has been set up on the server and data has been ingested into kafka -# or written to cassandra -options { - # These column values are used to identify the shard group for the partition - shardKeyColumns = [ "__name__", "_ns" ] - # suffixes from map values are stripped before hashing - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - # default data column name to be used referencing the value column - valueColumn = "value" - # column name to use when referencing the name column - metricColumn = "__name__" - # these columns are ignored in calculation of full partition key hash - ignoreTagsOnPartitionKeyHash = [ "le" ] - - # These key-names will be replaced in the key map during ingestion/query. - # Temporary workaround. Will be deprecated. - copyTags = { } -} - +schema = "prometheus" # e.g. one shard per kafka partition num-shards = 100 diff --git a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala index 24300e93e7..3a1f20821a 100644 --- a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala +++ b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala @@ -26,7 +26,6 @@ import org.jctools.queues.MpscGrowableArrayQueue import org.rogach.scallop._ import filodb.coordinator.{FilodbSettings, ShardMapper, StoreFactory} -import filodb.core.GlobalConfig import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.gateway.conversion._ @@ -58,8 +57,9 @@ import filodb.timeseries.TestTimeseriesProducer */ 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()) + val settings = new FilodbSettings() + val config = settings.allConfig + val storeFactory = StoreFactory(settings, Scheduler.io()) // ==== Metrics ==== val numInfluxMessages = Kamon.counter("num-influx-messages") @@ -89,7 +89,7 @@ object GatewayServer extends StrictLogging { val sourceConfig = ConfigFactory.parseFile(new java.io.File(userOpts.sourceConfigPath())) val numShards = sourceConfig.getInt("num-shards") - val dataset = Dataset.fromConfig(sourceConfig) + val dataset = settings.datasetFromStream(sourceConfig) // NOTE: the spread MUST match the default spread used in the HTTP module for consistency between querying // and ingestion sharding diff --git a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala index 0ec5d38a42..01f104b100 100644 --- a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala @@ -19,13 +19,7 @@ object PrometheusApiRouteSpec extends ActorSpecConfig { inline-dataset-configs = [ { dataset = "prometheus" - definition { - partition-columns = ["tags:map"] - data-columns = ["timestamp:ts", "value:double"] - row-key-columns = [ "timestamp" ] - downsamplers = [] - } - ${TestData.optionsString} + schema = "prometheus" num-shards = 4 min-num-nodes = 1 sourcefactory = "${classOf[NoOpStreamFactory].getName}" diff --git a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala index 377ea0699d..944cc7f3de 100644 --- a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala @@ -14,8 +14,7 @@ import filodb.core.store.ChunkSet import filodb.memory.format.{vectors => bv, TupleRowReader, UnsafeUtils} object IntSumReadBenchmark { - val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), "rownum", - DatasetOptions.DefaultOptions) + val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), DatasetOptions.DefaultOptions) val rowIt = Iterator.from(0).map { row => (Some(scala.util.Random.nextInt), Some(row.toLong), Some(0)) } val partKey = NamesTestData.defaultPartKey val rowColumns = Seq("int", "rownum", "part") diff --git a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala index 3c84e524ad..2421e9cb61 100644 --- a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala +++ b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala @@ -8,7 +8,6 @@ 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 /** @@ -39,7 +38,7 @@ object TestConsumer extends App { import monix.execution.Scheduler.Implicits.global val ingestConf = IngestionConfig(sourceConf, classOf[KafkaIngestionStreamFactory].getClass.getName).get - val dataset = Dataset.fromConfig(sourceConf) + val dataset = settings.datasetFromStream(sourceConf) val ctor = Class.forName(ingestConf.streamFactoryClass).getConstructors.head val streamFactory = ctor.newInstance().asInstanceOf[IngestionStreamFactory] diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 2cdd60e8bb..519086f144 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -194,6 +194,7 @@ object FiloBuild extends Build { "io.kamon" %% "kamon-akka-remote-2.5" % "1.1.0", logbackDep % Test, scalaTest % Test, + "com.softwaremill.quicklens" %% "quicklens" % "1.4.12" % Test, scalaCheck % "test" ) @@ -274,6 +275,7 @@ object FiloBuild extends Build { lazy val gatewayDeps = commonDeps ++ Seq( scalaxyDep, + logbackDep, "io.monix" %% "monix-kafka-1x" % monixKafkaVersion, "org.rogach" %% "scallop" % "3.1.1" ) diff --git a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala index 350cb50ce0..233252f2c7 100644 --- a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala +++ b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala @@ -9,10 +9,10 @@ import filodb.core.metadata.{Dataset, DatasetOptions} */ object FormatConversion { // An official Prometheus-format Dataset object with a single timestamp and value - val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double")) - .copy(options = DatasetOptions(Seq("__name__", "_ns"), - "__name__", "value", Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), - Map("exporter" -> "_ns", "job" -> "_ns"))) + val options = DatasetOptions(Seq("__name__", "_ns"), + "__name__", Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), + Map("exporter" -> "_ns", "job" -> "_ns")) + val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double"), options) /** * Extracts a java ArrayList of labels from the TimeSeries diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index b4432e3832..e2b8dd5d73 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -41,15 +41,13 @@ final case class SelectRawPartitionsExec(id: String, require(colIds.nonEmpty) protected[filodb] def schemaOfDoExecute(dataset: Dataset): ResultSchema = { - val numRowKeyCols = colIds.zip(dataset.rowKeyIDs).takeWhile { case (a, b) => a == b }.length - // Add the max column to the schema together with Histograms for max computation -- just in case it's needed // But make sure the max column isn't already included histMaxColumn(dataset, colIds).filter { mId => !(colIds contains mId) } .map { maxColId => - ResultSchema(dataset.infosFromIDs(colIds :+ maxColId), numRowKeyCols, colIDs = (colIds :+ maxColId)) + ResultSchema(dataset.infosFromIDs(colIds :+ maxColId), 1, colIDs = (colIds :+ maxColId)) }.getOrElse { - ResultSchema(dataset.infosFromIDs(colIds), numRowKeyCols, colIDs = colIds) + ResultSchema(dataset.infosFromIDs(colIds), 1, colIDs = colIds) } } @@ -58,8 +56,6 @@ final case class SelectRawPartitionsExec(id: String, queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { - require(colIds.indexOfSlice(dataset.rowKeyIDs) == 0) - val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) source.rangeVectors(dataset, colIds, partMethod, chunkMethod) } From c46813f75ad9c8524b0e58dc010dd8c9fac44618 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 25 Jul 2019 18:06:23 -0700 Subject: [PATCH 03/28] feat(query): rate and increase functions for HistogramColumn (#431) --- .../scala/filodb.core/memstore/MemStore.scala | 4 +- .../src/test/scala/filodb.core/TestData.scala | 11 +-- .../filodb.memory/format/BinaryVector.scala | 16 +--- .../format/vectors/DoubleVector.scala | 32 ++++--- .../format/vectors/Histogram.scala | 29 +++++++ .../format/vectors/HistogramVector.scala | 64 +++++++++++++- .../format/vectors/DoubleVectorTest.scala | 6 +- .../format/vectors/HistogramTest.scala | 3 + .../format/vectors/HistogramVectorTest.scala | 72 ++++++++++++++++ .../query/exec/rangefn/RangeFunction.scala | 4 +- .../query/exec/rangefn/RateFunctions.scala | 85 ++++++++++++++++++- .../exec/rangefn/RateFunctionsSpec.scala | 76 ++++++++++++++++- 12 files changed, 362 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index 80f0ac0a76..cdfb1181a1 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -232,7 +232,9 @@ object MemStore { detectDrops = col.params.as[Option[Boolean]]("detectDrops").getOrElse(false)) case TimestampColumn => bv.LongBinaryVector.timestampVector(memFactory, maxElements) case StringColumn => bv.UTF8Vector.appendingVector(memFactory, maxElements, config.maxBlobBufferSize) - case HistogramColumn => bv.HistogramVector.appending(memFactory, config.maxBlobBufferSize) + case HistogramColumn => val counter = col.params.as[Option[Boolean]]("counter").getOrElse(false) + if (counter) bv.HistogramVector.appendingSect(memFactory, config.maxBlobBufferSize) + else bv.HistogramVector.appending(memFactory, config.maxBlobBufferSize) case other: Column.ColumnType => ??? } }.toArray diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index c18b3e1736..19b4d75678 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -52,7 +52,7 @@ object TestData { val sourceConf = ConfigFactory.parseString(sourceConfStr) val storeConf = StoreConfig(sourceConf.getConfig("store")) - val nativeMem = new NativeMemoryManager(10 * 1024 * 1024) + val nativeMem = new NativeMemoryManager(50 * 1024 * 1024) val optionsString = """ options { @@ -364,12 +364,13 @@ object MachineMetricsData { private val histBufferPool = new WriteBufferPool(TestData.nativeMem, histDataset, TestData.storeConf) // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData - def histogramRV(startTS: Long, pubFreq: Long = 10000L, numSamples: Int = 100, numBuckets: Int = 8): + def histogramRV(startTS: Long, pubFreq: Long = 10000L, numSamples: Int = 100, numBuckets: Int = 8, + ds: Dataset = histDataset, pool: WriteBufferPool = histBufferPool): (Stream[Seq[Any]], RawDataRangeVector) = { val histData = linearHistSeries(startTS, 1, pubFreq.toInt, numBuckets).take(numSamples) - val container = records(histDataset, histData).records - val part = TimeSeriesPartitionSpec.makePart(0, histDataset, partKey=histPartKey, bufferPool=histBufferPool) - container.iterate(histDataset.ingestionSchema).foreach { row => part.ingest(row, histIngestBH) } + val container = records(ds, histData).records + val part = TimeSeriesPartitionSpec.makePart(0, ds, partKey=histPartKey, bufferPool=pool) + container.iterate(ds.ingestionSchema).foreach { row => part.ingest(row, histIngestBH) } // Now flush and ingest the rest to ensure two separate chunks part.switchBuffers(histIngestBH, encode = true) (histData, RawDataRangeVector(null, part, AllChunkScan, Array(0, 3))) // select timestamp and histogram columns only diff --git a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala index aff9f7855a..268286d4cf 100644 --- a/memory/src/main/scala/filodb.memory/format/BinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/BinaryVector.scala @@ -132,16 +132,9 @@ trait VectorDataReader extends AvailableReader { * CorrectionMeta stores the type-specific correction amount for counter vectors. * It is also used to propagate and accumulate corrections as one iterates through vectors. */ -trait CorrectionMeta { - def correction: Double -} - -object NoCorrection extends CorrectionMeta { - def correction: Double = 0.0 -} +trait CorrectionMeta -// TODO: move to DoubleVector -final case class DoubleCorrection(lastValue: Double, correction: Double = 0.0) extends CorrectionMeta +object NoCorrection extends CorrectionMeta /** * Trait that extends VectorDataReaders with methods assisting counter-like vectors that may reset or need correction @@ -158,15 +151,14 @@ trait CounterVectorReader extends VectorDataReader { def detectDropAndCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta /** - * Updates the CorrectionMeta from this vector from startElement to the end, + * Updates the CorrectionMeta from this vector, * namely the total correction over that period plus the last value of the vector. * Returns a new CorrectionMeta which has lastValue plus the total running correction including prev correction. * IE this method should upward adjust the correction in meta based on any new corrections detected in this chunk. * @param vector the BinaryVectorPtr native address of the BinaryVector - * @param startElement the starting element # to seek out correction metadata * @param meta CorrectionMeta with total running correction info. lastValue is ignored */ - def updateCorrection(vector: BinaryVectorPtr, startElement: Int, meta: CorrectionMeta): CorrectionMeta + def updateCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta } // An efficient iterator for the bitmap mask, rotating a mask as we go diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 3c087fb6be..806a52dab4 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -10,7 +10,6 @@ import filodb.memory.format._ import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.memory.format.Encodings._ - object DoubleVector { /** * Creates a new MaskedDoubleAppendingVector, allocating a byte array of the right size for the max # @@ -93,6 +92,8 @@ object DoubleVector { } } +final case class DoubleCorrection(lastValue: Double, correction: Double = 0.0) extends CorrectionMeta + /** * An iterator optimized for speed and type-specific to avoid boxing. * It has no hasNext() method - because it is guaranteed to visit every element, and this way @@ -169,10 +170,12 @@ trait DoubleVectorDataReader extends CounterVectorReader { } // Default implementation for vectors with no correction - def updateCorrection(vector: BinaryVectorPtr, startElement: Int, meta: CorrectionMeta): CorrectionMeta = { - // Return the last value and simply pass on the previous correction value - DoubleCorrection(apply(vector, length(vector) - 1), meta.correction) - } + def updateCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta = + meta match { + // Return the last value and simply pass on the previous correction value + case DoubleCorrection(_, corr) => DoubleCorrection(apply(vector, length(vector) - 1), corr) + case NoCorrection => DoubleCorrection(apply(vector, length(vector) - 1), 0.0) + } /** * Retrieves the element at position/row n, with counter correction, taking into account a previous @@ -180,9 +183,10 @@ trait DoubleVectorDataReader extends CounterVectorReader { * values starting no lower than the initial correction factor in correctionMeta. * NOTE: this is a default implementation for vectors having no correction */ - def correctedValue(vector: BinaryVectorPtr, n: Int, correctionMeta: CorrectionMeta): Double = { + def correctedValue(vector: BinaryVectorPtr, n: Int, meta: CorrectionMeta): Double = meta match { // Since this is a vector that needs no correction, simply add the correction amount to the original value - apply(vector, n) + correctionMeta.correction + case DoubleCorrection(_, corr) => apply(vector, n) + corr + case NoCorrection => apply(vector, n) } } @@ -274,13 +278,21 @@ extends DoubleVectorDataReader { override def correctedValue(vector: BinaryVectorPtr, n: Int, correctionMeta: CorrectionMeta): Double = { assert(vector == vect) - corrected(n) + correctionMeta.correction // corrected value + any carryover correction + correctionMeta match { + // corrected value + any carryover correction + case DoubleCorrection(_, corr) => corrected(n) + corr + case NoCorrection => corrected(n) + } } - override def updateCorrection(vector: BinaryVectorPtr, startElement: Int, meta: CorrectionMeta): CorrectionMeta = { + override def updateCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta = { assert(vector == vect) + val lastValue = apply(vector, length(vector) - 1) // Return the last (original) value and all corrections onward - DoubleCorrection(apply(vector, length(vector) - 1), meta.correction + _correction) + meta match { + case DoubleCorrection(_, corr) => DoubleCorrection(lastValue, corr + _correction) + case NoCorrection => DoubleCorrection(lastValue, _correction) + } } } 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 da34f21d64..3347b26c01 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/Histogram.scala @@ -131,6 +131,11 @@ trait HistogramWithBuckets extends Histogram { } } +object HistogramWithBuckets { + // Can be used for an initial "empty" or "null" Histogram. No buckets - can't aggregate or do anything + val empty = LongHistogram(HistogramBuckets.emptyBuckets, Array[Long]()) +} + final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) extends HistogramWithBuckets { final def bucketValue(no: Int): Double = values(no).toDouble final def serialize(intoBuf: Option[MutableDirectBuffer] = None): MutableDirectBuffer = { @@ -138,6 +143,27 @@ final case class LongHistogram(buckets: HistogramBuckets, values: Array[Long]) e BinaryHistogram.writeDelta(buckets, values, buf) buf } + + /** + * Adds the buckets from other into this LongHistogram + */ + final def add(other: LongHistogram): Unit = { + assert(other.buckets == buckets) + for { b <- 0 until numBuckets optimized } { + values(b) += other.values(b) + } + } + + final def populateFrom(other: LongHistogram): Unit = { + require(other.buckets == buckets) + System.arraycopy(other.values, 0, values, 0, values.size) + } + + final def copy: LongHistogram = { + val newHist = LongHistogram.empty(buckets) + newHist.populateFrom(this) + newHist + } } object LongHistogram { @@ -146,6 +172,9 @@ object LongHistogram { val res = NibblePack.unpackToSink(packedValues, NibblePack.DeltaSink(values), bucketDef.numBuckets) if (res == NibblePack.Ok) Some(LongHistogram(bucketDef, values)) else None } + + def empty(bucketDef: HistogramBuckets): LongHistogram = + LongHistogram(bucketDef, new Array[Long](bucketDef.numBuckets)) } /** 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 435e72517a..7aa8d9fd0e 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/HistogramVector.scala @@ -474,13 +474,19 @@ class RowHistogramReader(histVect: Ptr.U8) extends HistogramReader with SectionR } } +final case class HistogramCorrection(lastValue: LongHistogram, correction: LongHistogram) extends CorrectionMeta + +trait CounterHistogramReader extends HistogramReader with CounterVectorReader { + def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets +} + /** - * A reader for SectDelta encoded histograms + * A reader for SectDelta encoded histograms, including correction/drop functionality */ -class SectDeltaHistogramReader(histVect: Ptr.U8) extends RowHistogramReader(histVect) { +class SectDeltaHistogramReader(histVect: Ptr.U8) extends RowHistogramReader(histVect) with CounterHistogramReader { // baseHist is section base histogram; summedHist used to compute base + delta or other sums - private val summedHist = MutableHistogram.empty(buckets) - private val baseHist = LongHistogram(buckets, new Array[Long](buckets.numBuckets)) + private val summedHist = LongHistogram.empty(buckets) + private val baseHist = summedHist.copy private val baseSink = NibblePack.DeltaSink(baseHist.values) // override setSection: also set the base histogram for getting real value @@ -514,4 +520,54 @@ class SectDeltaHistogramReader(histVect: Ptr.U8) extends RowHistogramReader(hist // TODO: optimized summing. It's wasteful to apply the base + delta math so many times ... // instead add delta + base * n if possible. However, do we care about sum performance on increasing histograms? + + def detectDropAndCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta = meta match { + case NoCorrection => meta // No last value, cannot compare. Just pass it on. + case h @ HistogramCorrection(lastValue, correction) => + val firstValue = apply(0) + // Last value is the new delta correction. Also assume the correction is already a cloned independent thing + if (firstValue < lastValue) { + correction.add(lastValue) + h + } else { meta } + } + + // code to go through and build a list of corrections and corresponding index values.. (dropIndex, correction) + lazy val corrections = { + var index = 0 + // Step 1: build an iterator of (starting-index, section) for each section + iterateSections.map { case (s) => val o = (index, s); index += s.numElements; o }.collect { + case (i, s) if i > 0 && s.sectionType == Section.TypeDrop => + (i, apply(i - 1).asInstanceOf[LongHistogram].copy) + }.toBuffer + } + + def updateCorrection(vector: BinaryVectorPtr, meta: CorrectionMeta): CorrectionMeta = { + val correction = meta match { + case NoCorrection => LongHistogram.empty(buckets) + case HistogramCorrection(_, corr) => corr + } + // Go through and add corrections + corrections.foreach { case (_, corr) => correction.add(corr) } + + HistogramCorrection(apply(length - 1).asInstanceOf[LongHistogram].copy, correction) + } + + def correctedValue(n: Int, meta: CorrectionMeta): HistogramWithBuckets = { + // Get the raw histogram value -- COPY it as we need to modify it, and also + // calling corrections below might modify the temporary value + val h = apply(n).asInstanceOf[LongHistogram].copy + + // Apply any necessary corrections + corrections.foreach { case (ci, corr) => + if (ci <= n) h.add(corr) + } + + // Plus any carryover previous corrections + meta match { + case NoCorrection => h + case HistogramCorrection(_, corr) => h.add(corr) + h + } + } } \ No newline at end of file diff --git a/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala index 06040de128..73b4ee7bba 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/DoubleVectorTest.scala @@ -221,10 +221,10 @@ class DoubleVectorTest extends NativeVectorTest { } it("should updateCorrections correctly") { - reader.updateCorrection(frozen, 0, NoCorrection) shouldEqual DoubleCorrection(7678, 0.0) + reader.updateCorrection(frozen, NoCorrection) shouldEqual DoubleCorrection(7678, 0.0) val corr1 = DoubleCorrection(999.9, 50.0) - reader.updateCorrection(frozen, 0, corr1) shouldEqual DoubleCorrection(7678, 50.0) + reader.updateCorrection(frozen, corr1) shouldEqual DoubleCorrection(7678, 50.0) } it("should detect drops with DoubleCounterAppender and carry flag to optimized version") { @@ -273,7 +273,7 @@ class DoubleVectorTest extends NativeVectorTest { reader.correctedValue(sc, 2, corr1) shouldEqual 161.5 reader.correctedValue(sc, 4, corr1) shouldEqual (21.1 + 102.5 + 50) - reader.updateCorrection(sc, 0, corr1) shouldEqual DoubleCorrection(21.1, 102.5 + 50.0) + reader.updateCorrection(sc, corr1) shouldEqual DoubleCorrection(21.1, 102.5 + 50.0) } it("should read out length and values correctly for corrected vectors") { diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala index 3cb580edfd..f49d14eb14 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramTest.scala @@ -24,6 +24,9 @@ object HistogramTest { LongHistogram(customScheme, buckets.take(customScheme.numBuckets).map(_.toLong)) } + val correction1 = LongHistogram(bucketScheme, Array(1, 2, 3, 4, 5, 6, 7, 8)) + val correction2 = LongHistogram(bucketScheme, Array(2, 4, 6, 8, 10, 12, 14, 18)) + val quantile50Result = Seq(37.333333333333336, 10.8, 8.666666666666666, 28.75) } diff --git a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala index 2a68372b7f..4403c74a5a 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/HistogramVectorTest.scala @@ -100,6 +100,8 @@ class HistogramVectorTest extends NativeVectorTest { appender.length shouldEqual 0 } + val lastIncrHist = LongHistogram(bucketScheme, incrHistBuckets.last.map(_.toLong)) + it("should append, optimize, and query SectDelta histograms") { val appender = HistogramVector.appendingSect(memFactory, 1024) incrHistBuckets.foreach { rawBuckets => @@ -124,6 +126,12 @@ class HistogramVectorTest extends NativeVectorTest { val h = optReader(i) verifyHistogram(h, i, incrHistBuckets) } + + val oReader2 = optReader.asInstanceOf[CounterHistogramReader] + oReader2.updateCorrection(optimized, NoCorrection) shouldEqual + HistogramCorrection(lastIncrHist, LongHistogram.empty(bucketScheme)) + oReader2.updateCorrection(optimized, HistogramCorrection(lastIncrHist, correction1.copy)) shouldEqual + HistogramCorrection(lastIncrHist, correction1) } it("SectDelta vectors should detect and handle drops correctly") { @@ -147,6 +155,14 @@ class HistogramVectorTest extends NativeVectorTest { verifyHistogram(reader(i), i, incrHistBuckets) verifyHistogram(reader(4 + i), i, incrHistBuckets) } + + // Now, verify updateCorrection will propagate correction correctly + reader.updateCorrection(appender.addr, NoCorrection) shouldEqual + HistogramCorrection(lastIncrHist, lastIncrHist) + val corr2 = correction1.copy + corr2.add(lastIncrHist) + reader.updateCorrection(appender.addr, HistogramCorrection(lastIncrHist, correction1.copy)) shouldEqual + HistogramCorrection(lastIncrHist, corr2) } it("should reject BinaryHistograms of schema different from first schema ingested") { @@ -209,4 +225,60 @@ class HistogramVectorTest extends NativeVectorTest { verifyHistogram(optReader(elemNo), elemNo % (rawLongBuckets.length)) } } + + val incrAppender = HistogramVector.appendingSect(memFactory, 1024) + incrHistBuckets.foreach { rawBuckets => + BinaryHistogram.writeDelta(bucketScheme, rawBuckets.map(_.toLong), buffer) + incrAppender.addData(buffer) shouldEqual Ack + } + val incrAddr = incrAppender.addr + val incrReader = incrAppender.reader.asInstanceOf[CounterHistogramReader] + + it("should detect drop correctly at beginning of chunk and adjust CorrectionMeta") { + incrReader.detectDropAndCorrection(incrAddr, NoCorrection) shouldEqual NoCorrection + + // No drop in first value, correction should be returned unchanged + val meta1 = HistogramCorrection(correction1, correction2.copy) + incrReader.detectDropAndCorrection(incrAddr, meta1) shouldEqual meta1 + + // Drop in first value, correction should be done + val meta2 = HistogramCorrection(lastIncrHist, correction2.copy) + val corr3 = correction2.copy + corr3.add(lastIncrHist) + incrReader.detectDropAndCorrection(incrAddr, meta2) shouldEqual + HistogramCorrection(lastIncrHist, corr3) + } + + it("should return correctedValue with correction adjustment even if vector has no drops") { + val incr1 = LongHistogram(bucketScheme, incrHistBuckets(1).map(_.toLong)) + incrReader.correctedValue(1, NoCorrection) shouldEqual incr1 + + val meta1 = HistogramCorrection(correction1, correction2.copy) + val adjustedHist = correction2.copy + adjustedHist.add(incr1) + incrReader.correctedValue(1, meta1) shouldEqual adjustedHist + } + + it("should return correctedValue with vector with drops") { + val appender2 = HistogramVector.appendingSect(memFactory, 1024) + incrHistBuckets.foreach { rawBuckets => + BinaryHistogram.writeDelta(bucketScheme, rawBuckets.map(_.toLong), buffer) + appender2.addData(buffer) shouldEqual Ack + } + incrHistBuckets.foreach { rawBuckets => + BinaryHistogram.writeDelta(bucketScheme, rawBuckets.map(_.toLong + 15L), buffer) + appender2.addData(buffer) shouldEqual Ack + } + val reader = appender2.reader.asInstanceOf[CounterHistogramReader] + + // now check corrected hist + val incr5 = LongHistogram(bucketScheme, incrHistBuckets(1).map(_.toLong + 15L)) + incr5.add(lastIncrHist) + reader.correctedValue(5, NoCorrection) shouldEqual incr5 + + // value @5 plus correction plus carryover correction from meta + val meta1 = HistogramCorrection(correction1, correction2.copy) + incr5.add(correction2) + reader.correctedValue(5, meta1) shouldEqual incr5 + } } \ No newline at end of file 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 2d94028e8a..ec36706547 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -146,7 +146,7 @@ trait CounterChunkedRangeFunction[R <: MutableRowReader] extends ChunkedRangeFun } // Add any corrections from this chunk, pass on lastValue also to next chunk computation - correctionMeta = ccReader.updateCorrection(valueVector, startRowNum, correctionMeta) + correctionMeta = ccReader.updateCorrection(valueVector, correctionMeta) } /** @@ -298,6 +298,8 @@ object RangeFunction { case None => () => new LastSampleChunkedFunctionH case Some(SumOverTime) if maxCol.isDefined => () => new SumAndMaxOverTimeFuncHD(maxCol.get) case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionH + case Some(Rate) => () => new HistRateFunction + case Some(Increase) => () => new HistIncreaseFunction case _ => ??? } diff --git a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala index eb221e3543..bc0f62d502 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RateFunctions.scala @@ -1,9 +1,11 @@ package filodb.query.exec.rangefn +import scalaxy.loops._ + +import filodb.memory.format.{vectors => bv, CounterVectorReader} import filodb.memory.format.BinaryVector.BinaryVectorPtr -import filodb.memory.format.CounterVectorReader import filodb.query.QueryConfig -import filodb.query.exec.TransientRow +import filodb.query.exec.{TransientHistRow, TransientRow} object RateFunctions { @@ -220,3 +222,82 @@ class ChunkedDeltaFunction extends ChunkedRateFunctionBase { } } } + +/** + * A base class for chunked calculation of rate etc for increasing/counter-like histograms. + * Note that the rate of two histograms is itself a histogram. + * Similar algorithm to ChunkedRateFunctionBase. + * It is O(nWindows * nChunks) which is usually << O(nSamples). + */ +abstract class HistogramRateFunctionBase extends CounterChunkedRangeFunction[TransientHistRow] { + var numSamples = 0 + var lowestTime = Long.MaxValue + var lowestValue: bv.HistogramWithBuckets = bv.HistogramWithBuckets.empty + var highestTime = 0L + var highestValue: bv.HistogramWithBuckets = bv.HistogramWithBuckets.empty + + def isCounter: Boolean + def isRate: Boolean + + override def reset(): Unit = { + numSamples = 0 + lowestTime = Long.MaxValue + lowestValue = bv.HistogramWithBuckets.empty + highestTime = 0L + highestValue = bv.HistogramWithBuckets.empty + super.reset() + } + + def addTimeChunks(vector: BinaryVectorPtr, reader: CounterVectorReader, + startRowNum: Int, endRowNum: Int, + startTime: Long, endTime: Long): Unit = reader match { + case histReader: bv.CounterHistogramReader => + if (startTime < lowestTime || endTime > highestTime) { + numSamples += endRowNum - startRowNum + 1 + if (startTime < lowestTime) { + lowestTime = startTime + lowestValue = histReader.correctedValue(startRowNum, correctionMeta) + } + if (endTime > highestTime) { + highestTime = endTime + highestValue = histReader.correctedValue(endRowNum, correctionMeta) + } + } + case other: CounterVectorReader => + } + + override def apply(windowStart: Long, windowEnd: Long, sampleToEmit: TransientHistRow): Unit = { + if (highestTime > lowestTime) { + // NOTE: It seems in order to match previous code, we have to adjust the windowStart by -1 so it's "inclusive" + // TODO: handle case where schemas are different and we need to interpolate schemas + if (highestValue.buckets == lowestValue.buckets) { + val rateArray = new Array[Double](lowestValue.numBuckets) + for { b <- 0 until rateArray.size optimized } { + rateArray(b) = RateFunctions.extrapolatedRate( + windowStart - 1, windowEnd, numSamples, + lowestTime, lowestValue.bucketValue(b), + highestTime, highestValue.bucketValue(b), + isCounter, isRate) + } + sampleToEmit.setValues(windowEnd, bv.MutableHistogram(lowestValue.buckets, rateArray)) + } else { + sampleToEmit.setValues(windowEnd, bv.HistogramWithBuckets.empty) + } + } else { + sampleToEmit.setValues(windowEnd, bv.HistogramWithBuckets.empty) + } + } + + def apply(endTimestamp: Long, sampleToEmit: TransientHistRow): Unit = ??? +} + +class HistRateFunction extends HistogramRateFunctionBase { + def isCounter: Boolean = true + def isRate: Boolean = true +} + +class HistIncreaseFunction extends HistogramRateFunctionBase { + def isCounter: Boolean = true + def isRate: Boolean = false +} + diff --git a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala index 111ba860c4..da1f7e5890 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/RateFunctionsSpec.scala @@ -2,7 +2,11 @@ package filodb.query.exec.rangefn import scala.util.Random -import filodb.query.exec.{ChunkedWindowIteratorD, QueueBasedWindow, TransientRow} +import filodb.core.{MachineMetricsData, TestData} +import filodb.core.memstore.{TimeSeriesPartition, WriteBufferPool} +import filodb.core.metadata.Dataset +import filodb.memory.format.vectors.MutableHistogram +import filodb.query.exec.{ChunkedWindowIteratorD, ChunkedWindowIteratorH, QueueBasedWindow, TransientRow} import filodb.query.util.IndexedArrayQueue class RateFunctionsSpec extends RawDataWindowingSpec { @@ -148,7 +152,6 @@ class RateFunctionsSpec extends RawDataWindowingSpec { val it = new ChunkedWindowIteratorD(flatRV, endTs, 10000, endTs, endTs - startTs, new ChunkedRateFunction, queryConfig) it.next.getDouble(1) shouldEqual 0.0 - } // Also ensures that chunked rate works across chunk boundaries @@ -184,6 +187,75 @@ class RateFunctionsSpec extends RawDataWindowingSpec { } } + val promHistDS = Dataset("histogram", Seq("tags:map"), + Seq("timestamp:ts", "count:long", "sum:long", "h:hist:counter=true")) + val histBufferPool = new WriteBufferPool(TestData.nativeMem, promHistDS, TestData.storeConf) + + it("should compute rate for Histogram RVs") { + val (data, rv) = MachineMetricsData.histogramRV(100000L, numSamples=10, pool=histBufferPool, ds=promHistDS) + val startTs = 99500L + val endTs = 161000L // just past 7th sample + val lastTime = 160000L + val headTime = 100000L + val headHist = data(0)(3).asInstanceOf[MutableHistogram] + val lastHist = data(6)(3).asInstanceOf[MutableHistogram] + val expectedRates = (0 until headHist.numBuckets).map { b => + (lastHist.bucketValue(b) - headHist.bucketValue(b)) / (lastTime - headTime) * 1000 + } + val expected = MutableHistogram(MachineMetricsData.histBucketScheme, expectedRates.toArray) + + // One window, start=end=endTS + val it = new ChunkedWindowIteratorH(rv, endTs, 100000, endTs, endTs - startTs, + new HistRateFunction, queryConfig) + // Scheme should have remained the same + val answer = it.next.getHistogram(1) + answer.numBuckets shouldEqual expected.numBuckets + + // Have to compare each bucket with floating point error tolerance + for { b <- 0 until expected.numBuckets } { + answer.bucketTop(b) shouldEqual expected.bucketTop(b) + answer.bucketValue(b) shouldEqual expected.bucketValue(b) +- errorOk + } + } + + it("should compute rate for Histogram RVs with drop") { + val (data, rv) = MachineMetricsData.histogramRV(100000L, numSamples=7, pool=histBufferPool, ds=promHistDS) + + // Inject a few more samples with original data, which means a drop + val part = rv.partition.asInstanceOf[TimeSeriesPartition] + val dropData = data.map(d => (d.head.asInstanceOf[Long] + 70000L) +: d.drop(1)) + val container = MachineMetricsData.records(promHistDS, dropData).records + container.iterate(promHistDS.ingestionSchema).foreach { row => part.ingest(row, ingestBlockHolder) } + part.switchBuffers(ingestBlockHolder, encode = true) + + + val startTs = 99500L + val endTs = 171000L // just past 8th sample, the first dropped one + val lastTime = 170000L + val headTime = 100000L + val headHist = data(0)(3).asInstanceOf[MutableHistogram] + val corrHist = data(6)(3).asInstanceOf[MutableHistogram] + val lastHist = headHist.copy // 8th sample == first sample + correction + lastHist.add(corrHist) + val expectedRates = (0 until headHist.numBuckets).map { b => + (lastHist.bucketValue(b) - headHist.bucketValue(b)) / (lastTime - headTime) * 1000 + } + val expected = MutableHistogram(MachineMetricsData.histBucketScheme, expectedRates.toArray) + + // One window, start=end=endTS + val it = new ChunkedWindowIteratorH(rv, endTs, 110000, endTs, endTs - startTs, + new HistRateFunction, queryConfig) + // Scheme should have remained the same + val answer = it.next.getHistogram(1) + answer.numBuckets shouldEqual expected.numBuckets + + // Have to compare each bucket with floating point error tolerance + for { b <- 0 until expected.numBuckets } { + answer.bucketTop(b) shouldEqual expected.bucketTop(b) + answer.bucketValue(b) shouldEqual expected.bucketValue(b) +- errorOk + } +} + it ("irate should work when start and end are outside window") { val startTs = 8071950L val endTs = 8163070L From bf0ae4cef94f82c4f2fbe576e2c8307ef41e17f4 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Mon, 29 Jul 2019 17:28:30 -0700 Subject: [PATCH 04/28] Revert "feat(core, *): PartitionSchema and DataSchemas: multi-schema part I (#440)" (#452) This reverts commit 0bb54ff9ba7de896cc58f3b89fceca1e09555cea. --- README.md | 8 +- cli/src/main/scala/filodb.cli/CliMain.scala | 21 +- conf/timeseries-128shards-source.conf | 17 +- conf/timeseries-dev-source.conf | 38 ++- conf/timeseries-ds-1m-dev-source.conf | 17 +- conf/timeseries-standalonetest-source.conf | 17 +- .../filodb.coordinator/FilodbSettings.scala | 23 +- .../filodb.coordinator/NodeClusterActor.scala | 2 +- .../NodeCoordinatorActor.scala | 4 +- .../queryengine/Utils.scala | 14 +- .../queryengine2/QueryEngine.scala | 10 +- .../ClusterRecoverySpec.scala | 22 +- .../queryengine2/QueryEngineSpec.scala | 6 +- core/src/main/resources/filodb-defaults.conf | 43 --- .../downsample/ShardDownsampler.scala | 8 +- .../scala/filodb.core/metadata/Dataset.scala | 134 +++++++--- .../scala/filodb.core/metadata/Schemas.scala | 242 ----------------- .../src/test/scala/filodb.core/TestData.scala | 64 +++-- .../binaryrecord2/BinaryRecordSpec.scala | 3 +- .../downsample/ShardDownsamplerSpec.scala | 13 +- .../filodb.core/metadata/DatasetSpec.scala | 175 +++++++++++- .../filodb.core/metadata/SchemasSpec.scala | 249 ------------------ .../filodb.core/query/KeyFilterSpec.scala | 10 +- .../filodb.core/store/ColumnStoreSpec.scala | 19 ++ .../filodb.core/store/MetaStoreSpec.scala | 32 +-- doc/ingestion.md | 38 ++- .../scala/filodb/gateway/GatewayServer.scala | 8 +- .../filodb/http/PrometheusApiRouteSpec.scala | 8 +- .../filodb.jmh/IntSumReadBenchmark.scala | 3 +- .../scala/filodb/kafka/TestConsumer.scala | 3 +- project/FiloBuild.scala | 2 - .../filodb/prometheus/FormatConversion.scala | 8 +- .../query/exec/SelectRawPartitionsExec.scala | 8 +- 33 files changed, 532 insertions(+), 737 deletions(-) delete mode 100644 core/src/main/scala/filodb.core/metadata/Schemas.scala delete mode 100644 core/src/test/scala/filodb.core/metadata/SchemasSpec.scala diff --git a/README.md b/README.md index afa4deb031..25d69b9239 100644 --- a/README.md +++ b/README.md @@ -327,16 +327,10 @@ The **partition key** differentiates time series and also controls distribution The data points use a configurable schema consisting of multiple columns. Each column definition consists of `name:columntype`, with optional parameters. For examples, see the examples below, or see the introductory walk-through above where two datasets are created. -A single partition key schema is used for a running FiloDB cluster, though multiple data schemas may be supported. These schemas are defined in the config file - see the `partition-schema` and `schemas` sections of `filodb-defaults.conf`. The CLI command `validateSchemas` may be run to verify schemas defined in config files, as follows: - - ./filo-cli -Dconfig.file=conf/timeseries-filodb-server.conf --command validateSchemas - ### Dataset Configuration THIS IS IMPORTANT TO READ AND UNDERSTAND. -Each "dataset" ingests one stream or Kafka topic of raw time series data, and is also the unit of isolation. Each dataset contains its own offheap memory, and can have independent data retention and ingestion properties. - Datasets are setup and loaded into the server via configuration files referred to by application.conf loaded by the server. See `conf/timeseries-dev-source.conf` for an example. It is important to note that some aspects of the dataset, like its column definition are immutable. This is primarily because the data columns are used to populate persistent @@ -350,6 +344,7 @@ that part of the cluster could be with the old config and the rest could have ne ### Prometheus FiloDB Schema for Operational Metrics * Partition key = `tags:map` +* Row key = `timestamp` * Columns: `timestamp:ts,value:double:detectDrops=true` The above is the classic Prometheus-compatible schema. It supports indexing on any tag. Thus standard Prometheus queries that filter by a tag such as `hostname` or `datacenter` for example would work fine. Note that the Prometheus metric name is encoded as a key `__name__`, which is the Prometheus standard when exporting tags. @@ -363,6 +358,7 @@ NOTE: `detectDrops=true` allows for proper and efficient rate calculation on Pro Let's say that one had a metrics client, such as CodaHale metrics, which pre-aggregates percentiles and sends them along with the metric. If we used the Prometheus schema, each percentile would wind up in its own time series. This is fine, but incurs significant overhead as the partition key has to then be sent with each percentile over the wire. Instead we can have a schema which includes all the percentiles together when sending the data: * Partition key = `metricName:string,tags:map` +* Row key = `timestamp` * Columns: `timestamp:ts,min:double,max:double,p50:double,p90:double,p95:double,p99:double,p999:double` ### Data Modelling and Performance Considerations diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 45880ffd9c..08fa757d7c 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -9,13 +9,12 @@ import scala.util.Try import com.opencsv.CSVWriter import com.quantifind.sumac.{ArgMain, FieldArgs} import monix.reactive.Observable -import org.scalactic._ import filodb.coordinator._ import filodb.coordinator.client._ import filodb.coordinator.client.QueryCommands.{SpreadChange, SpreadProvider, StaticSpreadProvider} import filodb.core._ -import filodb.core.metadata.{Column, Schemas} +import filodb.core.metadata.Column import filodb.memory.format.RowReader import filodb.prometheus.ast.{InMemoryParam, TimeRangeParams, TimeStepParams, WriteBuffersParam} import filodb.prometheus.parse.Parser @@ -145,8 +144,6 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { val (remote, ref) = getClientAndRef(args) dumpShardStatus(remote, ref) - case Some("validateSchemas") => validateSchemas() - case Some("timeseriesMetadata") => require(args.host.nonEmpty && args.dataset.nonEmpty && args.matcher.nonEmpty, "--host, --dataset and --matcher must be defined") val remote = Client.standaloneClient(system, args.host.get, args.port) @@ -205,22 +202,6 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { } } - def validateSchemas(): Unit = { - Schemas.fromConfig(config) match { - case Good(Schemas(partSchema, data, _)) => - println("Schema validated.\nPartition schema:") - partSchema.columns.foreach(c => println(s"\t$c")) - data.foreach { case (schemaName, sch) => - println(s"Schema $schemaName:") - sch.columns.foreach(c => println(s"\t$c")) - } - case Bad(errors) => - println(s"Schema validation errors:") - errors.foreach { case (name, err) => println(s"$name\t$err")} - exitCode = 1 - } - } - final case class QOptions(limit: Int, sampleLimit: Int, everyN: Option[Int], timeout: FiniteDuration, shardOverrides: Option[Seq[Int]], diff --git a/conf/timeseries-128shards-source.conf b/conf/timeseries-128shards-source.conf index 8e003126b5..3415a25338 100644 --- a/conf/timeseries-128shards-source.conf +++ b/conf/timeseries-128shards-source.conf @@ -1,5 +1,20 @@ dataset = "prometheus" - schema = "prometheus" + + definition { + partition-columns = ["tags:map"] + data-columns = ["timestamp:ts", "value:double:detectDrops=true"] + row-key-columns = [ "timestamp" ] + downsamplers = [ ] + } + + options { + shardKeyColumns = [ "__name__", "_ns" ] + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + valueColumn = "value" + metricColumn = "__name__" + ignoreTagsOnPartitionKeyHash = [ "le" ] + copyTags = { } + } num-shards = 128 min-num-nodes = 2 diff --git a/conf/timeseries-dev-source.conf b/conf/timeseries-dev-source.conf index 776e6e06a7..e0159d13da 100644 --- a/conf/timeseries-dev-source.conf +++ b/conf/timeseries-dev-source.conf @@ -1,11 +1,45 @@ dataset = "prometheus" - # Name of schema used for this dataset stream. See filodb.schemas in filodb-defaults or any other server conf - schema = "prometheus" + # Schema used for defining the BinaryRecord used in ingestion and persistence. + # + # Should not change once dataset has been set up on the server and data has been ingested into kafka + # or written to cassandra + definition { + # defines the unique identifier for partition + partition-columns = ["tags:map"] + # Schema of all of the values stored against the partition key. This includes the row-keys as well + data-columns = ["timestamp:ts", "value:double:detectDrops=true"] + # Clustering key for each row. Together with partition key, they form the primary key. + row-key-columns = [ "timestamp" ] + # List of downsamplers for the data columns + downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)" ] + } + + # Dataset Options + # + # Should not change once dataset has been set up on the server and data has been ingested into kafka + # or written to cassandra + options { + # These column values are used to identify the shard group for the partition + shardKeyColumns = [ "__name__", "_ns" ] + # suffixes from map values are stripped before hashing + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + # default data column name to be used referencing the value column + valueColumn = "value" + # column name to use when referencing the name column + metricColumn = "__name__" + # these columns are ignored in calculation of full partition key hash + ignoreTagsOnPartitionKeyHash = [ "le" ] + + # These key-names will be replaced in the key map during ingestion/query. + # Temporary workaround. Will be deprecated. + copyTags = { } + } # Should not change once dataset has been set up on the server and data has been persisted to cassandra num-shards = 4 + min-num-nodes = 2 # Length of chunks to be written, roughly sourcefactory = "filodb.kafka.KafkaIngestionStreamFactory" diff --git a/conf/timeseries-ds-1m-dev-source.conf b/conf/timeseries-ds-1m-dev-source.conf index 33b428d0d3..9c6d7903e7 100644 --- a/conf/timeseries-ds-1m-dev-source.conf +++ b/conf/timeseries-ds-1m-dev-source.conf @@ -1,5 +1,20 @@ dataset = "prometheus_ds_1m" - schema = "prom-ds-gauge" + + definition { + partition-columns = ["tags:map"] + data-columns = [ "timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double" ] + row-key-columns = [ "timestamp" ] + downsamplers = [ ] + } + + options { + shardKeyColumns = [ "__name__", "_ns" ] + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + valueColumn = "avg" + metricColumn = "__name__" + ignoreTagsOnPartitionKeyHash = [ "le" ] + copyTags = { } + } num-shards = 4 min-num-nodes = 2 diff --git a/conf/timeseries-standalonetest-source.conf b/conf/timeseries-standalonetest-source.conf index 4a0baa0fb2..1ac94dfeb0 100644 --- a/conf/timeseries-standalonetest-source.conf +++ b/conf/timeseries-standalonetest-source.conf @@ -1,5 +1,20 @@ dataset = "prometheus" - schema = "prometheus" + + definition { + partition-columns = ["tags:map"] + data-columns = ["timestamp:ts", "value:double:detectDrops=true"] + row-key-columns = [ "timestamp" ] + downsamplers = [ ] + } + + options { + shardKeyColumns = [ "__name__", "_ns" ] + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + valueColumn = "value" + metricColumn = "__name__" + ignoreTagsOnPartitionKeyHash = [ "le" ] + copyTags = { } + } num-shards = 4 min-num-nodes = 2 diff --git a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala index b5ad3b01ce..c77611f8e5 100755 --- a/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala +++ b/coordinator/src/main/scala/filodb.coordinator/FilodbSettings.scala @@ -6,15 +6,14 @@ import scala.concurrent.duration.FiniteDuration import akka.actor.{ActorPath, Address, RootActorPath} import com.typesafe.config.{Config, ConfigFactory} import net.ceedubs.ficus.Ficus._ -import org.scalactic._ -import filodb.core.metadata.{Dataset, Schemas} import filodb.core.GlobalConfig /** Settings for the FiloCluster Akka Extension which gets * config from `GlobalConfig`. Uses Ficus. */ final class FilodbSettings(val conf: Config) { + def this() = this(ConfigFactory.empty) ConfigFactory.invalidateCaches() @@ -45,32 +44,18 @@ final class FilodbSettings(val conf: Config) { /** The timeout to use to resolve an actor ref for new nodes. */ val ResolveActorTimeout = config.as[FiniteDuration]("tasks.timeouts.resolve-actor") - val datasetConfPaths = config.as[Seq[String]]("dataset-configs") + val datasets = config.as[Seq[String]]("dataset-configs") /** * Returns IngestionConfig/dataset configuration from parsing dataset-configs file paths. * If those are empty, then parse the "streams" config key for inline configs. */ val streamConfigs: Seq[Config] = - if (datasetConfPaths.nonEmpty) { - datasetConfPaths.map { d => ConfigFactory.parseFile(new java.io.File(d)) } + if (datasets.nonEmpty) { + datasets.map { d => ConfigFactory.parseFile(new java.io.File(d)) } } else { config.as[Seq[Config]]("inline-dataset-configs") } - - val schemas = Schemas.fromConfig(config) match { - case Good(sch) => sch - case Bad(errs) => throw new RuntimeException("Errors parsing schemas:\n" + - errs.map { case (ds, err) => s"Schema $ds\t$err" }.mkString("\n")) - } - - // Creates a Dataset from a stream config. NOTE: this is a temporary thing to keep old code using Dataset - // compatible and minimize changes. The Dataset name is taken from the dataset/stream config, but the schema - // underneath points to one of the schemas above and schema may have a different name. The approach below - // allows one schema (with one schema name) to be shared amongst datasets using different names. - def datasetFromStream(streamConf: Config): Dataset = - Dataset(streamConf.getString("dataset"), - schemas.schemas(streamConf.getString("schema"))) } /** Consistent naming: allows other actors to accurately filter diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala index b4644d74f0..66d64ba0f2 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeClusterActor.scala @@ -226,7 +226,7 @@ private[filodb] class NodeClusterActor(settings: FilodbSettings, // shard and dataset state can be recovered correctly. First all the datasets are set up. // Then shard state is recovered, and finally cluster membership events are replayed. settings.streamConfigs.foreach { config => - val dataset = settings.datasetFromStream(config) + val dataset = Dataset.fromConfig(config) val ingestion = IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get initializeDataset(dataset, ingestion, None) } diff --git a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala index 24e8646261..46cf5d67a4 100644 --- a/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/NodeCoordinatorActor.scala @@ -157,7 +157,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, def datasetHandlers: Receive = LoggingReceive { case CreateDataset(datasetObj, db) => // used only for unit testing now - createDataset(sender(), datasetObj) + createDataset(sender(), datasetObj.copy(database = db)) case TruncateDataset(ref) => truncateDataset(sender(), ref) @@ -211,7 +211,7 @@ private[filodb] final class NodeCoordinatorActor(metaStore: MetaStore, if (!datasetsInitialized) { logger.debug(s"Initializing stream configs: ${settings.streamConfigs}") settings.streamConfigs.foreach { config => - val dataset = settings.datasetFromStream(config) + val dataset = Dataset.fromConfig(config) val ingestion = IngestionConfig(config, NodeClusterActor.noOpSource.streamFactoryClass).get initializeDataset(dataset, ingestion) } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 7087984e2d..5687cde7e3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -12,7 +12,7 @@ import monix.reactive.Observable import org.scalactic._ import filodb.coordinator.ShardMapper -import filodb.core.ErrorResponse +import filodb.core.{ErrorResponse, Types} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} @@ -28,6 +28,18 @@ final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) exten object Utils extends StrictLogging { import filodb.coordinator.client.QueryCommands._ import TrySugar._ + import filodb.coordinator.client.QueryCommands._ + + /** + * Convert column name strings into columnIDs. NOTE: column names should not include row key columns + * as those are automatically prepended. + */ + def getColumnIDs(dataset: Dataset, colStrs: Seq[String]): Seq[Types.ColumnId] Or ErrorResponse = + dataset.colIDs(colStrs: _*).badMap(missing => UndefinedColumns(missing.toSet)) + .map { ids => // avoid duplication if first ids are already row keys + if (ids.take(dataset.rowKeyIDs.length) == dataset.rowKeyIDs) { ids } + else { dataset.rowKeyIDs ++ ids } + } /** * Validates a PartitionQuery, returning a set of PartitionScanMethods with shard numbers. diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 9ce1771a96..126c9cbbd3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -328,7 +328,7 @@ class QueryEngine(dataset: Dataset, options: QueryOptions, lp: RawChunkMeta, spreadProvider : SpreadProvider): PlanResult = { // Translate column name to ID and validate here - val colName = if (lp.column.isEmpty) dataset.schema.data.valueColName else lp.column + val colName = if (lp.column.isEmpty) dataset.options.valueColumn else lp.column val colID = dataset.colIDs(colName).get.head val renamedFilters = renameMetricFilter(lp.filters) val metaExec = shardsFromFilters(renamedFilters, options, spreadProvider).map { shard => @@ -368,13 +368,13 @@ class QueryEngine(dataset: Dataset, * as those are automatically prepended. */ private def getColumnIDs(dataset: Dataset, cols: Seq[String]): Seq[Types.ColumnId] = { - val realCols = if (cols.isEmpty) Seq(dataset.schema.data.valueColName) else cols + val realCols = if (cols.isEmpty) Seq(dataset.options.valueColumn) else cols val ids = dataset.colIDs(realCols: _*) .recover(missing => throw new BadQueryException(s"Undefined columns $missing")) .get - // avoid duplication if first id is a timestamp, otherwise add it - if (ids.take(1) == Dataset.rowKeyIDs) { ids } - else { Dataset.rowKeyIDs ++ ids } + // avoid duplication if first ids are already row keys + if (ids.take(dataset.rowKeyIDs.length) == dataset.rowKeyIDs) { ids } + else { dataset.rowKeyIDs ++ ids } } private def toChunkScanMethod(rangeSelector: RangeSelector): ChunkScanMethod = { diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index b8826f8b3c..74ff8634c9 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -24,23 +24,17 @@ object ClusterRecoverySpecConfig extends MultiNodeConfig { val ourConf = s""" filodb { memstore.groups-per-shard = 4 - partition-schema { - columns = ["Actor2Code:string", "Actor2Name:string"] - predefined-keys = [] - ${GdeltTestData.datasetOptionConfig} - } - schemas { - gdelt { - columns = ["GLOBALEVENTID:long", "SQLDATE:long", "MonthYear:int", - "Year:int", "NumArticles:int", "AvgTone:double"] - value-column = "AvgTone" - downsamplers = [] - } - } inline-dataset-configs = [ { dataset = "gdelt" - schema = "gdelt" + definition { + partition-columns = ["Actor2Code:string", "Actor2Name:string"] + data-columns = ["GLOBALEVENTID:long", "SQLDATE:long", "MonthYear:int", + "Year:int", "NumArticles:int", "AvgTone:double"] + row-key-columns = [ "GLOBALEVENTID" ] + downsamplers = [] + } + ${GdeltTestData.datasetOptionConfig} num-shards = 2 min-num-nodes = 2 sourcefactory = "${classOf[sources.CsvStreamFactory].getName}" diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index 0f8f093406..0bf0abdf06 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -126,12 +126,10 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - import com.softwaremill.quicklens._ - it("should rename Prom __name__ filters if dataset has different metric column") { // Custom QueryEngine with different dataset with different metric name - val datasetOpts = dataset.options.copy(metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job")) - val dataset2 = dataset.modify(_.schema.partition.options).setTo(datasetOpts) + val dataset2 = dataset.copy(options = dataset.options.copy( + metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job"))) val engine2 = new QueryEngine(dataset2, mapperRef) // materialized exec plan diff --git a/core/src/main/resources/filodb-defaults.conf b/core/src/main/resources/filodb-defaults.conf index 86e48c54a8..db392592ad 100644 --- a/core/src/main/resources/filodb-defaults.conf +++ b/core/src/main/resources/filodb-defaults.conf @@ -14,49 +14,6 @@ filodb { # ] inline-dataset-configs = [] - # Definition of cluster-wide partition key scheme. The partition key defines each unique time series, - # such as labels or tags, and is used for sharding time series across the cluster. - # The below definition is standard for Prometheus schema - partition-schema { - # Typical column types used: map, string. Also possible: ts,long,double - columns = ["tags:map"] - - # Predefined keys allow space to be saved for over the wire tags with the given keys - predefined-keys = ["_ns", "app", "__name__", "instance", "dc", "le"] - - options { - copyTags = {} - ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } - ignoreTagsOnPartitionKeyHash = ["le"] - metricColumn = "__name__" - shardKeyColumns = [ "__name__", "_ns" ] - } - } - - # Definitions of possible data schemas to be used in all datasets - # Each one must have a unique name and column schema. - # FiloDB will refuse to start if the schema definitions have errors. Use the validateSchemas CLI command to check. - schemas { - prometheus { - # Each column def is of name:type format. Type may be ts,long,double,string,int - # The first column must be ts or long - columns = ["timestamp:ts", "value:double:detectDrops=true"] - - # Default column to query using PromQL - value-column = "value" - - # Downsampling configuration. See doc/downsampling.md - downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)" ] - } - - # Used for downsampled gauge data - prom-ds-gauge { - columns = [ "timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double" ] - value-column = "avg" - downsamplers = [] - } - } - tasks { # Frequency with which new shard maps are published shardmap-publish-frequency = 5s diff --git a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala index 4146db844e..9af51c7586 100644 --- a/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala +++ b/core/src/main/scala/filodb.core/downsample/ShardDownsampler.scala @@ -25,11 +25,9 @@ class ShardDownsampler(dataset: Dataset, resolutions: Seq[Int], publisher: DownsamplePublisher, stats: TimeSeriesShardStats) extends StrictLogging { - private val downsamplers = dataset.schema.data.downsamplers - if (enabled) { logger.info(s"Downsampling enabled for dataset=${dataset.ref} shard=$shardNum with " + - s"following downsamplers: ${downsamplers.map(_.encoded)} at resolutions: $resolutions") + s"following downsamplers: ${dataset.downsamplers.map(_.encoded)} at resolutions: $resolutions") } else { logger.info(s"Downsampling disabled for dataset=${dataset.ref} shard=$shardNum") } @@ -60,7 +58,7 @@ class ShardDownsampler(dataset: Dataset, */ private[downsample] def downsampleIngestSchema(): RecordSchema = { // The name of the column in downsample record does not matter at the ingestion side. Type does matter. - val downsampleCols = downsamplers.map { d => ColumnInfo(s"${d.name.entryName}", d.colType) } + val downsampleCols = dataset.downsamplers.map { d => ColumnInfo(s"${d.name.entryName}", d.colType) } new RecordSchema(downsampleCols ++ dataset.partKeySchema.columns, Some(downsampleCols.size), dataset.ingestionSchema.predefinedKeys) } @@ -96,7 +94,7 @@ class ShardDownsampler(dataset: Dataset, val endRowNum = Math.min(tsReader.ceilingIndex(vecPtr, pEnd), chunkset.numRows - 1) builder.startNewRecord() // for each downsampler, add downsample column value - downsamplers.foreach { + dataset.downsamplers.foreach { case d: TimeChunkDownsampler => builder.addLong(d.downsampleChunk(part, chunkset, startRowNum, endRowNum)) case d: DoubleChunkDownsampler => diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index 83c66b0ec8..575111132e 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -7,11 +7,12 @@ import net.ceedubs.ficus.Ficus._ import org.scalactic._ import filodb.core._ -import filodb.core.binaryrecord2._ +import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2, _} import filodb.core.downsample.ChunkDownsampler import filodb.core.query.ColumnInfo +import filodb.core.store.ChunkSetInfo import filodb.memory.{BinaryRegion, MemFactory} -import filodb.memory.format.{RowReader, TypedIterator, ZeroCopyUTF8String => ZCUTF8} +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. @@ -25,30 +26,31 @@ import filodb.memory.format.{RowReader, TypedIterator, ZeroCopyUTF8String => ZCU * partition columns: metricName:string, tags:map * data columns: timestamp:long, value:double * - * NOTE: this data structure will be deprecated slowly in favor of PartitionSchema/DataSchema. - * NOTE2: name is used for ingestion stream name, which is separate from the name of the schema. - * * The Column IDs (used for querying) for data columns are numbered starting with 0, and for partition * columns are numbered starting with PartColStartIndex. This means position is the same or easily derived * * The rowKeyIDs are the dataColumns IDs/positions for the "row key", typically a timestamp column but * something which makes a value unique within a partition and describes a range of data in a chunk. */ -final case class Dataset(name: String, schema: Schema) { - val options = schema.partition.options - val dataColumns = schema.data.columns - val partitionColumns = schema.partition.columns - - val ref = DatasetRef(name, None) - val rowKeyColumns = schema.data.columns take 1 - - val ingestionSchema = schema.ingestionSchema - val comparator = schema.comparator - val partKeySchema = schema.partKeySchema +final case class Dataset(name: String, + partitionColumns: Seq[Column], + dataColumns: Seq[Column], + rowKeyIDs: Seq[Int], + downsamplers: Seq[ChunkDownsampler], + database: Option[String] = None, + options: DatasetOptions = DatasetOptions.DefaultOptions) { + require(rowKeyIDs.nonEmpty) + val ref = DatasetRef(name, database) + val rowKeyColumns = rowKeyIDs.map(dataColumns) + val rowKeyRouting = rowKeyIDs.toArray + + val ingestionSchema = RecordSchema2.ingestion(this) // TODO: add predefined keys yo + val comparator = new RecordComparator(ingestionSchema) + val partKeySchema = comparator.partitionKeySchema // Used to create a `VectorDataReader` of correct type for a given data column ID; type PtrToDataReader - val dataReaders = schema.data.readers - val numDataColumns = schema.data.columns.length + val dataReaders = dataColumns.map(col => BinaryVector.defaultPtrToReader(col.columnType.clazz)).toArray + val numDataColumns = dataColumns.length // Used for ChunkSetReader.binarySearchKeyChunks val rowKeyOrdering = CompositeReaderOrdering(rowKeyColumns.map(_.columnType.keyType)) @@ -57,8 +59,8 @@ final case class Dataset(name: String, schema: Schema) { val timestampColID = timestampColumn.id // The number of bytes of chunkset metadata including vector pointers in memory - val chunkSetInfoSize = schema.data.chunkSetInfoSize - val blockMetaSize = schema.data.blockMetaSize + val chunkSetInfoSize = ChunkSetInfo.chunkSetInfoSize(dataColumns.length) + val blockMetaSize = chunkSetInfoSize + 4 private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, partKeySchema, 10240) @@ -79,8 +81,8 @@ final case class Dataset(name: String, schema: Schema) { */ def partColIterator(columnID: Int, base: Any, offset: Long): TypedIterator = { val partColPos = columnID - Dataset.PartColStartIndex - require(Dataset.isPartitionID(columnID) && partColPos < schema.partition.columns.length) - schema.partition.columns(partColPos).columnType match { + require(Dataset.isPartitionID(columnID) && partColPos < partitionColumns.length) + partitionColumns(partColPos).columnType match { case StringColumn => new PartKeyUTF8Iterator(partKeySchema, base, offset, partColPos) case LongColumn => new PartKeyLongIterator(partKeySchema, base, offset, partColPos) case TimestampColumn => new PartKeyLongIterator(partKeySchema, base, offset, partColPos) @@ -91,7 +93,7 @@ final case class Dataset(name: String, schema: Schema) { /** * Extracts a timestamp out of a RowReader, assuming data columns are first (ingestion order) */ - final def timestamp(dataRowReader: RowReader): Long = dataRowReader.getLong(0) + def timestamp(dataRowReader: RowReader): Long = dataRowReader.getLong(rowKeyIDs.head) import Accumulation._ import OptionSugar._ @@ -99,8 +101,8 @@ final case class Dataset(name: String, schema: Schema) { * Returns the column IDs for the named columns or the missing column names */ def colIDs(colNames: String*): Seq[Int] Or Seq[String] = - colNames.map { n => schema.data.columns.find(_.name == n).map(_.id) - .orElse { schema.partition.columns.find(_.name == n).map(_.id) } + colNames.map { n => dataColumns.find(_.name == n).map(_.id) + .orElse { partitionColumns.find(_.name == n).map(_.id) } .toOr(One(n)) } .combined.badMap(_.toSeq) @@ -110,23 +112,38 @@ final case class Dataset(name: String, schema: Schema) { * over the input RowReader to return data columns corresponding to dataset definition. */ def dataRouting(colNames: Seq[String]): Array[Int] = - schema.data.columns.map { c => colNames.indexOf(c.name) }.toArray + dataColumns.map { c => colNames.indexOf(c.name) }.toArray /** * Returns a routing from data + partition columns (as required for ingestion BinaryRecords) to * the input RowReader columns whose names are passed in. */ def ingestRouting(colNames: Seq[String]): Array[Int] = - dataRouting(colNames) ++ schema.partition.columns.map { c => colNames.indexOf(c.name) } + dataRouting(colNames) ++ partitionColumns.map { c => colNames.indexOf(c.name) } /** Returns the Column instance given the ID */ def columnFromID(columnID: Int): Column = - if (Dataset.isPartitionID(columnID)) { schema.partition.columns(columnID - Dataset.PartColStartIndex) } - else { schema.data.columns(columnID) } + if (Dataset.isPartitionID(columnID)) { partitionColumns(columnID - Dataset.PartColStartIndex) } + else { dataColumns(columnID) } /** Returns ColumnInfos from a set of column IDs. Throws exception if ID is invalid */ def infosFromIDs(ids: Seq[Types.ColumnId]): Seq[ColumnInfo] = ids.map(columnFromID).map { c => ColumnInfo(c.name, c.columnType) } + + def toConfig: Config = { + + val c = Map[String, Any] ( + "dataset" -> name, + "definition.partition-columns" -> partitionColumns.map(_.toStringNotation).asJava, + "definition.data-columns" -> dataColumns.map(_.toStringNotation).asJava, + "definition.row-key-columns" -> rowKeyIDs.map(dataColumns(_).name).asJava, + "definition.downsamplers" -> downsamplers.map(_.encoded).asJava + ).asJava + + ConfigFactory.parseMap(c).withFallback(ConfigFactory.parseString( + s""" options ${options} """)) + + } } /** @@ -135,6 +152,7 @@ final case class Dataset(name: String, schema: Schema) { */ case class DatasetOptions(shardKeyColumns: Seq[String], metricColumn: String, + valueColumn: String, // TODO: deprecate these options once we move all input to Telegraf/Influx // They are needed only to differentiate raw Prometheus-sourced data ignoreShardKeyColumnSuffixes: Map[String, Seq[String]] = Map.empty, @@ -149,6 +167,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], val map: Map[String, Any] = Map( "shardKeyColumns" -> shardKeyColumns.asJava, "metricColumn" -> metricColumn, + "valueColumn" -> valueColumn, "ignoreShardKeyColumnSuffixes" -> ignoreShardKeyColumnSuffixes.mapValues(_.asJava).asJava, "ignoreTagsOnPartitionKeyHash" -> ignoreTagsOnPartitionKeyHash.asJava, @@ -169,6 +188,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], object DatasetOptions { val DefaultOptions = DatasetOptions(shardKeyColumns = Nil, metricColumn = "__name__", + valueColumn = "value", // defaults that work well for Prometheus ignoreShardKeyColumnSuffixes = Map("__name__" -> Seq("_bucket", "_count", "_sum")), @@ -181,6 +201,7 @@ object DatasetOptions { def fromConfig(config: Config): DatasetOptions = DatasetOptions(shardKeyColumns = config.as[Seq[String]]("shardKeyColumns"), metricColumn = config.getString("metricColumn"), + valueColumn = config.getString("valueColumn"), ignoreShardKeyColumnSuffixes = config.as[Map[String, Seq[String]]]("ignoreShardKeyColumnSuffixes"), ignoreTagsOnPartitionKeyHash = config.as[Seq[String]]("ignoreTagsOnPartitionKeyHash"), @@ -191,7 +212,18 @@ object DatasetOptions { * Contains many helper functions especially pertaining to Dataset creation and validation. */ object Dataset { - val rowKeyIDs = Seq(0) // First or timestamp column is always the row keys + def fromConfig(config: Config): Dataset = { + val dataset = config.getString("dataset") + val defn = config.getConfig("definition") + val options = config.getConfig("options") + + val partitionCols = defn.as[Seq[String]]("partition-columns") + val dataCols = defn.as[Seq[String]]("data-columns") + val downsamplers = defn.as[Seq[String]]("downsamplers") + val rowKeyColumns = defn.as[Seq[String]]("row-key-columns") + + Dataset.make(dataset, partitionCols, dataCols, rowKeyColumns, downsamplers, DatasetOptions.fromConfig(options)).get + } /** * Creates a new Dataset with various options @@ -199,30 +231,32 @@ object Dataset { * @param name The name of the dataset * @param partitionColumns list of partition columns in name:type form * @param dataColumns list of data columns in name:type form + * @param keyColumns the key column names, no :type * @return a Dataset, or throws an exception if a dataset cannot be created */ def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], keyColumns: Seq[String]): Dataset = - apply(name, partitionColumns, dataColumns, Nil, DatasetOptions.DefaultOptions) + apply(name, partitionColumns, dataColumns, keyColumns, Nil, DatasetOptions.DefaultOptions) def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], + keyColumns: Seq[String], downsamplers: Seq[String], options : DatasetOptions): Dataset = - make(name, partitionColumns, dataColumns, downsamplers, options).badMap(BadSchemaError).toTry.get + make(name, partitionColumns, dataColumns, keyColumns, downsamplers, options).badMap(BadSchemaError).toTry.get def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String], - options: DatasetOptions): Dataset = - apply(name, partitionColumns, dataColumns, Nil, options) + keyColumn: String, options: DatasetOptions): Dataset = + apply(name, partitionColumns, dataColumns, Seq(keyColumn), Nil, options) def apply(name: String, partitionColumns: Seq[String], dataColumns: Seq[String]): Dataset = - apply(name, partitionColumns, dataColumns, DatasetOptions.DefaultOptions) + apply(name, partitionColumns, dataColumns, "timestamp", DatasetOptions.DefaultOptions) sealed trait BadSchema case class BadDownsampler(msg: String) extends BadSchema @@ -234,7 +268,6 @@ object Dataset { case class UnknownRowKeyColumn(keyColumn: String) extends BadSchema case class IllegalMapColumn(reason: String) extends BadSchema case class NoTimestampRowKey(colName: String, colType: String) extends BadSchema - case class HashConflict(detail: String) extends BadSchema case class BadSchemaError(badSchema: BadSchema) extends Exception(badSchema.toString) @@ -271,6 +304,14 @@ object Dataset { nothing2 <- validatePartMapCol() } yield () } + def getRowKeyIDs(dataColumns: Seq[Column], rowKeyColNames: Seq[String]): Seq[Int] Or BadSchema = { + val indices = rowKeyColNames.map { rowKeyCol => dataColumns.indexWhere(_.name == rowKeyCol) } + indices.zip(rowKeyColNames).find(_._1 < 0) match { + case Some((_, col)) => Bad(UnknownRowKeyColumn(col)) + case None => Good(indices) + } + } + def validateTimeSeries(dataColumns: Seq[Column], rowKeyIDs: Seq[Int]): Unit Or BadSchema = dataColumns(rowKeyIDs.head).columnType match { case Column.ColumnType.LongColumn => Good(()) @@ -303,13 +344,20 @@ object Dataset { def make(name: String, partitionColNameTypes: Seq[String], dataColNameTypes: Seq[String], + keyColumnNames: Seq[String], downsamplerNames: Seq[String] = Seq.empty, - options: DatasetOptions = DatasetOptions.DefaultOptions, - valueColumn: Option[String] = None): Dataset Or BadSchema = { - // Default value column is the last data column name - val valueCol = valueColumn.getOrElse(dataColNameTypes.last.split(":").head) - for { partSchema <- PartitionSchema.make(partitionColNameTypes, options) - dataSchema <- DataSchema.make(name, dataColNameTypes, downsamplerNames, valueCol) } - yield { Dataset(name, Schema(partSchema, dataSchema)) } + options: DatasetOptions = DatasetOptions.DefaultOptions): Dataset Or BadSchema = { + + for {partColumns <- Column.makeColumnsFromNameTypeList(partitionColNameTypes, PartColStartIndex) + dataColumns <- Column.makeColumnsFromNameTypeList(dataColNameTypes) + _ <- validateMapColumn(partColumns, dataColumns) + rowKeyIDs <- getRowKeyIDs(dataColumns, keyColumnNames) + downsamplers <- validateDownsamplers(downsamplerNames) + _ <- validateTimeSeries(dataColumns, rowKeyIDs)} + yield { + Dataset(name, partColumns, dataColumns, rowKeyIDs, downsamplers, None, options) + } } + + } diff --git a/core/src/main/scala/filodb.core/metadata/Schemas.scala b/core/src/main/scala/filodb.core/metadata/Schemas.scala deleted file mode 100644 index 87eefa4c93..0000000000 --- a/core/src/main/scala/filodb.core/metadata/Schemas.scala +++ /dev/null @@ -1,242 +0,0 @@ -package filodb.core.metadata - -import com.typesafe.config.Config -import net.ceedubs.ficus.Ficus._ -import org.scalactic._ - -import filodb.core.binaryrecord2.{RecordBuilder, RecordComparator, RecordSchema} -import filodb.core.downsample.ChunkDownsampler -import filodb.core.query.ColumnInfo -import filodb.core.store.ChunkSetInfo -import filodb.core.Types._ -import filodb.memory.{BinaryRegion, MemFactory} -import filodb.memory.format.BinaryVector - -/** - * A DataSchema describes the data columns within a time series - the actual data that would vary from sample to - * sample and is encoded. It has a unique hash code for each unique DataSchema. - * One Dataset in FiloDB can comprise multiple DataSchemas. - * One DataSchema should be used for each type of metric or data, such as gauge, histogram, etc. - * The "timestamp" or rowkey is the first column and must be either a LongColumn or TimestampColumn. - * DataSchemas are intended to be built from config through Schemas. - */ -final case class DataSchema private(name: String, - columns: Seq[Column], - downsamplers: Seq[ChunkDownsampler], - hash: Int, - valueColumn: ColumnId) { - val timestampColumn = columns.head - val timestampColID = 0 - - // Used to create a `VectorDataReader` of correct type for a given data column ID; type PtrToDataReader - val readers = columns.map(col => BinaryVector.defaultPtrToReader(col.columnType.clazz)).toArray - - // The number of bytes of chunkset metadata including vector pointers in memory - val chunkSetInfoSize = ChunkSetInfo.chunkSetInfoSize(columns.length) - val blockMetaSize = chunkSetInfoSize + 4 - - def valueColName: String = columns(valueColumn).name -} - -/** - * A PartitionSchema is the schema describing the unique "key" of each time series, such as labels. - * The columns inside PartitionSchema are used for distribution and sharding, as well as filtering and searching - * for time series during querying. - * There should only be ONE PartitionSchema across the entire Database. - */ -final case class PartitionSchema(columns: Seq[Column], - predefinedKeys: Seq[String], - options: DatasetOptions) { - import PartitionSchema._ - - val binSchema = new RecordSchema(columns.map(c => ColumnInfo(c.name, c.columnType)), Some(0), predefinedKeys) - - private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, binSchema, DefaultContainerSize) - - /** - * Creates a PartitionKey (BinaryRecord v2) from individual parts. Horribly slow, use for testing only. - */ - def partKey(parts: Any*): Array[Byte] = { - val offset = partKeyBuilder.addFromObjects(parts: _*) - val bytes = binSchema.asByteArray(partKeyBuilder.allContainers.head.base, offset) - partKeyBuilder.reset() - bytes - } -} - -object DataSchema { - import Dataset._ - import java.nio.charset.StandardCharsets.UTF_8 - - def validateValueColumn(dataColumns: Seq[Column], valueColName: String): ColumnId Or BadSchema = { - val index = dataColumns.indexWhere(_.name == valueColName) - if (index < 0) Bad(BadColumnName(valueColName, s"$valueColName not a valid data column")) - else Good(index) - } - - /** - * Generates a unique 16-bit hash from the column names and types. Sensitive to order. - */ - def genHash(columns: Seq[Column]): Int = { - var hash = 7 - for { col <- columns } { - // Use XXHash to get high quality hash for column name. String.hashCode is _horrible_ - hash = 31 * hash + (BinaryRegion.hash32(col.name.getBytes(UTF_8)) * col.columnType.hashCode) - } - hash & 0x0ffff - } - - /** - * Creates and validates a new DataSchema - * @param name The name of the schema - * @param dataColNameTypes list of data columns in name:type[:params] form - * @return Good(Dataset) or Bad(BadSchema) - */ - def make(name: String, - dataColNameTypes: Seq[String], - downsamplerNames: Seq[String] = Seq.empty, - valueColumn: String): DataSchema Or BadSchema = { - - for { dataColumns <- Column.makeColumnsFromNameTypeList(dataColNameTypes) - downsamplers <- validateDownsamplers(downsamplerNames) - valueColID <- validateValueColumn(dataColumns, valueColumn) - _ <- validateTimeSeries(dataColumns, Seq(0)) } - yield { - DataSchema(name, dataColumns, downsamplers, genHash(dataColumns), valueColID) - } - } - - /** - * Parses a DataSchema from config object, like this: - * {{{ - * { - * prometheus { - * columns = ["timestamp:ts", "value:double:detectDrops=true"] - * value-column = "value" - * downsamplers = [] - * } - * } - * }}} - * - * From the example above, pass in "prometheus" as the schemaName. - * It is advisable to parse the outer config of all schemas using `.as[Map[String, Config]]` - */ - def fromConfig(schemaName: String, conf: Config): DataSchema Or BadSchema = - make(schemaName, - conf.as[Seq[String]]("columns"), - conf.as[Seq[String]]("downsamplers"), - conf.getString("value-column")) -} - -object PartitionSchema { - import Dataset._ - - val DefaultContainerSize = 10240 - - /** - * Creates and validates a new PartitionSchema - * @param partColNameTypes list of partition columns in name:type[:params] form - * @param options - * @param predefinedKeys - * @return Good(Dataset) or Bad(BadSchema) - */ - def make(partColNameTypes: Seq[String], - options: DatasetOptions, - predefinedKeys: Seq[String] = Seq.empty): PartitionSchema Or BadSchema = { - - for { partColumns <- Column.makeColumnsFromNameTypeList(partColNameTypes, PartColStartIndex) - _ <- validateMapColumn(partColumns, Nil) } - yield { - PartitionSchema(partColumns, predefinedKeys, options) - } - } - - /** - * Parses a PartitionSchema from config. Format: - * {{{ - * columns = ["tags:map"] - * predefined-keys = ["_ns", "app", "__name__", "instance", "dc"] - * options { - * ... # See DatasetOptions parsing format - * } - * }}} - */ - def fromConfig(partConfig: Config): PartitionSchema Or BadSchema = - make(partConfig.as[Seq[String]]("columns"), - DatasetOptions.fromConfig(partConfig.getConfig("options")), - partConfig.as[Option[Seq[String]]]("predefined-keys").getOrElse(Nil)) -} - -/** - * A Schema combines a PartitionSchema with a DataSchema, forming all the columns of a single ingestion record. - */ -final case class Schema(partition: PartitionSchema, data: DataSchema) { - val allColumns = data.columns ++ partition.columns - val ingestionSchema = new RecordSchema(allColumns.map(c => ColumnInfo(c.name, c.columnType)), - Some(data.columns.length), - partition.predefinedKeys) - - val comparator = new RecordComparator(ingestionSchema) - val partKeySchema = comparator.partitionKeySchema -} - -final case class Schemas(part: PartitionSchema, - data: Map[String, DataSchema], - schemas: Map[String, Schema]) - -/** - * Singleton with code to load all schemas from config, verify no conflicts, and ensure there is only - * one PartitionSchema. Config schema: - * {{{ - * filodb { - * partition-schema { - * columns = ["tags:map"] - * } - * schemas { - * prometheus { ... } - * # etc - * } - * } - * }}} - */ -object Schemas { - import Dataset._ - import Accumulation._ - - // Validates all the data schemas from config, including checking hash conflicts, and returns all errors found - def validateDataSchemas(schemas: Map[String, Config]): Seq[DataSchema] Or Seq[(String, BadSchema)] = { - // get all data schemas parsed, combining errors - val parsed = schemas.toSeq.map { case (schemaName, schemaConf) => - DataSchema.fromConfig(schemaName, schemaConf) - .badMap(err => One((schemaName, err))) - }.combined.badMap(_.toSeq) - - // Check for no hash conflicts - parsed.filter { schemas => - val uniqueHashes = schemas.map(_.hash).toSet - if (uniqueHashes.size == schemas.length) Pass - else Fail(Seq(("", HashConflict(s"${schemas.length - uniqueHashes.size + 1} schemas have the same hash")))) - } - } - - /** - * Parse and initialize all the data schemas and single partition schema from config. - * Verifies that all of the schemas are conflict-free (no conflicting hash) and config parses correctly. - * @param config a Config object at the filodb config level, ie "partition-schema" is an entry - */ - def fromConfig(config: Config): Schemas Or Seq[(String, BadSchema)] = { - for { - partSchema <- PartitionSchema.fromConfig(config.getConfig("partition-schema")) - .badMap(e => Seq(("", e))) - dataSchemas <- validateDataSchemas(config.as[Map[String, Config]]("schemas")) - } yield { - val data = new collection.mutable.HashMap[String, DataSchema] - val schemas = new collection.mutable.HashMap[String, Schema] - dataSchemas.foreach { schema => - data(schema.name) = schema - schemas(schema.name) = Schema(partSchema, schema) - } - Schemas(partSchema, data.toMap, schemas.toMap) - } - } -} \ No newline at end of file diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 19b4d75678..663db59ccb 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -69,24 +69,24 @@ object TestData { object NamesTestData { def mapper(rows: Seq[Product]): Seq[RowReader] = rows.map(TupleRowReader) - val dataColSpecs = Seq("age:long:interval=10", "first:string", "last:string") - val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, DatasetOptions.DefaultOptions) + val dataColSpecs = Seq("first:string", "last:string", "age:long:interval=10") + val dataset = Dataset("dataset", Seq("seg:int"), dataColSpecs, "age", DatasetOptions.DefaultOptions) // NOTE: first 3 columns are the data columns, thus names could be used for either complete record // or the data column rowReader - val names = Seq((Some(24L), Some("Khalil"), Some("Mack"), Some(0)), - (Some(28L), Some("Ndamukong"), Some("Suh"), Some(0)), - (Some(25L), Some("Rodney"), Some("Hudson"), Some(0)), - (Some(40L), Some("Jerry"), None, Some(0)), - (Some(39L), Some("Peyton"), Some("Manning"), Some(0)), - (Some(29L), Some("Terrance"), Some("Knighton"), Some(0))) - - val altNames = Seq((Some(24L), Some("Stacy"), Some("McGee"), Some(0)), - (Some(28L), Some("Bruce"), Some("Irvin"), Some(0)), - (Some(25L), Some("Amari"), Some("Cooper"), Some(0)), - (Some(40L), Some("Jerry"), None, Some(0)), - (Some(39L), Some("Derek"), Some("Carr"), Some(0)), - (Some(29L), Some("Karl"), Some("Joseph"), Some(0))) + val names = Seq((Some("Khalil"), Some("Mack"), Some(24L), Some(0)), + (Some("Ndamukong"), Some("Suh"), Some(28L), Some(0)), + (Some("Rodney"), Some("Hudson"), Some(25L), Some(0)), + (Some("Jerry"), None, Some(40L), Some(0)), + (Some("Peyton"), Some("Manning"), Some(39L), Some(0)), + (Some("Terrance"), Some("Knighton"), Some(29L), Some(0))) + + val altNames = Seq((Some("Stacy"), Some("McGee"), Some(24L), Some(0)), + (Some("Bruce"), Some("Irvin"), Some(28L), Some(0)), + (Some("Amari"), Some("Cooper"), Some(25L), Some(0)), + (Some("Jerry"), None, Some(40L), Some(0)), + (Some("Derek"), Some("Carr"), Some(39L), Some(0)), + (Some("Karl"), Some("Joseph"), Some(29L), Some(0))) val firstKey = dataset.timestamp(mapper(names).head) val lastKey = dataset.timestamp(mapper(names).last) @@ -98,12 +98,12 @@ object NamesTestData { def chunkSetStream(data: Seq[Product] = names): Observable[ChunkSet] = TestData.toChunkSetStream(dataset, defaultPartKey, mapper(data)) - val firstNames = names.map(_._2.get) + val firstNames = names.map(_._1.get) val utf8FirstNames = firstNames.map(_.utf8) val sortedFirstNames = Seq("Khalil", "Rodney", "Ndamukong", "Terrance", "Peyton", "Jerry") val sortedUtf8Firsts = sortedFirstNames.map(_.utf8) - val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, DatasetOptions.DefaultOptions) + val largeDataset = Dataset("dataset", Seq("league:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) val lotLotNames = { for { league <- Seq("nfc", "afc") @@ -111,8 +111,8 @@ object NamesTestData { chunk <- 0 to numChunks startRowNo = numChunks * 10000 + chunk * 100 rowNo <- startRowNo to (startRowNo + 99) } - yield { (Some(rowNo.toLong), // the unique row key - names(rowNo % 6)._1, names(rowNo % 6)._2, + yield { (names(rowNo % 6)._1, names(rowNo % 6)._2, + Some(rowNo.toLong), // the unique row key Some(rowNo / 10000 * 10000), // the segment key Some(league)) } // partition key } @@ -181,21 +181,25 @@ object GdeltTestData { } val seqReaders = records.map { record => SeqRowReader(record.productIterator.toList) } - // NOTE: For all datasets the row key is GLOBALEVENTID - // Dataset1: Partition keys (Actor2Code, Year) - val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), DatasetOptions.DefaultOptions) + // Dataset1: Partition keys (Actor2Code, Year) / Row key GLOBALEVENTID + val dataset1 = Dataset("gdelt", Seq(schema(4), schema(3)), schema.patch(3, Nil, 2), "GLOBALEVENTID", DatasetOptions.DefaultOptions) - // Dataset2: Partition key (MonthYear) - val dataset2 = Dataset("gdelt", Seq(schema(2)), schema.patch(2, Nil, 1)) + // Dataset2: Partition key (MonthYear) / Row keys (GLOBALEVENTID, Actor2Code) + val dataset2 = Dataset("gdelt", Seq(schema(2)), schema.patch(2, Nil, 1), Seq("GLOBALEVENTID", "Actor2Code")) val partBuilder2 = new RecordBuilder(TestData.nativeMem, dataset2.partKeySchema, 10240) // Dataset3: same as Dataset1 for now val dataset3 = dataset1 + // Dataset4: One big partition (Year) with (Actor2Code, GLOBALEVENTID) rowkey + // to easily test row key scans + // val dataset4 = Dataset("gdelt", Seq(schema(3)), schema.patch(3, Nil, 1), Seq("Actor2Code", "GLOBALEVENTID")) + // val partBuilder4 = new RecordBuilder(TestData.nativeMem, dataset4.partKeySchema, 10240) + // Proj 6: partition Actor2Code,Actor2Name to test partition key bitmap indexing val datasetOptions = DatasetOptions.DefaultOptions.copy( shardKeyColumns = Seq( "__name__","_ns")) - val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), datasetOptions) + val dataset6 = Dataset("gdelt", schema.slice(4, 6), schema.patch(4, Nil, 2), "GLOBALEVENTID", datasetOptions) val datasetOptionConfig = """ options { @@ -394,6 +398,7 @@ object MachineMetricsData { // A simulation of custom machine metrics data - for testing extractTimeBucket object CustomMetricsData { + val columns = Seq("timestamp:ts", "min:double", "avg:double", "max:double", "count:long") //Partition Key with multiple string columns @@ -401,8 +406,9 @@ object CustomMetricsData { val metricdataset = Dataset.make("tsdbdata", partitionColumns, columns, + Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("metric", "_ns"), "metric")).get + DatasetOptions(Seq("metric", "_ns"), "metric", "count")).get val partKeyBuilder = new RecordBuilder(TestData.nativeMem, metricdataset.partKeySchema, 2048) val defaultPartKey = partKeyBuilder.addFromObjects("metric1", "app1") @@ -411,8 +417,9 @@ object CustomMetricsData { val metricdataset2 = Dataset.make("tsdbdata", partitionColumns2, columns, + Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__"), "__name__")).get + DatasetOptions(Seq("__name__"), "__name__", "count")).get val partKeyBuilder2 = new RecordBuilder(TestData.nativeMem, metricdataset2.partKeySchema, 2048) val defaultPartKey2 = partKeyBuilder2.addFromObjects(Map(ZeroCopyUTF8String("abc") -> ZeroCopyUTF8String("cba"))) @@ -422,8 +429,9 @@ object MetricsTestData { val timeseriesDataset = Dataset.make("timeseries", Seq("tags:map"), Seq("timestamp:ts", "value:double:detectDrops=true"), + Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__")).get + DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get val builder = new RecordBuilder(MemFactory.onHeapFactory, timeseriesDataset.ingestionSchema) diff --git a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala index dfb3e92ca7..021cadc36e 100644 --- a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala +++ b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala @@ -604,8 +604,9 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be val timeseriesDataset = Dataset.make("timeseries", Seq("tags:map"), Seq("timestamp:ts", "value:double"), + Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__", Map("dummy" -> Seq("_bucket")))).get + DatasetOptions(Seq("__name__", "job"), "__name__", "value", Map("dummy" -> Seq("_bucket")))).get val metricName4 = RecordBuilder.trimShardColumn(timeseriesDataset, "__name__", "heap_usage_bucket") metricName4 shouldEqual "heap_usage_bucket" diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index 83299ff140..da76c1e532 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -20,14 +20,16 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll val promDataset = Dataset.make("custom1", Seq("someStr:string", "tags:map"), Seq("timestamp:ts", "value:double"), + Seq("timestamp"), Seq("tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)"), - DatasetOptions(Seq("__name__", "job"), "__name__")).get + DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get val customDataset = Dataset.make("custom2", Seq("name:string", "namespace:string", "instance:string"), Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist:counter=false"), + Seq("timestamp"), Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)", "hSum(6)"), - DatasetOptions(Seq("name", "namespace"), "name")).get + DatasetOptions(Seq("name", "namespace"), "name", "total")).get private val blockStore = MMD.blockStore protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) @@ -199,13 +201,12 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll // avg val expectedAvgs2 = expectedSums2.zip(expectedCounts2).map { case (sum,count) => sum/count } downsampledData2.map(_._6) shouldEqual expectedAvgs2 - } - import com.softwaremill.quicklens._ + } val histDSDownsamplers = Seq("tTime(0)", "tTime(1)", "tTime(2)", "hSum(3)") - val histDSDataset = modify(MMD.histDataset)(_.schema.data.downsamplers) - .setTo(Dataset.validateDownsamplers(histDSDownsamplers).get) + val histDSDataset = MMD.histDataset.copy( + downsamplers = Dataset.validateDownsamplers(histDSDownsamplers).get) // Create downsampleOps for histogram dataset. Samples every 10s, downsample freq 60s/1min val downsampleOpsH = new ShardDownsampler(histDSDataset, 0, true, Seq(60000), NoOpDownsamplePublisher, diff --git a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala index b3f6dfe615..9225c7a477 100644 --- a/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala +++ b/core/src/test/scala/filodb.core/metadata/DatasetSpec.scala @@ -1,20 +1,183 @@ package filodb.core.metadata +import com.typesafe.config.ConfigFactory import org.scalatest.{FunSpec, Matchers} import filodb.core._ import filodb.core.query.ColumnInfo -// DEPRECATED: remove soon class DatasetSpec extends FunSpec with Matchers { import Column.ColumnType._ import Dataset._ import NamesTestData._ + describe("Dataset creation") { + it("should load/write dataset from/to config") { + val config = ConfigFactory.parseString( + """ + | dataset = "prometheus" + | + | definition { + | partition-columns = ["tags:map"] + | data-columns = ["timestamp:ts", "value:double"] + | row-key-columns = [ "timestamp" ] + | downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)"] + | } + | + | options { + | shardKeyColumns = [ "__name__", "_ns" ] + | ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + | valueColumn = "value" + | metricColumn = "__name__" + | ignoreTagsOnPartitionKeyHash = [ "le" ] + | copyTags = { } + | } + """.stripMargin) + val dataset = Dataset.fromConfig(config) + + val config2 = dataset.toConfig + val dataset2 = Dataset.fromConfig(config2) + + dataset shouldEqual dataset2 + } + } + describe("Dataset validation") { + it("should return NotNameColonType if column specifiers not name:type format") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "column2", Seq("age")) + resp1.isBad shouldEqual true + resp1.swap.get shouldEqual ColumnErrors(Seq(NotNameColonType("column2"))) + + intercept[BadSchemaError] { + Dataset("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", "age", DatasetOptions.DefaultOptions) + } + } + + it("should return BadColumnParams if name:type:params portion not valid key=value pairs") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "column2:a:b", Seq("age")) + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + } + + it("should return BadColumnParams if required param config not specified") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:foo=bar", Seq("age")) + resp1.isBad shouldEqual true + resp1.swap.get shouldBe a[ColumnErrors] + val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs + errors should have length 1 + errors.head shouldBe a[BadColumnParams] + + val resp2 = Dataset.make("dataset", Seq("part:string"), dataColSpecs :+ "h:hist:counter=bar", Seq("age")) + resp2.isBad shouldEqual true + resp2.swap.get shouldBe a[ColumnErrors] + val errors2 = resp2.swap.get.asInstanceOf[ColumnErrors].errs + errors2 should have length 1 + errors2.head shouldBe a[BadColumnParams] + } + + it("should return BadColumnName if illegal chars in column name") { + val resp1 = Dataset.make("dataset", Seq("part:string"), Seq("col, umn1:string"), Seq("age")) + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (1) + errors.head shouldBe a[BadColumnName] + } + + it("should return BadColumnType if unsupported type specified in column spec") { + val resp1 = Dataset.make("dataset", Seq("part:linkedlist"), dataColSpecs, Seq("age")) + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (1) + errors.head shouldEqual BadColumnType("linkedlist") + } + + it("should return multiple column spec errors") { + val resp1 = Dataset.make("dataset", Seq("part:string"), + Seq("first:str", "age:long", "la(st):int"), Seq("age")) + resp1.isBad shouldEqual true + val errors = resp1.swap.get match { + case ColumnErrors(errs) => errs + case x => throw new RuntimeException(s"Did not expect $x") + } + errors should have length (2) + errors.head shouldEqual BadColumnType("str") + } + + it("should return UnknownRowKeyColumn if row key column(s) not in data columns") { + val resp1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("column2")) + resp1.isBad shouldEqual true + resp1.swap.get shouldEqual UnknownRowKeyColumn("column2") + + val resp2 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("age", "column9")) + resp2.isBad shouldEqual true + resp2.swap.get shouldEqual UnknownRowKeyColumn("column9") + } + + it("should allow MapColumns only in last position of partition key") { + val mapCol = "tags:map" + + // OK: only partition column is map + val ds1 = Dataset("dataset", Seq(mapCol), dataColSpecs, "age", DatasetOptions.DefaultOptions) + ds1.partitionColumns.map(_.name) should equal (Seq("tags")) + + // OK: last partition column is map + val ds2 = Dataset("dataset", Seq("first:string", mapCol), dataColSpecs drop 1, "age", DatasetOptions.DefaultOptions) + ds2.partitionColumns.map(_.name) should equal (Seq("first", "tags")) + + // Not OK: first partition column is map + val resp3 = Dataset.make("dataset", Seq(mapCol, "first:string"), dataColSpecs drop 1, Seq("age")) + resp3.isBad shouldEqual true + resp3.swap.get shouldBe an[IllegalMapColumn] + + // Not OK: map in data columns, not partition column + intercept[BadSchemaError] { + Dataset("dataset", Seq("seg:int"), dataColSpecs :+ mapCol, Seq("age")) } + } + + it("should return NoTimestampRowKey if non timestamp used for row key") { + val ds1 = Dataset.make("dataset", Seq("part:string"), dataColSpecs, Seq("first")) + ds1.isBad shouldEqual true + ds1.swap.get shouldBe a[NoTimestampRowKey] + } + + it("should return a valid Dataset when a good specification passed") { + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) + ds.rowKeyIDs shouldEqual Seq(2) + ds.dataColumns should have length (3) + ds.dataColumns.map(_.id) shouldEqual Seq(0, 1, 2) + ds.dataColumns.map(_.columnType) shouldEqual Seq(StringColumn, StringColumn, LongColumn) + ds.partitionColumns should have length (1) + ds.partitionColumns.map(_.columnType) shouldEqual Seq(StringColumn) + ds.partitionColumns.map(_.id) shouldEqual Seq(PartColStartIndex) + Dataset.isPartitionID(ds.partitionColumns.head.id) shouldEqual true + ds.timestampColumn.name shouldEqual "age" + ds.rowKeyRouting shouldEqual Array(2) + } + + it("should return valid Dataset when multiple row key columns specified") { + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, Seq("age", "first")) + ds.rowKeyIDs shouldEqual Seq(2, 0) + ds.dataColumns should have length (3) + ds.dataColumns.map(_.id) shouldEqual Seq(0, 1, 2) + ds.dataColumns.map(_.columnType) shouldEqual Seq(StringColumn, StringColumn, LongColumn) + ds.partitionColumns should have length (1) + ds.partitionColumns.map(_.columnType) shouldEqual Seq(StringColumn) + ds.timestampColumn.name shouldEqual "age" + ds.rowKeyRouting shouldEqual Array(2, 0) + } + it("should return IDs for column names or seq of missing names") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, DatasetOptions.DefaultOptions) - ds.colIDs("first", "age").get shouldEqual Seq(1, 0) + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) + ds.colIDs("first", "age").get shouldEqual Seq(0, 2) ds.colIDs("part").get shouldEqual Seq(Dataset.PartColStartIndex) @@ -24,11 +187,11 @@ class DatasetSpec extends FunSpec with Matchers { } it("should return ColumnInfos for colIDs") { - val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, DatasetOptions.DefaultOptions) - val infos = ds.infosFromIDs(Seq(1, 0)) + val ds = Dataset("dataset", Seq("part:string"), dataColSpecs, "age", DatasetOptions.DefaultOptions) + val infos = ds.infosFromIDs(Seq(0, 2)) infos shouldEqual Seq(ColumnInfo("first", StringColumn), ColumnInfo("age", LongColumn)) - val infos2 = ds.infosFromIDs(Seq(PartColStartIndex, 2)) + val infos2 = ds.infosFromIDs(Seq(PartColStartIndex, 1)) infos2 shouldEqual Seq(ColumnInfo("part", StringColumn), ColumnInfo("last", StringColumn)) } diff --git a/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala b/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala deleted file mode 100644 index 8c70410ad3..0000000000 --- a/core/src/test/scala/filodb.core/metadata/SchemasSpec.scala +++ /dev/null @@ -1,249 +0,0 @@ -package filodb.core.metadata - -import com.typesafe.config.ConfigFactory -import org.scalatest.{FunSpec, Matchers} - -import filodb.core._ - -class SchemasSpec extends FunSpec with Matchers { - import Column.ColumnType._ - import Dataset._ - import NamesTestData._ - - describe("DataSchema") { - it("should return NotNameColonType if column specifiers not name:type format") { - val resp1 = DataSchema.make("dataset", dataColSpecs :+ "column2", Nil, "first") - resp1.isBad shouldEqual true - resp1.swap.get shouldEqual ColumnErrors(Seq(NotNameColonType("column2"))) - } - - it("should return BadColumnParams if name:type:params portion not valid key=value pairs") { - val resp1 = DataSchema.make("dataset", dataColSpecs :+ "column2:a:b", Nil, "first") - resp1.isBad shouldEqual true - resp1.swap.get shouldBe a[ColumnErrors] - val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs - errors should have length 1 - errors.head shouldBe a[BadColumnParams] - } - - it("should return BadColumnParams if required param config not specified") { - val resp1 = DataSchema.make("dataset", dataColSpecs :+ "h:hist:foo=bar", Nil, "first") - resp1.isBad shouldEqual true - resp1.swap.get shouldBe a[ColumnErrors] - val errors = resp1.swap.get.asInstanceOf[ColumnErrors].errs - errors should have length 1 - errors.head shouldBe a[BadColumnParams] - - val resp2 = DataSchema.make("dataset", dataColSpecs :+ "h:hist:counter=bar", Nil, "first") - resp2.isBad shouldEqual true - resp2.swap.get shouldBe a[ColumnErrors] - val errors2 = resp2.swap.get.asInstanceOf[ColumnErrors].errs - errors2 should have length 1 - errors2.head shouldBe a[BadColumnParams] - } - - it("should return BadColumnName if illegal chars in column name") { - val resp1 = DataSchema.make("dataset", Seq("col, umn1:string"), Nil, "first") - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (1) - errors.head shouldBe a[BadColumnName] - } - - it("should return BadColumnType if unsupported type specified in column spec") { - val resp1 = DataSchema.make("dataset", dataColSpecs :+ "part:linkedlist", Nil, "first") - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (1) - errors.head shouldEqual BadColumnType("linkedlist") - } - - it("should return BadColumnName if value column not one of other columns") { - val conf2 = ConfigFactory.parseString(""" - { - columns = ["first:string", "last:string", "age:long"] - value-column = "first2" - downsamplers = [] - }""") - val resp = DataSchema.fromConfig("dataset", conf2) - resp.isBad shouldEqual true - resp.swap.get shouldBe a[BadColumnName] - } - - it("should return multiple column spec errors") { - val resp1 = DataSchema.make("dataset", Seq("first:str", "age:long", "la(st):int"), Nil, "first") - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (2) - errors.head shouldEqual BadColumnType("str") - } - - it("should return NoTimestampRowKey if non timestamp used for row key / first column") { - val ds1 = DataSchema.make("dataset", Seq("first:string", "age:long"), Nil, "first") - ds1.isBad shouldEqual true - ds1.swap.get shouldBe a[NoTimestampRowKey] - } - - it("should return a valid Dataset when a good specification passed") { - val conf2 = ConfigFactory.parseString(""" - { - columns = ["timestamp:ts", "code:long", "event:string"] - value-column = "event" - downsamplers = [] - }""") - val schema = DataSchema.fromConfig("dataset", conf2).get - schema.columns should have length (3) - schema.columns.map(_.id) shouldEqual Seq(0, 1, 2) - schema.columns.map(_.columnType) shouldEqual Seq(TimestampColumn, LongColumn, StringColumn) - schema.timestampColumn.name shouldEqual "timestamp" - } - } - - val partSchemaStr = """{ - columns = ["tags:map"] - predefined-keys = ["_ns", "app", "__name__", "instance", "dc"] - options { - copyTags = {} - ignoreShardKeyColumnSuffixes = {} - ignoreTagsOnPartitionKeyHash = ["le"] - metricColumn = "__name__" - shardKeyColumns = ["__name__", "_ns"] - } - }""" - - describe("PartitionSchema") { - it("should allow MapColumns only in last position of partition key") { - val mapCol = "tags:map" - - // OK: only partition column is map - val ds1 = PartitionSchema.make(Seq(mapCol), DatasetOptions.DefaultOptions).get - ds1.columns.map(_.name) should equal (Seq("tags")) - - // OK: last partition column is map - val ds2 = PartitionSchema.make(Seq("first:string", mapCol), DatasetOptions.DefaultOptions).get - ds2.columns.map(_.name) should equal (Seq("first", "tags")) - - // Not OK: first partition column is map - val resp3 = PartitionSchema.make(Seq(mapCol, "first:string"), DatasetOptions.DefaultOptions) - resp3.isBad shouldEqual true - resp3.swap.get shouldBe an[IllegalMapColumn] - } - - it("should return BadColumnType if unsupported type specified in column spec") { - val resp1 = PartitionSchema.make(Seq("first:strolo"), DatasetOptions.DefaultOptions) - resp1.isBad shouldEqual true - val errors = resp1.swap.get match { - case ColumnErrors(errs) => errs - case x => throw new RuntimeException(s"Did not expect $x") - } - errors should have length (1) - errors.head shouldEqual BadColumnType("strolo") - } - - it("should parse config with options") { - val conf2 = ConfigFactory.parseString(partSchemaStr) - val schema = PartitionSchema.fromConfig(conf2).get - - schema.columns.map(_.columnType) shouldEqual Seq(MapColumn) - schema.predefinedKeys shouldEqual Seq("_ns", "app", "__name__", "instance", "dc") - } - } - - describe("Schemas") { - it("should return all errors from every data schema") { - val conf2 = ConfigFactory.parseString(s""" - { - partition-schema $partSchemaStr - schemas { - prom1 { - columns = ["timestamp:tsa", "code:long", "event:string"] - value-column = "event" - downsamplers = [] - } - prom2 { - columns = ["timestamp:ts", "code:long", "ev. ent:string"] - value-column = "foo" - downsamplers = [] - } - prom3 { - columns = ["timestamp:ts", "code:long", "event:string"] - value-column = "event" - downsamplers = [] - } - } - }""") - val resp = Schemas.fromConfig(conf2) - resp.isBad shouldEqual true - val errors = resp.swap.get - errors should have length (2) - errors.map(_._1).toSet shouldEqual Set("prom1", "prom2") - errors.map(_._2.getClass).toSet shouldEqual Set(classOf[ColumnErrors]) - } - - it("should detect and report hash conflicts") { - val conf2 = ConfigFactory.parseString(s""" - { - partition-schema $partSchemaStr - schemas { - prom { - columns = ["timestamp:ts", "value:double"] - value-column = "value" - downsamplers = [] - } - prom2 { - columns = ["timestamp:ts", "value:double"] - value-column = "timestamp" - downsamplers = [] - } - } - }""") - val resp = Schemas.fromConfig(conf2) - resp.isBad shouldEqual true - val errors = resp.swap.get - errors.map(_._2.getClass) shouldEqual Seq(classOf[HashConflict]) - } - - it("should return Schemas instance with every schema parsed") { - val conf2 = ConfigFactory.parseString(s""" - { - partition-schema $partSchemaStr - schemas { - prom { - columns = ["timestamp:ts", "value:double"] - value-column = "value" - downsamplers = [] - } - hist { - columns = ["timestamp:ts", "count:long", "sum:long", "h:hist:counter=true"] - value-column = "h" - downsamplers = [] - } - } - }""") - val schemas = Schemas.fromConfig(conf2).get - - schemas.part.columns.map(_.columnType) shouldEqual Seq(MapColumn) - schemas.part.columns.map(_.id) shouldEqual Seq(PartColStartIndex) - schemas.part.predefinedKeys shouldEqual Seq("_ns", "app", "__name__", "instance", "dc") - Dataset.isPartitionID(schemas.part.columns.head.id) shouldEqual true - - schemas.data.keySet shouldEqual Set("prom", "hist") - schemas.schemas.keySet shouldEqual Set("prom", "hist") - schemas.data("prom").columns.map(_.columnType) shouldEqual Seq(TimestampColumn, DoubleColumn) - schemas.data("prom").columns.map(_.id) shouldEqual Seq(0, 1) - schemas.data("prom").timestampColumn.name shouldEqual "timestamp" - schemas.data("hist").columns.map(_.columnType) shouldEqual - Seq(TimestampColumn, LongColumn, LongColumn, HistogramColumn) - // println(schemas.data.values.map(_.hash)) - } - } -} diff --git a/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala b/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala index 7437c24618..2bff5d1782 100644 --- a/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala +++ b/core/src/test/scala/filodb.core/query/KeyFilterSpec.scala @@ -12,15 +12,15 @@ class KeyFilterSpec extends FunSpec with Matchers { import Filter._ it("should parse values for regular KeyTypes") { - KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc") should equal ("abc".utf8) - KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc".utf8) should equal ("abc".utf8) + KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc") should equal ("abc".utf8) + KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc".utf8) should equal ("abc".utf8) KeyFilter.parseSingleValue(dataset.partitionColumns.head, -15) should equal (-15) - KeyFilter.parseValues(dataset.dataColumns(1), Set("abc", "def")) should equal (Set("abc".utf8, "def".utf8)) + KeyFilter.parseValues(dataset.dataColumns.head, Set("abc", "def")) should equal (Set("abc".utf8, "def".utf8)) } it("should validate equalsFunc for string and other types") { - val eqFunc1 = Equals(KeyFilter.parseSingleValue(dataset.dataColumns(1), "abc")).filterFunc + val eqFunc1 = Equals(KeyFilter.parseSingleValue(dataset.dataColumns.head, "abc")).filterFunc eqFunc1("abc".utf8) should equal (true) eqFunc1("abc") should equal (false) eqFunc1(15) should equal (false) @@ -30,7 +30,7 @@ class KeyFilterSpec extends FunSpec with Matchers { } it("should validate inFunc for string and other types") { - val inFunc1 = In(KeyFilter.parseValues(dataset.dataColumns(1), Set("abc", "def")).toSet).filterFunc + val inFunc1 = In(KeyFilter.parseValues(dataset.dataColumns.head, Set("abc", "def")).toSet).filterFunc inFunc1("abc".utf8) should equal (true) inFunc1("aaa".utf8) should equal (false) inFunc1(15) should equal (false) diff --git a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala index 71d2391828..ea6c060c70 100644 --- a/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/ColumnStoreSpec.scala @@ -28,17 +28,21 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { val policy = new FixedMaxPartitionsEvictionPolicy(100) // Since 99 GDELT rows, this will never evict val memStore = new TimeSeriesMemStore(config, colStore, metaStore, Some(policy)) + val datasetDb2 = dataset.copy(database = Some("unittest2")) + // First create the tables in C* override def beforeAll(): Unit = { super.beforeAll() metaStore.initialize().futureValue colStore.initialize(dataset.ref).futureValue colStore.initialize(GdeltTestData.dataset2.ref).futureValue + colStore.initialize(datasetDb2.ref).futureValue } before { colStore.truncate(dataset.ref).futureValue colStore.truncate(GdeltTestData.dataset2.ref).futureValue + colStore.truncate(datasetDb2.ref).futureValue memStore.reset() metaStore.clearAllData() } @@ -103,6 +107,21 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { rowIt2.map(_.getLong(0)).toSeq should equal (Seq(24L, 28L, 25L, 40L, 39L, 29L)) } + it should "read back rows written in another database" ignore { + whenReady(colStore.write(datasetDb2, chunkSetStream())) { response => + response should equal (Success) + } + + val paramSet = colStore.getScanSplits(dataset.ref, 1) + paramSet should have length (1) + + val rowIt = memStore.scanRows(datasetDb2, Seq(0, 1, 2), FilteredPartitionScan(paramSet.head)) + rowIt.map(_.getLong(2)).toSeq should equal (Seq(24L, 28L, 25L, 40L, 39L, 29L)) + + // Check that original keyspace/database has no data + memStore.scanRows(dataset, Seq(0), partScan).toSeq should have length (0) + } + it should "read back rows written with multi-column row keys" ignore { import GdeltTestData._ val stream = toChunkSetStream(dataset2, partBuilder2.addFromObjects(197901), dataRows(dataset2)) diff --git a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala index 9f817db1fc..74f3b83ad5 100644 --- a/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala +++ b/core/src/test/scala/filodb.core/store/MetaStoreSpec.scala @@ -19,35 +19,35 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { metaStore.initialize().futureValue } - val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), + val dataset = Dataset("foo", Seq("part:string"), Seq("timestamp:long", "value:double"), "timestamp", DatasetOptions.DefaultOptions) before { metaStore.clearAllData().futureValue } describe("checkpoint api") { it("should allow reading and writing checkpoints for shard") { - val ref = DatasetRef("gdelt-" + UUID.randomUUID()) // Add uuid so tests can be rerun + val ds = dataset.copy(name = "gdelt-" + UUID.randomUUID()) // Add uuid so tests can be rerun // when there is no data for a shard, then return Long.MinValue as the checkpoint - metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual Long.MinValue - metaStore.readCheckpoints(ref, 2).futureValue.isEmpty shouldBe true + metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual Long.MinValue + metaStore.readCheckpoints(ds.ref, 2).futureValue.isEmpty shouldBe true // should be able to insert checkpoints for new groups - metaStore.writeCheckpoint(ref, 2, 1, 10).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 2, 9).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 3, 13).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 4, 5).futureValue shouldEqual Success - metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual 5 + metaStore.writeCheckpoint(ds.ref, 2, 1, 10).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 2, 9).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 3, 13).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 4, 5).futureValue shouldEqual Success + metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual 5 // should be able to update checkpoints for existing groups - metaStore.writeCheckpoint(ref, 2, 1, 12).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 2, 15).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 3, 17).futureValue shouldEqual Success - metaStore.writeCheckpoint(ref, 2, 4, 7).futureValue shouldEqual Success - metaStore.readEarliestCheckpoint(ref, 2).futureValue shouldEqual 7 + metaStore.writeCheckpoint(ds.ref, 2, 1, 12).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 2, 15).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 3, 17).futureValue shouldEqual Success + metaStore.writeCheckpoint(ds.ref, 2, 4, 7).futureValue shouldEqual Success + metaStore.readEarliestCheckpoint(ds.ref, 2).futureValue shouldEqual 7 // should be able to retrieve raw offsets as well - val offsets = metaStore.readCheckpoints(ref, 2).futureValue + val offsets = metaStore.readCheckpoints(ds.ref, 2).futureValue offsets.size shouldEqual 4 offsets(1) shouldEqual 12 offsets(2) shouldEqual 15 @@ -56,7 +56,7 @@ with BeforeAndAfter with BeforeAndAfterAll with ScalaFutures { // should be able to clear the table metaStore.clearAllData().futureValue - metaStore.readCheckpoints(ref, 2).futureValue.isEmpty shouldBe true + metaStore.readCheckpoints(ds.ref, 2).futureValue.isEmpty shouldBe true } } diff --git a/doc/ingestion.md b/doc/ingestion.md index 8b30acafcb..a916f714f6 100644 --- a/doc/ingestion.md +++ b/doc/ingestion.md @@ -33,7 +33,43 @@ NOTE: for the latest and most up to date, see [timeseries-dev-source.conf](../co ```yaml dataset = "example" -schema = "prometheus" + +# Schema used for defining the BinaryRecord used in ingestion and persistence. +# +# Should not change once dataset has been set up on the server and data has been ingested into kafka +# or written to cassandra +definition { + # defines the unique identifier for partition + partition-columns = ["tags:map"] + # Schema of all of the values stored against the partition key. This includes the row-keys as well + data-columns = ["timestamp:ts", "value:double:detectDrops=true"] + # Clustering key for each row. Together with partition key, they form the primary key. + row-key-columns = [ "timestamp" ] + # List of downsamplers for the data columns + downsamplers = [ "tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)"] +} + +# Dataset Options +# +# Should not change once dataset has been set up on the server and data has been ingested into kafka +# or written to cassandra +options { + # These column values are used to identify the shard group for the partition + shardKeyColumns = [ "__name__", "_ns" ] + # suffixes from map values are stripped before hashing + ignoreShardKeyColumnSuffixes = { "__name__" = ["_bucket", "_count", "_sum"] } + # default data column name to be used referencing the value column + valueColumn = "value" + # column name to use when referencing the name column + metricColumn = "__name__" + # these columns are ignored in calculation of full partition key hash + ignoreTagsOnPartitionKeyHash = [ "le" ] + + # These key-names will be replaced in the key map during ingestion/query. + # Temporary workaround. Will be deprecated. + copyTags = { } +} + # e.g. one shard per kafka partition num-shards = 100 diff --git a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala index 3a1f20821a..24300e93e7 100644 --- a/gateway/src/main/scala/filodb/gateway/GatewayServer.scala +++ b/gateway/src/main/scala/filodb/gateway/GatewayServer.scala @@ -26,6 +26,7 @@ import org.jctools.queues.MpscGrowableArrayQueue import org.rogach.scallop._ import filodb.coordinator.{FilodbSettings, ShardMapper, StoreFactory} +import filodb.core.GlobalConfig import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.gateway.conversion._ @@ -57,9 +58,8 @@ import filodb.timeseries.TestTimeseriesProducer */ object GatewayServer extends StrictLogging { // Get global configuration using universal FiloDB/Akka-based config - val settings = new FilodbSettings() - val config = settings.allConfig - val storeFactory = StoreFactory(settings, Scheduler.io()) + val config = GlobalConfig.systemConfig + val storeFactory = StoreFactory(new FilodbSettings(config), Scheduler.io()) // ==== Metrics ==== val numInfluxMessages = Kamon.counter("num-influx-messages") @@ -89,7 +89,7 @@ object GatewayServer extends StrictLogging { val sourceConfig = ConfigFactory.parseFile(new java.io.File(userOpts.sourceConfigPath())) val numShards = sourceConfig.getInt("num-shards") - val dataset = settings.datasetFromStream(sourceConfig) + val dataset = Dataset.fromConfig(sourceConfig) // NOTE: the spread MUST match the default spread used in the HTTP module for consistency between querying // and ingestion sharding diff --git a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala index 01f104b100..0ec5d38a42 100644 --- a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala @@ -19,7 +19,13 @@ object PrometheusApiRouteSpec extends ActorSpecConfig { inline-dataset-configs = [ { dataset = "prometheus" - schema = "prometheus" + definition { + partition-columns = ["tags:map"] + data-columns = ["timestamp:ts", "value:double"] + row-key-columns = [ "timestamp" ] + downsamplers = [] + } + ${TestData.optionsString} num-shards = 4 min-num-nodes = 1 sourcefactory = "${classOf[NoOpStreamFactory].getName}" diff --git a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala index 944cc7f3de..377ea0699d 100644 --- a/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/IntSumReadBenchmark.scala @@ -14,7 +14,8 @@ import filodb.core.store.ChunkSet import filodb.memory.format.{vectors => bv, TupleRowReader, UnsafeUtils} object IntSumReadBenchmark { - val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), DatasetOptions.DefaultOptions) + val dataset = Dataset("dataset", Seq("part:int"), Seq("int:int", "rownum:long"), "rownum", + DatasetOptions.DefaultOptions) val rowIt = Iterator.from(0).map { row => (Some(scala.util.Random.nextInt), Some(row.toLong), Some(0)) } val partKey = NamesTestData.defaultPartKey val rowColumns = Seq("int", "rownum", "part") diff --git a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala index 2421e9cb61..3c84e524ad 100644 --- a/kafka/src/main/scala/filodb/kafka/TestConsumer.scala +++ b/kafka/src/main/scala/filodb/kafka/TestConsumer.scala @@ -8,6 +8,7 @@ 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 /** @@ -38,7 +39,7 @@ object TestConsumer extends App { import monix.execution.Scheduler.Implicits.global val ingestConf = IngestionConfig(sourceConf, classOf[KafkaIngestionStreamFactory].getClass.getName).get - val dataset = settings.datasetFromStream(sourceConf) + val dataset = Dataset.fromConfig(sourceConf) val ctor = Class.forName(ingestConf.streamFactoryClass).getConstructors.head val streamFactory = ctor.newInstance().asInstanceOf[IngestionStreamFactory] diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 519086f144..2cdd60e8bb 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -194,7 +194,6 @@ object FiloBuild extends Build { "io.kamon" %% "kamon-akka-remote-2.5" % "1.1.0", logbackDep % Test, scalaTest % Test, - "com.softwaremill.quicklens" %% "quicklens" % "1.4.12" % Test, scalaCheck % "test" ) @@ -275,7 +274,6 @@ object FiloBuild extends Build { lazy val gatewayDeps = commonDeps ++ Seq( scalaxyDep, - logbackDep, "io.monix" %% "monix-kafka-1x" % monixKafkaVersion, "org.rogach" %% "scallop" % "3.1.1" ) diff --git a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala index 233252f2c7..350cb50ce0 100644 --- a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala +++ b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala @@ -9,10 +9,10 @@ import filodb.core.metadata.{Dataset, DatasetOptions} */ object FormatConversion { // An official Prometheus-format Dataset object with a single timestamp and value - val options = DatasetOptions(Seq("__name__", "_ns"), - "__name__", Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), - Map("exporter" -> "_ns", "job" -> "_ns")) - val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double"), options) + val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double")) + .copy(options = DatasetOptions(Seq("__name__", "_ns"), + "__name__", "value", Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), + Map("exporter" -> "_ns", "job" -> "_ns"))) /** * Extracts a java ArrayList of labels from the TimeSeries diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index e2b8dd5d73..b4432e3832 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -41,13 +41,15 @@ final case class SelectRawPartitionsExec(id: String, require(colIds.nonEmpty) protected[filodb] def schemaOfDoExecute(dataset: Dataset): ResultSchema = { + val numRowKeyCols = colIds.zip(dataset.rowKeyIDs).takeWhile { case (a, b) => a == b }.length + // Add the max column to the schema together with Histograms for max computation -- just in case it's needed // But make sure the max column isn't already included histMaxColumn(dataset, colIds).filter { mId => !(colIds contains mId) } .map { maxColId => - ResultSchema(dataset.infosFromIDs(colIds :+ maxColId), 1, colIDs = (colIds :+ maxColId)) + ResultSchema(dataset.infosFromIDs(colIds :+ maxColId), numRowKeyCols, colIDs = (colIds :+ maxColId)) }.getOrElse { - ResultSchema(dataset.infosFromIDs(colIds), 1, colIDs = colIds) + ResultSchema(dataset.infosFromIDs(colIds), numRowKeyCols, colIDs = colIds) } } @@ -56,6 +58,8 @@ final case class SelectRawPartitionsExec(id: String, queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { + require(colIds.indexOfSlice(dataset.rowKeyIDs) == 0) + val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) source.rangeVectors(dataset, colIds, partMethod, chunkMethod) } From 4c5b51c871eb53b9e18faa257d6dcb81dc4e8737 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Tue, 30 Jul 2019 16:00:05 -0700 Subject: [PATCH 05/28] feat(query, core): QueryResult need not always be serializable (#451) Currently QueryResult uses SerializableRangeVectors only. With the new InProcessDispatcher which is coming soon, QueryResult can be a result of an ExecPlan that is executed within the current JVM. So we should not be forced into converting results into a SerializableRangeVector. Hence QueryResult will use RangeVector and not force use of SerializableRangeVector. --- .../client/SerializationSpec.scala | 9 +++++--- .../scala/filodb.core/query/RangeVector.scala | 11 +++++++--- .../prometheus/query/PrometheusModel.scala | 6 +++--- .../main/scala/filodb/query/ResultTypes.scala | 8 +++---- .../scala/filodb/query/exec/ExecPlan.scala | 6 +++--- .../filodb/query/exec/MetadataExecPlan.scala | 21 +++++++++++-------- .../filodb/query/exec/SetOperatorExec.scala | 20 +++++++++--------- .../filodb/query/exec/MetadataExecSpec.scala | 12 ++++++----- 8 files changed, 53 insertions(+), 40 deletions(-) diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index 4281d92395..cf71155ae7 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -7,7 +7,7 @@ import org.scalatest.concurrent.ScalaFutures import filodb.coordinator.{ActorSpecConfig, ActorTest, ShardMapper} import filodb.coordinator.queryengine2.QueryEngine -import filodb.core.{MachineMetricsData, MetricsTestData, NamesTestData} +import filodb.core.{query, MachineMetricsData, MetricsTestData, NamesTestData} import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.metadata.Column.ColumnType import filodb.core.store.IngestionConfig @@ -155,7 +155,9 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) result.id shouldEqual roundTripResult.id for { i <- 0 until roundTripResult.result.size } { // BinaryVector deserializes to different impl, so cannot compare top level object - roundTripResult.result(i).schema shouldEqual result.result(i).schema + roundTripResult.result(i) + .asInstanceOf[query.SerializableRangeVector].schema shouldEqual result.result(i) + .asInstanceOf[query.SerializableRangeVector].schema roundTripResult.result(i).rows.map(_.getDouble(1)).toSeq shouldEqual result.result(i).rows.map(_.getDouble(1)).toSeq roundTripResult.result(i).key.labelValues shouldEqual result.result(i).key.labelValues @@ -299,7 +301,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) srv.rows.size shouldEqual 1 val actual = srv.rows.map(record => { val rowReader = record.asInstanceOf[BinaryRecordRowReader] - srv.schema.toStringPairs(rowReader.recordBase, rowReader.recordOffset).toMap + srv.asInstanceOf[query.SerializableRangeVector] + .schema.toStringPairs(rowReader.recordBase, rowReader.recordOffset).toMap }) actual.toList shouldEqual expected } diff --git a/core/src/main/scala/filodb.core/query/RangeVector.scala b/core/src/main/scala/filodb.core/query/RangeVector.scala index d5255bdf6e..eef70973e5 100644 --- a/core/src/main/scala/filodb.core/query/RangeVector.scala +++ b/core/src/main/scala/filodb.core/query/RangeVector.scala @@ -90,6 +90,8 @@ object CustomRangeVectorKey { trait RangeVector { def key: RangeVectorKey def rows: Iterator[RowReader] + def numRows: Option[Int] = None + def prettyPrint(formatTime: Boolean = true): String = "RV String Not supported" } // First column of columnIDs should be the timestamp column @@ -133,18 +135,21 @@ final case class ChunkInfoRangeVector(key: RangeVectorKey, * only serialized once as a single instance. */ final class SerializableRangeVector(val key: RangeVectorKey, - val numRows: Int, + val numRowsInt: Int, containers: Seq[RecordContainer], val schema: RecordSchema, startRecordNo: Int) extends RangeVector with java.io.Serializable { + + override val numRows = Some(numRowsInt) + // Possible for records to spill across containers, so we read from all containers override def rows: Iterator[RowReader] = - containers.toIterator.flatMap(_.iterate(schema)).drop(startRecordNo).take(numRows) + containers.toIterator.flatMap(_.iterate(schema)).drop(startRecordNo).take(numRowsInt) /** * Pretty prints all the elements into strings using record schema */ - def prettyPrint(formatTime: Boolean = true): String = { + override def prettyPrint(formatTime: Boolean = true): String = { val curTime = System.currentTimeMillis key.toString + "\n\t" + rows.map { diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index 123e5777e7..f70b767f65 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -2,7 +2,7 @@ package filodb.prometheus.query import remote.RemoteStorage._ -import filodb.core.query.{ColumnFilter, Filter, SerializableRangeVector} +import filodb.core.query.{ColumnFilter, Filter, RangeVector} import filodb.query.{IntervalSelector, LogicalPlan, QueryResultType, RawSeries} import filodb.query.exec.ExecPlan @@ -67,7 +67,7 @@ object PrometheusModel { /** * Used to send out raw data */ - def toPromTimeSeries(srv: SerializableRangeVector): TimeSeries = { + def toPromTimeSeries(srv: RangeVector): TimeSeries = { val b = TimeSeries.newBuilder() srv.key.labelValues.foreach {lv => b.addLabels(LabelPair.newBuilder().setName(lv._1.toString).setValue(lv._2.toString)) @@ -99,7 +99,7 @@ object PrometheusModel { /** * Used to send out HTTP response */ - def toPromResult(srv: SerializableRangeVector, verbose: Boolean): Result = { + def toPromResult(srv: RangeVector, verbose: Boolean): Result = { val tags = srv.key.labelValues.map { case (k, v) => (k.toString, v.toString)} ++ (if (verbose) Map("_shards_" -> srv.key.sourceShards.mkString(","), "_partIds_" -> srv.key.partIds.mkString(",")) diff --git a/query/src/main/scala/filodb/query/ResultTypes.scala b/query/src/main/scala/filodb/query/ResultTypes.scala index 69020df1b4..8e78cb39fc 100644 --- a/query/src/main/scala/filodb/query/ResultTypes.scala +++ b/query/src/main/scala/filodb/query/ResultTypes.scala @@ -3,7 +3,7 @@ package filodb.query import enumeratum.{Enum, EnumEntry} import filodb.core.{DatasetRef, ErrorResponse, NodeCommand, NodeResponse} -import filodb.core.query.{ResultSchema, SerializableRangeVector} +import filodb.core.query.{RangeVector, ResultSchema} trait QueryCommand extends NodeCommand with java.io.Serializable { def submitTime: Long @@ -35,11 +35,11 @@ object QueryResultType extends Enum[QueryResultType] { final case class QueryResult(id: String, resultSchema: ResultSchema, - result: Seq[SerializableRangeVector]) extends QueryResponse { + result: Seq[RangeVector]) extends QueryResponse { def resultType: QueryResultType = { result match { - case Seq(one) => if (one.numRows == 1) QueryResultType.Scalar else QueryResultType.RangeVectors - case many: Seq[SerializableRangeVector] => if (many.forall(_.numRows == 1)) QueryResultType.InstantVector + case Seq(one) => if (one.numRows.contains(1)) QueryResultType.Scalar else QueryResultType.RangeVectors + case many: Seq[RangeVector] => if (many.forall(_.numRows.contains(1))) QueryResultType.InstantVector else QueryResultType.RangeVectors } } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index bb1489a3f3..f9ccd8ac91 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -125,7 +125,7 @@ trait ExecPlan extends QueryCommand { finalRes._1 .map { case srv: SerializableRangeVector => - numResultSamples += srv.numRows + numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > limit) throw new BadQueryException(s"This query results in more than $limit samples. " + @@ -134,7 +134,7 @@ trait ExecPlan extends QueryCommand { case rv: RangeVector => // materialize, and limit rows per RV val srv = SerializableRangeVector(rv, builder, recSchema, printTree(false)) - numResultSamples += srv.numRows + numResultSamples += srv.numRowsInt // fail the query instead of limiting range vectors and returning incomplete/inaccurate results if (enforceLimit && numResultSamples > limit) throw new BadQueryException(s"This query results in more than $limit samples. " + @@ -222,7 +222,7 @@ trait ExecPlan extends QueryCommand { } } - protected def rowIterAccumulator(srvsList: List[Seq[SerializableRangeVector]]): Iterator[RowReader] = { + protected def rowIterAccumulator(srvsList: List[Seq[RangeVector]]): Iterator[RowReader] = { new Iterator[RowReader] { val listSize = srvsList.size diff --git a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala index 6a22e62f79..35c7350132 100644 --- a/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/MetadataExecPlan.scala @@ -12,8 +12,7 @@ import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.core.store.ChunkSource -import filodb.memory.format._ -import filodb.memory.format.UTF8MapIteratorRowReader +import filodb.memory.format.{UTF8MapIteratorRowReader, ZeroCopyUTF8String} import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ import filodb.query.Query.qLogger @@ -87,13 +86,17 @@ final case class LabelValuesDistConcatExec(id: String, case (QueryError(_, ex), _) => throw ex }.toListL.map { resp => var metadataResult = scala.collection.mutable.Set.empty[Map[ZeroCopyUTF8String, ZeroCopyUTF8String]] - resp.foreach(rv => { - metadataResult ++= rv(0).rows.map(rowReader => { - val binaryRowReader = rowReader.asInstanceOf[BinaryRecordRowReader] - rv(0).schema.toStringPairs(binaryRowReader.recordBase, binaryRowReader.recordOffset) - .map(pair => pair._1.utf8 -> pair._2.utf8).toMap - }) - }) + resp.foreach { rv => + metadataResult ++= rv.head.rows.map { rowReader => + val binaryRowReader = rowReader.asInstanceOf[BinaryRecordRowReader] + rv.head match { + case srv: SerializableRangeVector => + srv.schema.toStringPairs (binaryRowReader.recordBase, binaryRowReader.recordOffset) + .map (pair => pair._1.utf8 -> pair._2.utf8).toMap + case _ => throw new UnsupportedOperationException("Metadata query currently needs SRV results") + } + } + } //distinct -> result may have duplicates in case of labelValues IteratorBackedRangeVector(new CustomRangeVectorKey(Map.empty), new UTF8MapIteratorRowReader(metadataResult.toIterator)) diff --git a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala index 2b1c041197..110d366e97 100644 --- a/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala +++ b/query/src/main/scala/filodb/query/exec/SetOperatorExec.scala @@ -61,7 +61,7 @@ final case class SetOperatorExec(id: String, val lhsRvs = resp.filter(_._2 < lhs.size).flatMap(_._1) val rhsRvs = resp.filter(_._2 >= lhs.size).flatMap(_._1) - val results: List[SerializableRangeVector] = binaryOp match { + val results: List[RangeVector] = binaryOp match { case LAND => setOpAnd(lhsRvs, rhsRvs) case LOR => setOpOr(lhsRvs, rhsRvs) case LUnless => setOpUnless(lhsRvs, rhsRvs) @@ -78,10 +78,10 @@ final case class SetOperatorExec(id: String, else rvk.labelValues.filterNot(lv => ignoringLabels.contains(lv._1)) } - private def setOpAnd(lhsRvs: List[SerializableRangeVector] - , rhsRvs: List[SerializableRangeVector]): List[SerializableRangeVector] = { + private def setOpAnd(lhsRvs: List[RangeVector] + , rhsRvs: List[RangeVector]): List[RangeVector] = { val rhsKeysSet = new mutable.HashSet[Map[Utf8Str, Utf8Str]]() - var result = new ListBuffer[SerializableRangeVector]() + var result = new ListBuffer[RangeVector]() rhsRvs.foreach { rv => val jk = joinKeys(rv.key) if (!jk.isEmpty) @@ -99,10 +99,10 @@ final case class SetOperatorExec(id: String, result.toList } - private def setOpOr(lhsRvs: List[SerializableRangeVector] - , rhsRvs: List[SerializableRangeVector]): List[SerializableRangeVector] = { + private def setOpOr(lhsRvs: List[RangeVector] + , rhsRvs: List[RangeVector]): List[RangeVector] = { val lhsKeysSet = new mutable.HashSet[Map[Utf8Str, Utf8Str]]() - var result = new ListBuffer[SerializableRangeVector]() + var result = new ListBuffer[RangeVector]() // Add everything from left hand side range vector lhsRvs.foreach { rv => val jk = joinKeys(rv.key) @@ -119,10 +119,10 @@ final case class SetOperatorExec(id: String, result.toList } - private def setOpUnless(lhsRvs: List[SerializableRangeVector] - , rhsRvs: List[SerializableRangeVector]): List[SerializableRangeVector] = { + private def setOpUnless(lhsRvs: List[RangeVector] + , rhsRvs: List[RangeVector]): List[RangeVector] = { val rhsKeysSet = new mutable.HashSet[Map[Utf8Str, Utf8Str]]() - var result = new ListBuffer[SerializableRangeVector]() + var result = new ListBuffer[RangeVector]() rhsRvs.foreach { rv => val jk = joinKeys(rv.key) rhsKeysSet += jk diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 225347f80a..6c7b49c722 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -11,11 +11,11 @@ import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} +import filodb.core.{query, TestData} import filodb.core.MetricsTestData._ -import filodb.core.TestData import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} -import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.query.{ColumnFilter, Filter, SerializableRangeVector} import filodb.core.store.{InMemoryMetaStore, NullColumnStore} import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ @@ -85,7 +85,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val rv = response(0) rv.rows.size shouldEqual 1 val record = rv.rows.next().asInstanceOf[BinaryRecordRowReader] - rv.schema.toStringPairs(record.recordBase, record.recordOffset) + rv.asInstanceOf[query.SerializableRangeVector].schema.toStringPairs(record.recordBase, record.recordOffset) } } result shouldEqual jobQueryResult1 @@ -118,7 +118,8 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val result = resp match { case QueryResult(id, _, response) => { response.size shouldEqual 1 - response(0).rows.map (row => response(0).schema.toStringPairs(row.getBlobBase(0), + response(0).rows.map (row => response(0).asInstanceOf[SerializableRangeVector] + .schema.toStringPairs(row.getBlobBase(0), row.getBlobOffset(0)).toMap).toList } } @@ -137,7 +138,8 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val result = resp match { case QueryResult(id, _, response) => { response.size shouldEqual 1 - response(0).rows.map (row => response(0).schema.toStringPairs(row.getBlobBase(0), + response(0).rows.map (row => response(0).asInstanceOf[query.SerializableRangeVector] + .schema.toStringPairs(row.getBlobBase(0), row.getBlobOffset(0)).toMap).toList } } From 134e3a30d6a9e1d5206a170e82c4d29698c687ed Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Wed, 31 Jul 2019 15:05:48 -0700 Subject: [PATCH 06/28] bug(core): Release IndexSearchers after each use (#453) After each acquire of searcher, we need to call release so that searcher releases the segments and they are deleted after merge. Prior to this change, release calls were not being made. Currently, we are seeing a constant growth of disk space because old segments are never deleted. Expecting that this change will fix it. Also renamed commit operation to refresh. Those two are technically different in Lucene. We are actually not doing commit at the bottom of the call stack. --- .../MemstoreCassandraSinkSpec.scala | 2 +- .../scala/filodb.coordinator/QueryActor.scala | 2 +- .../ClusterRecoverySpec.scala | 2 +- .../NodeCoordinatorActorSpec.scala | 16 +-- .../scala/filodb.core/memstore/MemStore.scala | 4 +- .../memstore/PartKeyLuceneIndex.scala | 99 +++++++++++-------- .../memstore/TimeSeriesMemStore.scala | 14 +-- .../memstore/TimeSeriesShard.scala | 8 +- .../memstore/DemandPagedChunkStoreSpec.scala | 2 +- .../memstore/PartKeyLuceneIndexSpec.scala | 32 +++--- .../TimeSeriesMemStoreForMetadataSpec.scala | 2 +- .../memstore/TimeSeriesMemStoreSpec.scala | 44 ++++----- .../filodb.jmh/HistogramQueryBenchmark.scala | 4 +- .../filodb.jmh/PartKeyIndexBenchmark.scala | 2 +- .../filodb.jmh/QueryAndIngestBenchmark.scala | 2 +- .../QueryHiCardInMemoryBenchmark.scala | 2 +- .../filodb.jmh/QueryInMemoryBenchmark.scala | 2 +- .../filodb.jmh/QueryOnDemandBenchmark.scala | 2 +- .../filodb/query/exec/MetadataExecSpec.scala | 2 +- .../exec/SelectRawPartitionsExecSpec.scala | 8 +- 20 files changed, 134 insertions(+), 117 deletions(-) diff --git a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala index ab1ee26530..716406fce6 100644 --- a/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala +++ b/cassandra/src/test/scala/filodb.cassandra/MemstoreCassandraSinkSpec.scala @@ -46,7 +46,7 @@ class MemstoreCassandraSinkSpec extends AllTablesTest { memStore.store.sinkStats.chunksetsWritten should be >= 3 memStore.store.sinkStats.chunksetsWritten should be <= 4 - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Verify data still in MemStore... all of it val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index 5c3fb5aba5..aeedf03e4d 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -163,7 +163,7 @@ final class QueryActor(memStore: MemStore, case q: ExecPlan => execPhysicalPlan2(q, sender()) case GetIndexNames(ref, limit, _) => - sender() ! memStore.indexNames(ref).take(limit).map(_._1).toBuffer + sender() ! memStore.indexNames(ref, limit).map(_._1).toBuffer case g: GetIndexValues => processIndexValues(g, sender()) case ThrowException(dataset) => diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index 74ff8634c9..67c65289e7 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -134,7 +134,7 @@ abstract class ClusterRecoverySpec extends ClusterSpec(ClusterRecoverySpecConfig case CurrentShardSnapshot(ref, newMap) if ref == dataset6.ref => mapper = newMap } } - cluster.memStore.commitIndexForTesting(dataset6.ref) + cluster.memStore.refreshIndexForTesting(dataset6.ref) enterBarrier("ingestion-stopped") // val query = LogicalPlanQuery(dataset6.ref, diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 202d0813d2..699c7e7642 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -145,7 +145,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, multiSeriesData().take(20)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Query existing partition: Series 1 val q1 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "Series 1"), Seq("min")), qOpt) @@ -191,7 +191,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew Aggregate(AggregationOperator.Avg, PeriodicSeries( RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), qOpt) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) probe.send(coordinatorActor, q2) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => @@ -232,7 +232,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(40)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val numQueries = 6 @@ -262,7 +262,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 1, records(dataset1, linearMultiSeries(130000L).take(20)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Should return results from both shards // shard 1 - timestamps 110000 -< 130000; shard 2 - timestamps 130000 <- 1400000 @@ -291,7 +291,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 1, records(dataset1, linearMultiSeries(130000L).take(20)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val queryOpt = QueryOptions(shardOverrides = Some(Seq(0, 1))) val series2 = (2 to 4).map(n => s"Series $n") @@ -317,7 +317,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(30)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) probe.send(coordinatorActor, GetIndexNames(ref)) probe.expectMsg(Seq("series")) @@ -331,7 +331,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, IngestRows(ref, 0, records(dataset1, linearMultiSeries().take(30)))) probe.expectMsg(Ack(0L)) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) probe.send(coordinatorActor, GetIndexNames(ref)) probe.expectMsg(Seq("series")) @@ -362,7 +362,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew probe.send(coordinatorActor, StatusActor.GetCurrentEvents) probe.expectMsg(Map(ref -> Seq(IngestionStarted(ref, 0, coordinatorActor)))) - memStore.commitIndexForTesting(dataset6.ref) + memStore.refreshIndexForTesting(dataset6.ref) // Also the original aggregator is sum(sum_over_time(....)) which is not quite represented by below plan // Below plan is really sum each time bucket val q2 = LogicalPlan2Query(ref, diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index cdfb1181a1..ac7e4e6da5 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -139,7 +139,7 @@ trait MemStore extends ChunkSource { * all shards on this node * @return an index name and shard number */ - def indexNames(dataset: DatasetRef): Iterator[(String, Int)] + def indexNames(dataset: DatasetRef, limit: Int): Seq[(String, Int)] /** * Returns values for a given index name (and # of series for each) for a dataset and shard, @@ -193,7 +193,7 @@ trait MemStore extends ChunkSource { /** * Commits the index immediately so that queries can pick up the latest changes. Used for testing. */ - def commitIndexForTesting(dataset: DatasetRef): Unit + def refreshIndexForTesting(dataset: DatasetRef): Unit /** * WARNING: truncates all the data in the memstore for the given dataset, and also the data diff --git a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala index 82cb286ddd..6bc4c6c9a6 100644 --- a/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala +++ b/core/src/main/scala/filodb.core/memstore/PartKeyLuceneIndex.scala @@ -158,10 +158,20 @@ class PartKeyLuceneIndex(dataset: Dataset, def partIdsEndedBefore(endedBefore: Long): EWAHCompressedBitmap = { val collector = new PartIdCollector() val deleteQuery = LongPoint.newRangeQuery(PartKeyLuceneIndex.END_TIME, 0, endedBefore) - searcherManager.acquire().search(deleteQuery, collector) + + withNewSearcher(s => s.search(deleteQuery, collector)) collector.result } + private def withNewSearcher(func: IndexSearcher => Unit): Unit = { + val s = searcherManager.acquire() + try { + func(s) + } finally { + searcherManager.release(s) + } + } + /** * Delete partitions with given partIds */ @@ -202,46 +212,54 @@ class PartKeyLuceneIndex(dataset: Dataset, */ def indexValues(fieldName: String, topK: Int = 100): Seq[TermInfo] = { // FIXME this API returns duplicate values because same value can be present in multiple lucene segments - val searcher = searcherManager.acquire() - val indexReader = searcher.getIndexReader - val segments = indexReader.leaves() + val freqOrder = Ordering.by[TermInfo, Int](_.freq) val topkResults = new PriorityQueue[TermInfo](topK, freqOrder) - var termsRead = 0 - segments.asScala.foreach { segment => - val terms = segment.reader().terms(fieldName) - - //scalastyle:off - if (terms != null) { - val termsEnum = terms.iterator() - var nextVal: BytesRef = termsEnum.next() - while (nextVal != null && termsRead < MAX_TERMS_TO_ITERATE) { - //scalastyle:on - val valu = BytesRef.deepCopyOf(nextVal) // copy is needed since lucene uses a mutable cursor underneath - val ret = new UTF8Str(valu.bytes, bytesRefToUnsafeOffset(valu.offset), valu.length) - val freq = termsEnum.docFreq() - if (topkResults.size < topK) { - topkResults.add(TermInfo(ret, freq)) - } - else if (topkResults.peek.freq < freq) { - topkResults.remove() - topkResults.add(TermInfo(ret, freq)) + + withNewSearcher { searcher => + val indexReader = searcher.getIndexReader + val segments = indexReader.leaves() + var termsRead = 0 + segments.asScala.foreach { segment => + val terms = segment.reader().terms(fieldName) + + //scalastyle:off + if (terms != null) { + val termsEnum = terms.iterator() + var nextVal: BytesRef = termsEnum.next() + while (nextVal != null && termsRead < MAX_TERMS_TO_ITERATE) { + //scalastyle:on + val valu = BytesRef.deepCopyOf(nextVal) // copy is needed since lucene uses a mutable cursor underneath + val ret = new UTF8Str(valu.bytes, bytesRefToUnsafeOffset(valu.offset), valu.length) + val freq = termsEnum.docFreq() + if (topkResults.size < topK) { + topkResults.add(TermInfo(ret, freq)) + } + else if (topkResults.peek.freq < freq) { + topkResults.remove() + topkResults.add(TermInfo(ret, freq)) + } + nextVal = termsEnum.next() + termsRead += 1 } - nextVal = termsEnum.next() - termsRead += 1 } } } topkResults.toArray(new Array[TermInfo](0)).sortBy(-_.freq).toSeq } - def indexNames: scala.Iterator[String] = { - val searcher = searcherManager.acquire() - val indexReader = searcher.getIndexReader - val segments = indexReader.leaves() - segments.asScala.iterator.flatMap { segment => - segment.reader().getFieldInfos.asScala.toIterator.map(_.name) - }.filterNot { n => ignoreIndexNames.contains(n) } + def indexNames(limit: Int): Seq[String] = { + + var ret: Seq[String] = Nil + withNewSearcher { searcher => + val indexReader = searcher.getIndexReader + val segments = indexReader.leaves() + val iter = segments.asScala.iterator.flatMap { segment => + segment.reader().getFieldInfos.asScala.toIterator.map(_.name) + }.filterNot { n => ignoreIndexNames.contains(n) } + ret = iter.take(limit).toSeq + } + ret } private def addIndexEntry(labelName: String, value: BytesRef, partIndex: Int): Unit = { @@ -319,7 +337,7 @@ class PartKeyLuceneIndex(dataset: Dataset, */ def partKeyFromPartId(partId: Int): Option[BytesRef] = { val collector = new SinglePartKeyCollector() - searcherManager.acquire().search(new TermQuery(new Term(PART_ID, partId.toString)), collector) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector) ) Option(collector.singleResult) } @@ -328,7 +346,7 @@ class PartKeyLuceneIndex(dataset: Dataset, */ def startTimeFromPartId(partId: Int): Long = { val collector = new NumericDocValueCollector(PartKeyLuceneIndex.START_TIME) - searcherManager.acquire().search(new TermQuery(new Term(PART_ID, partId.toString)), collector) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector)) collector.singleResult } @@ -347,7 +365,7 @@ class PartKeyLuceneIndex(dataset: Dataset, } // dont use BooleanQuery which will hit the 1024 term limit. Instead use TermInSetQuery which is // more efficient within Lucene - searcherManager.acquire().search(new TermInSetQuery(PART_ID, terms), collector) + withNewSearcher(s => s.search(new TermInSetQuery(PART_ID, terms), collector)) span.finish() collector.startTimes } @@ -357,7 +375,7 @@ class PartKeyLuceneIndex(dataset: Dataset, */ def endTimeFromPartId(partId: Int): Long = { val collector = new NumericDocValueCollector(PartKeyLuceneIndex.END_TIME) - searcherManager.acquire().search(new TermQuery(new Term(PART_ID, partId.toString)), collector) + withNewSearcher(s => s.search(new TermQuery(new Term(PART_ID, partId.toString)), collector)) collector.singleResult } @@ -371,13 +389,13 @@ class PartKeyLuceneIndex(dataset: Dataset, fromEndTime: Long = 0, toEndTime: Long = Long.MaxValue): EWAHCompressedBitmap = { val coll = new TopKPartIdsCollector(topk) - searcherManager.acquire().search(LongPoint.newRangeQuery(END_TIME, fromEndTime, toEndTime), coll) + withNewSearcher(s => s.search(LongPoint.newRangeQuery(END_TIME, fromEndTime, toEndTime), coll)) coll.topKPartIDsBitmap() } def foreachPartKeyStillIngesting(func: (Int, BytesRef) => Unit): Int = { val coll = new ActionCollector(func) - searcherManager.acquire().search(LongPoint.newExactQuery(END_TIME, Long.MaxValue), coll) + withNewSearcher(s => s.search(LongPoint.newExactQuery(END_TIME, Long.MaxValue), coll)) coll.numHits } @@ -403,7 +421,7 @@ class PartKeyLuceneIndex(dataset: Dataset, * Refresh readers with updates to index. May be expensive - use carefully. * @return */ - def commitBlocking(): Unit = { + def refreshReadersBlocking(): Unit = { searcherManager.maybeRefreshBlocking() logger.info("Refreshed index searchers to make reads consistent") } @@ -469,9 +487,8 @@ class PartKeyLuceneIndex(dataset: Dataset, booleanQuery.add(LongPoint.newRangeQuery(END_TIME, startTime, Long.MaxValue), Occur.FILTER) val query = booleanQuery.build() logger.debug(s"Querying dataset=${dataset.ref} shard=$shardNum partKeyIndex with: $query") - val searcher = searcherManager.acquire() val collector = new PartIdCollector() // passing zero for unlimited results - searcher.search(query, collector) + withNewSearcher(s => s.search(query, collector)) partKeySpan.finish() collector.result } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index 98d7a17747..b52b8b8877 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -73,9 +73,9 @@ extends MemStore with StrictLogging { /** * WARNING: use only for testing. Not performant */ - def commitIndexForTesting(dataset: DatasetRef): Unit = + def refreshIndexForTesting(dataset: DatasetRef): Unit = datasets.get(dataset).foreach(_.values().asScala.foreach { s => - s.commitPartKeyIndexBlocking() + s.refreshPartKeyIndexBlocking() }) /** @@ -180,13 +180,13 @@ extends MemStore with StrictLogging { } } - def indexNames(dataset: DatasetRef): Iterator[(String, Int)] = + def indexNames(dataset: DatasetRef, limit: Int): Seq[(String, Int)] = datasets.get(dataset).map { shards => - shards.entrySet.iterator.asScala.flatMap { entry => + shards.entrySet.asScala.flatMap { entry => val shardNum = entry.getKey.toInt - entry.getValue.indexNames.map { s => (s, shardNum) } - } - }.getOrElse(Iterator.empty) + entry.getValue.indexNames(limit).map { s => (s, shardNum) } + }.toSeq + }.getOrElse(Nil) def labelValues(dataset: DatasetRef, shard: Int, labelName: String, topK: Int = 100): Seq[TermInfo] = getShard(dataset, shard).map(_.labelValues(labelName, topK)).getOrElse(Nil) diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 67b9051577..24b1022cf2 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -476,7 +476,7 @@ class TimeSeriesShard(val dataset: Dataset, def completeIndexRecovery(): Unit = { assertThreadName(IngestSchedName) - commitPartKeyIndexBlocking() + refreshPartKeyIndexBlocking() startFlushingIndex() // start flushing index now that we have recovered logger.info(s"Bootstrapped index for dataset=${dataset.ref} shard=$shardNum") } @@ -555,7 +555,7 @@ class TimeSeriesShard(val dataset: Dataset, s" numPartsInIndex=${partIdMap.size} numIngestingParts=${partitions.size}") } - def indexNames: Iterator[String] = partKeyIndex.indexNames + def indexNames(limit: Int): Seq[String] = partKeyIndex.indexNames(limit) def labelValues(labelName: String, topK: Int): Seq[TermInfo] = partKeyIndex.indexValues(labelName, topK) @@ -650,9 +650,9 @@ class TimeSeriesShard(val dataset: Dataset, } /** - * WARNING: use only for testing. Not performant + * WARNING: Not performant. Use only in tests, or during initial bootstrap. */ - def commitPartKeyIndexBlocking(): Unit = partKeyIndex.commitBlocking() + def refreshPartKeyIndexBlocking(): Unit = partKeyIndex.refreshReadersBlocking() def closePartKeyIndex(): Unit = partKeyIndex.closeIndex() diff --git a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala index 4792f2c6af..863fffaefe 100644 --- a/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/DemandPagedChunkStoreSpec.scala @@ -42,7 +42,7 @@ class DemandPagedChunkStoreSpec extends FunSpec with AsyncTest { val initData = records(dataset1, linearMultiSeries(start).take(20)) memStore.ingest(dataset1.ref, 0, initData) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 val rawData = linearMultiSeries(start, timeStep=100000).drop(100).take(900) // makes 9 chunks per partition? diff --git a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala index 611dc48f25..3c7acc7dc2 100644 --- a/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartKeyLuceneIndexSpec.scala @@ -27,7 +27,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { before { keyIndex.reset() - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() } after { @@ -50,7 +50,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, System.currentTimeMillis())() } val end = System.currentTimeMillis() - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() // Should get empty iterator when passing no filters val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) @@ -90,7 +90,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, time)() if (i%2 == 0) keyIndex.upsertPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, time, time + 300)() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() keyIndex.indexNumEntries shouldEqual 10 @@ -112,7 +112,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { .zipWithIndex.foreach { case (addr, i) => keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, start + i)() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() val startTimes = keyIndex.startTimeFromPartIds((0 until numPartIds).iterator) for { i <- 0 until numPartIds} { @@ -129,7 +129,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { .zipWithIndex.foreach { case (addr, i) => keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, start + i, start + i + 100)() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() val pIds = keyIndex.partIdsEndedBefore(start + 200) for { i <- 0 until numPartIds} { @@ -137,7 +137,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { } keyIndex.removePartKeys(pIds) - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() for { i <- 0 until numPartIds} { keyIndex.partKeyFromPartId(i).isDefined shouldEqual (if (i <= 100) false else true) @@ -152,11 +152,11 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { .zipWithIndex.foreach { case (addr, i) => val time = System.currentTimeMillis() keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, time)() - keyIndex.commitBlocking() // updates need to be able to read startTime from index, so commit + keyIndex.refreshReadersBlocking() // updates need to be able to read startTime from index, so commit keyIndex.updatePartKeyWithEndTime(partKeyOnHeap(dataset6, ZeroPointer, addr), i, time + 10000)() } val end = System.currentTimeMillis() - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() // Should get empty iterator when passing no filters val partNums1 = keyIndex.partIdsFromFilters(Nil, start, end) @@ -194,7 +194,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, System.currentTimeMillis())() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() val filter2 = ColumnFilter("Actor2Name", Equals(UTF8Wrapper("REGIME".utf8))) val partNums2 = keyIndex.partIdsFromFilters(Seq(filter2), 0, Long.MaxValue) @@ -212,9 +212,9 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, System.currentTimeMillis())() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() - keyIndex.indexNames.toList shouldEqual Seq("Actor2Code", "Actor2Name") + keyIndex.indexNames(10).toList shouldEqual Seq("Actor2Code", "Actor2Name") keyIndex.indexValues("not_found").toSeq should equal (Nil) val infos = Seq("AFR", "CHN", "COP", "CVL", "EGYEDU").map(_.utf8).map(TermInfo(_, 1)) @@ -233,7 +233,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, System.currentTimeMillis())() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), ColumnFilter("Actor2Name", Equals("REGIME".utf8))) @@ -251,7 +251,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { partKeyFromRecords(dataset1, records(dataset1, readers.take(10))).zipWithIndex.foreach { case (addr, i) => index2.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, System.currentTimeMillis())() } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() val filters1 = Seq(ColumnFilter("Actor2Code", Equals("GOV".utf8)), ColumnFilter("Year", Equals(1979))) val partNums1 = index2.partIdsFromFilters(filters1, 0, Long.MaxValue) @@ -264,7 +264,7 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { .zipWithIndex.foreach { case (addr, i) => val partKeyBytes = partKeyOnHeap(dataset6, ZeroPointer, addr) keyIndex.addPartKey(partKeyBytes, i, System.currentTimeMillis())() - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() keyIndex.partKeyFromPartId(i).get.bytes shouldEqual partKeyBytes // keyIndex.partIdFromPartKey(new BytesRef(partKeyBytes)) shouldEqual i } @@ -276,12 +276,12 @@ class PartKeyLuceneIndexSpec extends FunSpec with Matchers with BeforeAndAfter { .zipWithIndex.map { case (addr, i) => val start = Math.abs(Random.nextLong()) keyIndex.addPartKey(partKeyOnHeap(dataset6, ZeroPointer, addr), i, start)() - keyIndex.commitBlocking() // updates need to be able to read startTime from index, so commit + keyIndex.refreshReadersBlocking() // updates need to be able to read startTime from index, so commit val end = start + Random.nextInt() keyIndex.updatePartKeyWithEndTime(partKeyOnHeap(dataset6, ZeroPointer, addr), i, end)() (end, start, i) } - keyIndex.commitBlocking() + keyIndex.refreshReadersBlocking() for { from <- 0 until 99 // for various from values diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala index 09f79c27c6..65017e225b 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreForMetadataSpec.scala @@ -48,7 +48,7 @@ class TimeSeriesMemStoreForMetadataSpec extends FunSpec with Matchers with Scala override def beforeAll(): Unit = { memStore.setup(timeseriesDataset, 0, TestData.storeConf) memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) - memStore.commitIndexForTesting(timeseriesDataset.ref) + memStore.refreshIndexForTesting(timeseriesDataset.ref) } it ("should read the metadata") { diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala index fefd96cc60..34058c7c06 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesMemStoreSpec.scala @@ -56,9 +56,9 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, rawData) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) - memStore.asInstanceOf[TimeSeriesMemStore].commitIndexForTesting(dataset1.ref) + memStore.asInstanceOf[TimeSeriesMemStore].refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 - memStore.indexNames(dataset1.ref).toSeq should equal (Seq(("series", 0))) + memStore.indexNames(dataset1.ref, 10).toSeq should equal (Seq(("series", 0))) memStore.latestOffset(dataset1.ref, 0) shouldEqual 0 val minSet = rawData.map(_(1).asInstanceOf[Double]).toSet @@ -83,7 +83,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.ingest(dataset1.ref, 0, data) } - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(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 @@ -94,7 +94,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset2, withMap(linearMultiSeries().take(20))) // 2 records per series x 10 series memStore.ingest(dataset2.ref, 0, data) - memStore.asInstanceOf[TimeSeriesMemStore].commitIndexForTesting(dataset2.ref) + memStore.asInstanceOf[TimeSeriesMemStore].refreshIndexForTesting(dataset2.ref) val split = memStore.getScanSplits(dataset2.ref, 1).head val filter = ColumnFilter("n", Filter.Equals("2".utf8)) val agg1 = memStore.scanRows(dataset2, Seq(1), FilteredPartitionScan(split, Seq(filter))).map(_.getDouble(0)).sum @@ -105,7 +105,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.setup(histDataset, 0, TestData.storeConf) val data = linearHistSeries().take(40) memStore.ingest(histDataset.ref, 0, records(histDataset, data)) - memStore.commitIndexForTesting(histDataset.ref) + memStore.refreshIndexForTesting(histDataset.ref) memStore.numRowsIngested(histDataset.ref, 0) shouldEqual 40L // Below will catch any partition match errors. Should only be 10 tsParts. @@ -155,7 +155,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.setup(dataset1, 0, TestData.storeConf) val data = records(dataset1, linearMultiSeries().take(20)) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val filter = ColumnFilter("series", Filter.Equals("Series 1".utf8)) val split = memStore.getScanSplits(dataset1.ref, 1).head @@ -170,7 +170,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.ingest(dataset2.ref, 0, data) val data2 = records(dataset2, withMap(linearMultiSeries(200000L, 6), 6).take(20)) // 5 series only memStore.ingest(dataset2.ref, 1, data2) - memStore.commitIndexForTesting(dataset2.ref) + memStore.refreshIndexForTesting(dataset2.ref) memStore.activeShards(dataset1.ref) should equal (Seq(0, 1)) memStore.numRowsIngested(dataset1.ref, 0) should equal (20L) @@ -178,7 +178,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val splits = memStore.getScanSplits(dataset2.ref, 1) splits should have length (2) - memStore.indexNames(dataset2.ref).toSet should equal ( + memStore.indexNames(dataset2.ref, 10).toSet should equal ( Set(("n", 0), ("series", 0), ("n", 1), ("series", 1))) val filter = ColumnFilter("n", Filter.Equals("2".utf8)) @@ -206,7 +206,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w fut1.futureValue fut2.futureValue - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) .map(_.getDouble(0)).sum @@ -245,7 +245,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // Two flushes and 3 chunksets have been flushed chunksetsWritten shouldEqual initChunksWritten + 4 - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Try reading - should be able to read optimized chunks too val splits = memStore.getScanSplits(dataset1.ref, 1) val agg1 = memStore.scanRows(dataset1, Seq(1), FilteredPartitionScan(splits.head)) @@ -299,7 +299,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w timeBucketRb.optimalContainerBytes(true).foreach { bytes => tsShard.extractTimeBucket(new IndexData(1, 0, RecordContainer(bytes)), partIdMap) } - tsShard.commitPartKeyIndexBlocking() + tsShard.refreshPartKeyIndexBlocking() partIdMap.size shouldEqual partKeys.size partKeys.zipWithIndex.foreach { case (off, i) => val readPartKey = tsShard.partKeyIndex.partKeyFromPartId(i).get @@ -353,7 +353,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // no flushes chunksetsWritten shouldEqual initChunksWritten - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Should have less than 50 records ingested // Try reading - should be able to read optimized chunks too val splits = memStore.getScanSplits(dataset1.ref, 1) @@ -367,9 +367,9 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, multiSeriesData().take(20)) // 2 records per series x 10 series memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 - memStore.indexNames(dataset1.ref).toSeq should equal (Seq(("series", 0))) + memStore.indexNames(dataset1.ref, 10).toSeq should equal (Seq(("series", 0))) memStore.truncate(dataset1.ref) @@ -390,7 +390,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w shard.updatePartEndTimeInIndex(part, part.timestampOfLatestSample) endTime = part.timestampOfLatestSample } - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) endTime } @@ -401,7 +401,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, linearMultiSeries().take(10)) memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 memStore.labelValues(dataset1.ref, 0, "series").toSeq should have length (10) @@ -419,7 +419,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.getShardE(dataset1.ref, 0).evictionWatermark shouldEqual endTime + 1 memStore.getShardE(dataset1.ref, 0).addPartitionsDisabled() shouldEqual false - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val split = memStore.getScanSplits(dataset1.ref, 1).head val parts = memStore.scanPartitions(dataset1, Seq(0, 1), FilteredPartitionScan(split)) .toListL.runAsync @@ -437,7 +437,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, linearMultiSeries().take(10)) memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 memStore.labelValues(dataset1.ref, 0, "series").toSeq should have length (10) @@ -473,7 +473,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, linearMultiSeries().take(10)) memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) val shard0 = memStore.getShard(dataset1.ref, 0).get val shard0Partitions = shard0.partitions @@ -517,7 +517,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w val data = records(dataset1, linearMultiSeries().take(10)) memStore.ingest(dataset1.ref, 0, data) - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) memStore.numPartitions(dataset1.ref, 0) shouldEqual 10 memStore.labelValues(dataset1.ref, 0, "series").toSeq should have length (10) @@ -531,7 +531,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w memStore.numPartitions(dataset1.ref, 0) shouldEqual 21 // due to the way the eviction policy works memStore.getShardE(dataset1.ref, 0).evictionWatermark shouldEqual 0 - memStore.commitIndexForTesting(dataset1.ref) + memStore.refreshIndexForTesting(dataset1.ref) // Check partitions are now 0 to 20, 21/22 did not get added val split = memStore.getScanSplits(dataset1.ref, 1).head val parts = memStore.scanPartitions(dataset1, Seq(0, 1), FilteredPartitionScan(split)) @@ -554,7 +554,7 @@ class TimeSeriesMemStoreSpec extends FunSpec with Matchers with BeforeAndAfter w // Ingest normal multi series data with 10 partitions. Should have 10 partitions. val data = records(dataset1, linearMultiSeries(numSeries = numSeries).take(numSeries)) store2.ingest(dataset1.ref, 0, data) - store2.commitIndexForTesting(dataset1.ref) + store2.refreshIndexForTesting(dataset1.ref) store2.numPartitions(dataset1.ref, 0) shouldEqual numSeries shard.bufferPool.poolSize shouldEqual 100 // Two allocations of 200 each = 400; used up 300; 400-300=100 diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index 0bbc5b4bbd..e069c5dcc1 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -53,7 +53,7 @@ class HistogramQueryBenchmark { memStore.setup(histDataset, 0, ingestConf) val hShard = memStore.getShardE(histDataset.ref, 0) histSchemaBuilder.allContainers.foreach { c => hShard.ingest(c, 0) } - memStore.commitIndexForTesting(histDataset.ref) // commit lucene index + memStore.refreshIndexForTesting(histDataset.ref) // commit lucene index // Prometheus hist data: 10 series * 66 = 660 series * 180 samples println("Ingesting containers of prometheus schema data....") @@ -65,7 +65,7 @@ class HistogramQueryBenchmark { memStore.setup(promDataset, 0, ingestConf) val pShard = memStore.getShardE(promDataset.ref, 0) promBuilder.allContainers.foreach { c => pShard.ingest(c, 0) } - memStore.commitIndexForTesting(promDataset.ref) // commit lucene index + memStore.refreshIndexForTesting(promDataset.ref) // commit lucene index val system = ActorSystem("test", ConfigFactory.load("filodb-defaults.conf")) private val cluster = FilodbCluster(system) diff --git a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala index f52faf427f..2bc5e2e9d8 100644 --- a/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/PartKeyIndexBenchmark.scala @@ -38,7 +38,7 @@ class PartKeyIndexBenchmark { } } partKeyBuilder.allContainers.foreach(_.consumeRecords(consumer)) - partKeyIndex.commitBlocking() + partKeyIndex.refreshReadersBlocking() @Benchmark @BenchmarkMode(Array(Mode.Throughput)) diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 825a481eee..6e1a8c98a0 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -114,7 +114,7 @@ class QueryAndIngestBenchmark extends StrictLogging { // Initial ingest just to populate index Await.result(ingestSamples(30), 30.seconds) Thread sleep 2000 - memstore.commitIndexForTesting(dataset.ref) // commit lucene index + memstore.refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Initial ingestion ended, indexes set up") /** diff --git a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index be21db64d8..092916fea7 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -95,7 +95,7 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { Await.result(producingFut, 200.seconds) Thread sleep 2000 val store = cluster.memStore.asInstanceOf[TimeSeriesMemStore] - store.commitIndexForTesting(dataset.ref) // commit lucene index + store.refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Ingestion ended") // Stuff for directly executing queries ourselves diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 73758a37a4..2f2ea2f985 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -97,7 +97,7 @@ class QueryInMemoryBenchmark extends StrictLogging { }.countL.runAsync Await.result(producingFut, 30.seconds) Thread sleep 2000 - cluster.memStore.asInstanceOf[TimeSeriesMemStore].commitIndexForTesting(dataset.ref) // commit lucene index + cluster.memStore.asInstanceOf[TimeSeriesMemStore].refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Ingestion ended") // Stuff for directly executing queries ourselves diff --git a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala index 79d710f5d0..5a8c56e75c 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala @@ -113,7 +113,7 @@ class QueryOnDemandBenchmark extends StrictLogging { }.countL.runAsync Await.result(producingFut, 30.seconds) Thread sleep 2000 - memStore.commitIndexForTesting(dataset.ref) // commit lucene index + memStore.refreshIndexForTesting(dataset.ref) // commit lucene index println(s"Ingestion ended.") // For each invocation, reclaim all blocks to make sure ODP is really happening diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index 6c7b49c722..ee453d4bc9 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -58,7 +58,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo override def beforeAll(): Unit = { memStore.setup(timeseriesDataset, 0, TestData.storeConf) memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) - memStore.commitIndexForTesting(timeseriesDataset.ref) + memStore.refreshIndexForTesting(timeseriesDataset.ref) } override def afterAll(): Unit = { diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index 2bd889b2eb..73d943a367 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -78,10 +78,10 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture memStore.ingest(MMD.histDataset.ref, 0, MMD.records(MMD.histDataset, histData)) memStore.setup(MMD.histMaxDS, 0, TestData.storeConf) memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) - memStore.commitIndexForTesting(timeseriesDataset.ref) - memStore.commitIndexForTesting(MMD.dataset1.ref) - memStore.commitIndexForTesting(MMD.histDataset.ref) - memStore.commitIndexForTesting(MMD.histMaxDS.ref) + memStore.refreshIndexForTesting(timeseriesDataset.ref) + memStore.refreshIndexForTesting(MMD.dataset1.ref) + memStore.refreshIndexForTesting(MMD.histDataset.ref) + memStore.refreshIndexForTesting(MMD.histMaxDS.ref) } override def afterAll(): Unit = { From 46a3b1f2898a27327e3b9869128e6b0c62b06425 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Thu, 1 Aug 2019 13:31:04 -0700 Subject: [PATCH 07/28] misc(prometheus): Add scala-parser-combinators dependencies directly to prometheus module (#454) --- project/FiloBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 2cdd60e8bb..95d755e1dc 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -269,7 +269,8 @@ object FiloBuild extends Build { logbackDep % "test,it") lazy val promDeps = Seq( - "com.google.protobuf" % "protobuf-java" % "2.5.0" + "com.google.protobuf" % "protobuf-java" % "2.5.0", + "org.scala-lang.modules" %% "scala-parser-combinators" % "1.1.1" ) lazy val gatewayDeps = commonDeps ++ Seq( From 6e5389e7602a0eeb62bfe3bb02319359723533c4 Mon Sep 17 00:00:00 2001 From: Vish Ramachandran Date: Fri, 2 Aug 2019 14:52:50 -0700 Subject: [PATCH 08/28] feat(query): Ability to respond to gauge related PromQL queries from downsampled data (#445) Use the right downsampled column based on the time range function used. Modify the time range function based on whether we are querying from downsampled data. For example, count_over_time when applied on downsampled data is sum_over_time on the count column. --- .../queryengine2/QueryEngine.scala | 19 ++- .../scala/filodb.core/metadata/Dataset.scala | 17 ++- .../src/test/scala/filodb.core/TestData.scala | 29 +++-- .../binaryrecord2/BinaryRecordSpec.scala | 2 +- .../downsample/ShardDownsamplerSpec.scala | 6 +- project/FiloBuild.scala | 1 + .../filodb/prometheus/FormatConversion.scala | 2 +- .../main/scala/filodb/query/LogicalPlan.scala | 1 - .../query/exec/AggrOverRangeVectors.scala | 6 +- .../scala/filodb/query/exec/ExecPlan.scala | 3 +- .../query/exec/HistogramQuantileMapper.scala | 3 +- .../query/exec/PeriodicSamplesMapper.scala | 35 ++++-- .../query/exec/RangeVectorTransformer.scala | 14 ++- .../query/exec/SelectRawPartitionsExec.scala | 48 ++++++-- .../filodb/query/exec/StitchRvsExec.scala | 3 +- .../filodb/query/exec/TransientRow.scala | 3 +- .../exec/rangefn/AggrOverTimeFunctions.scala | 74 ++++++++++++ .../query/exec/rangefn/RangeFunction.scala | 112 ++++++++++++------ .../query/exec/AggrOverRangeVectorsSpec.scala | 10 +- .../query/exec/BinaryJoinGroupingSpec.scala | 7 +- .../exec/BinaryJoinSetOperatorSpec.scala | 24 ++-- .../exec/HistogramQuantileMapperSpec.scala | 6 +- .../exec/PeriodicSamplesMapperSpec.scala | 3 +- .../exec/SelectRawPartitionsExecSpec.scala | 26 ++-- .../query/exec/WindowIteratorSpec.scala | 106 +++++++++++++---- .../rangefn/AggrOverTimeFunctionsSpec.scala | 18 ++- .../exec/rangefn/BinaryOperatorSpec.scala | 6 +- .../exec/rangefn/InstantFunctionSpec.scala | 38 +++--- .../query/exec/rangefn/LabelReplaceSpec.scala | 42 ++++--- .../query/exec/rangefn/LableJoinSpec.scala | 24 ++-- 30 files changed, 499 insertions(+), 189 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 126c9cbbd3..0b7ac6255a 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -131,7 +131,8 @@ class QueryEngine(dataset: Dataset, private def walkLogicalPlanTree(logicalPlan: LogicalPlan, queryId: String, submitTime: Long, - options: QueryOptions, spreadProvider: SpreadProvider): PlanResult = { + options: QueryOptions, + spreadProvider: SpreadProvider): PlanResult = { logicalPlan match { case lp: RawSeries => materializeRawSeries(queryId, submitTime, options, lp, spreadProvider) @@ -277,9 +278,10 @@ class QueryEngine(dataset: Dataset, } private def materializeLabelValues(queryId: String, - submitTime: Long, - options: QueryOptions, - lp: LabelValues, spreadProvider : SpreadProvider): PlanResult = { + submitTime: Long, + options: QueryOptions, + lp: LabelValues, + spreadProvider: SpreadProvider): PlanResult = { val filters = lp.labelConstraints.map { case (k, v) => new ColumnFilter(k, Filter.Equals(v)) }.toSeq @@ -326,7 +328,8 @@ class QueryEngine(dataset: Dataset, private def materializeRawChunkMeta(queryId: String, submitTime: Long, options: QueryOptions, - lp: RawChunkMeta, spreadProvider : SpreadProvider): PlanResult = { + lp: RawChunkMeta, + spreadProvider : SpreadProvider): PlanResult = { // Translate column name to ID and validate here val colName = if (lp.column.isEmpty) dataset.options.valueColumn else lp.column val colID = dataset.colIDs(colName).get.head @@ -368,13 +371,9 @@ class QueryEngine(dataset: Dataset, * as those are automatically prepended. */ private def getColumnIDs(dataset: Dataset, cols: Seq[String]): Seq[Types.ColumnId] = { - val realCols = if (cols.isEmpty) Seq(dataset.options.valueColumn) else cols - val ids = dataset.colIDs(realCols: _*) + dataset.colIDs(cols: _*) .recover(missing => throw new BadQueryException(s"Undefined columns $missing")) .get - // avoid duplication if first ids are already row keys - if (ids.take(dataset.rowKeyIDs.length) == dataset.rowKeyIDs) { ids } - else { dataset.rowKeyIDs ++ ids } } private def toChunkScanMethod(rangeSelector: RangeSelector): ChunkScanMethod = { diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index 575111132e..da1bae4996 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -106,6 +106,13 @@ final case class Dataset(name: String, .toOr(One(n)) } .combined.badMap(_.toSeq) + /** + * Throws exception on invalid column name + */ + def dataColId(colName: String): Int = { + dataColumns.find(_.name == colName).map(_.id).get + } + /** * Given a list of column names representing say CSV columns, returns a routing from each data column * in this dataset to the column number in that input column name list. To be used for RoutingRowReader @@ -153,6 +160,7 @@ final case class Dataset(name: String, case class DatasetOptions(shardKeyColumns: Seq[String], metricColumn: String, valueColumn: String, + hasDownsampledData: Boolean = false, // TODO: deprecate these options once we move all input to Telegraf/Influx // They are needed only to differentiate raw Prometheus-sourced data ignoreShardKeyColumnSuffixes: Map[String, Seq[String]] = Map.empty, @@ -164,14 +172,17 @@ case class DatasetOptions(shardKeyColumns: Seq[String], } def toConfig: Config = { - val map: Map[String, Any] = Map( + val map: scala.collection.mutable.Map[String, Any] = scala.collection.mutable.Map( "shardKeyColumns" -> shardKeyColumns.asJava, "metricColumn" -> metricColumn, "valueColumn" -> valueColumn, + "hasDownsampledData" -> hasDownsampledData, "ignoreShardKeyColumnSuffixes" -> ignoreShardKeyColumnSuffixes.mapValues(_.asJava).asJava, "ignoreTagsOnPartitionKeyHash" -> ignoreTagsOnPartitionKeyHash.asJava, "copyTags" -> copyTags.asJava) + + ConfigFactory.parseMap(map.asJava) } @@ -202,6 +213,7 @@ object DatasetOptions { DatasetOptions(shardKeyColumns = config.as[Seq[String]]("shardKeyColumns"), metricColumn = config.getString("metricColumn"), valueColumn = config.getString("valueColumn"), + hasDownsampledData = config.as[Option[Boolean]]("hasDownsampledData").getOrElse(false), ignoreShardKeyColumnSuffixes = config.as[Map[String, Seq[String]]]("ignoreShardKeyColumnSuffixes"), ignoreTagsOnPartitionKeyHash = config.as[Seq[String]]("ignoreTagsOnPartitionKeyHash"), @@ -245,7 +257,8 @@ object Dataset { dataColumns: Seq[String], keyColumns: Seq[String], downsamplers: Seq[String], options : DatasetOptions): Dataset = - make(name, partitionColumns, dataColumns, keyColumns, downsamplers, options).badMap(BadSchemaError).toTry.get + make(name, partitionColumns, dataColumns, keyColumns, + downsamplers, options).badMap(BadSchemaError).toTry.get def apply(name: String, partitionColumns: Seq[String], diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 663db59ccb..28599f235c 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -7,17 +7,16 @@ import monix.eval.Task import monix.reactive.Observable import org.joda.time.DateTime +import filodb.core.Types.{PartitionKey, UTF8Map} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore.{SomeData, TimeSeriesPartitionSpec, WriteBufferPool} -import filodb.core.metadata.Column.ColumnType import filodb.core.metadata.{Dataset, DatasetOptions} +import filodb.core.metadata.Column.ColumnType import filodb.core.query.RawDataRangeVector import filodb.core.store._ -import filodb.core.Types.{PartitionKey, UTF8Map} -import filodb.memory.format._ -import filodb.memory.format.ZeroCopyUTF8String._ -import filodb.memory.format.{vectors => bv} import filodb.memory._ +import filodb.memory.format.{vectors => bv, _} +import filodb.memory.format.ZeroCopyUTF8String._ object TestData { def toChunkSetStream(ds: Dataset, @@ -344,8 +343,10 @@ object MachineMetricsData { } } - val histMaxDS = Dataset("histmax", Seq("tags:map"), - Seq("timestamp:ts", "count:long", "sum:long", "max:double", "h:hist:counter=false")) + val histMaxDS = Dataset.make("histmax", Seq("tags:map"), + Seq("timestamp:ts", "count:long", "sum:long", "max:double", "h:hist:counter=false"), + Seq("timestamp"), + options = DatasetOptions.DefaultOptions).get // Pass in the output of linearHistSeries here. // Adds in the max column before h/hist @@ -408,7 +409,7 @@ object CustomMetricsData { columns, Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("metric", "_ns"), "metric", "count")).get + options = DatasetOptions(Seq("metric", "_ns"), "metric", "count", true)).get val partKeyBuilder = new RecordBuilder(TestData.nativeMem, metricdataset.partKeySchema, 2048) val defaultPartKey = partKeyBuilder.addFromObjects("metric1", "app1") @@ -419,7 +420,7 @@ object CustomMetricsData { columns, Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__"), "__name__", "count")).get + options = DatasetOptions(Seq("__name__"), "__name__", "count", true)).get val partKeyBuilder2 = new RecordBuilder(TestData.nativeMem, metricdataset2.partKeySchema, 2048) val defaultPartKey2 = partKeyBuilder2.addFromObjects(Map(ZeroCopyUTF8String("abc") -> ZeroCopyUTF8String("cba"))) @@ -431,7 +432,15 @@ object MetricsTestData { Seq("timestamp:ts", "value:double:detectDrops=true"), Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + options = DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + + val downsampleDataset = Dataset.make("tsdbdata", + Seq("tags:map"), + Seq("timestamp:ts", "min:double", "max:double", "sum:double", "count:double", "avg:double"), + Seq("timestamp"), + Seq.empty, + options = DatasetOptions(Seq("__name__"), "__name__", "average", true) + ).get val builder = new RecordBuilder(MemFactory.onHeapFactory, timeseriesDataset.ingestionSchema) diff --git a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala index 021cadc36e..c64b61f50d 100644 --- a/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala +++ b/core/src/test/scala/filodb.core/binaryrecord2/BinaryRecordSpec.scala @@ -606,7 +606,7 @@ class BinaryRecordSpec extends FunSpec with Matchers with BeforeAndAfter with Be Seq("timestamp:ts", "value:double"), Seq("timestamp"), Seq.empty, - DatasetOptions(Seq("__name__", "job"), "__name__", "value", Map("dummy" -> Seq("_bucket")))).get + options = DatasetOptions(Seq("__name__", "job"), "__name__", "value", false, Map("dummy" -> Seq("_bucket")))).get val metricName4 = RecordBuilder.trimShardColumn(timeseriesDataset, "__name__", "heap_usage_bucket") metricName4 shouldEqual "heap_usage_bucket" diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index da76c1e532..08f2746926 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -22,14 +22,14 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll Seq("timestamp:ts", "value:double"), Seq("timestamp"), Seq("tTime(0)", "dMin(1)", "dMax(1)", "dSum(1)", "dCount(1)", "dAvg(1)"), - DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + DatasetOptions(Seq("__name__", "job"), "__name__", "value", true)).get val customDataset = Dataset.make("custom2", Seq("name:string", "namespace:string", "instance:string"), Seq("timestamp:ts", "count:double", "min:double", "max:double", "total:double", "avg:double", "h:hist:counter=false"), Seq("timestamp"), Seq("tTime(0)", "dSum(1)", "dMin(2)", "dMax(3)", "dSum(4)", "dAvgAc(5@1)", "hSum(6)"), - DatasetOptions(Seq("name", "namespace"), "name", "total")).get + DatasetOptions(Seq("name", "namespace"), "name", "total", true)).get private val blockStore = MMD.blockStore protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) @@ -70,7 +70,7 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll it ("should formulate downsample ingest schema correctly for custom1 schema") { val dsSchema = downsampleOps.downsampleIngestSchema() dsSchema.columns.map(_.name) shouldEqual - Seq("tTime", "dMin", "dMax", "dSum", "dCount","dAvg", "someStr", "tags") + Seq("tTime", "dMin", "dMax", "dSum", "dCount", "dAvg", "someStr", "tags") dsSchema.columns.map(_.colType) shouldEqual Seq(TimestampColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, DoubleColumn, StringColumn, MapColumn) diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 95d755e1dc..38b56da908 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -275,6 +275,7 @@ object FiloBuild extends Build { lazy val gatewayDeps = commonDeps ++ Seq( scalaxyDep, + logbackDep, "io.monix" %% "monix-kafka-1x" % monixKafkaVersion, "org.rogach" %% "scallop" % "3.1.1" ) diff --git a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala index 350cb50ce0..cfd90e07cd 100644 --- a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala +++ b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala @@ -11,7 +11,7 @@ object FormatConversion { // An official Prometheus-format Dataset object with a single timestamp and value val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double")) .copy(options = DatasetOptions(Seq("__name__", "_ns"), - "__name__", "value", Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), + "__name__", "value", false, Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), Map("exporter" -> "_ns", "job" -> "_ns"))) /** diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 5ced4b9346..3874b24b95 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -38,7 +38,6 @@ case class RawSeries(rangeSelector: RangeSelector, filters: Seq[ColumnFilter], columns: Seq[String]) extends RawSeriesPlan - case class LabelValues(labelNames: Seq[String], labelConstraints: Map[String, String], lookbackTimeInMillis: Long) extends MetadataQueryPlan diff --git a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala index 589660f93d..f32985d46d 100644 --- a/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala +++ b/query/src/main/scala/filodb/query/exec/AggrOverRangeVectors.scala @@ -61,7 +61,8 @@ final case class AggregateMapReduce(aggrOp: AggregationOperator, protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams, without=$without, by=$by" - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { @@ -99,7 +100,8 @@ final case class AggregatePresenter(aggrOp: AggregationOperator, protected[exec] def args: String = s"aggrOp=$aggrOp, aggrParams=$aggrParams" - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index f9ccd8ac91..1e1dd7ad77 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -116,12 +116,11 @@ trait ExecPlan extends QueryCommand { val schema = schemaOfDoExecute(dataset) val finalRes = rangeVectorTransformers.foldLeft((res, schema)) { (acc, transf) => 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)) + (transf.apply(dataset, 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 => diff --git a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala index 27f9a0047a..a4693db05e 100644 --- a/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala +++ b/query/src/main/scala/filodb/query/exec/HistogramQuantileMapper.scala @@ -4,6 +4,7 @@ import monix.reactive.Observable import org.agrona.MutableDirectBuffer import scalaxy.loops._ +import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.vectors.Histogram @@ -45,7 +46,7 @@ case class HistogramQuantileMapper(funcParams: Seq[Any]) extends RangeVectorTran * but should be the rate of increase for that bucket counter. The histogram_quantile function should always * be preceded by a rate function or a sum-of-rate function. */ - override def apply(source: Observable[RangeVector], + override def apply(dataset: Dataset, source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { val res = source.toListL.map { rvs => diff --git a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala index fd58aa2e9c..bb5953d966 100644 --- a/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala +++ b/query/src/main/scala/filodb/query/exec/PeriodicSamplesMapper.scala @@ -9,9 +9,9 @@ import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.core.store.{ChunkSetInfo, WindowedChunkIterator} import filodb.memory.format.{vectors => bv, _} -import filodb.query.{BadQueryException, Query, QueryConfig, RangeFunctionId} -import filodb.query.exec.rangefn.{ChunkedRangeFunction, RangeFunction, Window} +import filodb.query._ import filodb.query.Query.qLogger +import filodb.query.exec.rangefn._ import filodb.query.util.IndexedArrayQueue /** @@ -37,7 +37,8 @@ final case class PeriodicSamplesMapper(start: Long, s"start=$start, step=$step, end=$end, window=$window, functionId=$functionId, funcParams=$funcParams" - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { @@ -47,7 +48,7 @@ final case class PeriodicSamplesMapper(start: Long, val valColType = RangeVectorTransformer.valueColumnType(sourceSchema) val maxCol = if (valColType == ColumnType.HistogramColumn && sourceSchema.colIDs.length > 2) sourceSchema.columns.zip(sourceSchema.colIDs).find(_._1.name == "max").map(_._2) else None - val rangeFuncGen = RangeFunction.generatorFor(functionId, valColType, queryConfig, funcParams, maxCol) + val rangeFuncGen = RangeFunction.generatorFor(dataset, functionId, valColType, queryConfig, funcParams, maxCol) // Generate one range function to check if it is chunked val sampleRangeFunc = rangeFuncGen() @@ -89,15 +90,23 @@ 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 - }, fixedVectorLen = Some(((end - start)/step).toInt + 1)) + override def schema(dataset: Dataset, source: ResultSchema): ResultSchema = { + + if (dataset.options.hasDownsampledData) { + source.copy(columns = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn))) + // TODO need to alter for histograms and other schema types + } else { + 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 + }, fixedVectorLen = Some(((end - start) / step).toInt + 1)) + } + } } /** diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index 964d805f1b..f5c318b6d9 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -26,7 +26,8 @@ import filodb.query.exec.rangefn._ * compute intensive and not I/O intensive. */ trait RangeVectorTransformer extends java.io.Serializable { - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] @@ -60,7 +61,8 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, protected[exec] def args: String = s"function=$function, funcParams=$funcParams" - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { @@ -82,7 +84,7 @@ final case class InstantVectorFunctionMapper(function: InstantFunctionId, if (function == HistogramQuantile) { // Special mapper to pull all buckets together from different Prom-schema time series val mapper = HistogramQuantileMapper(funcParams) - mapper.apply(source, queryConfig, limit, sourceSchema) + mapper.apply(dataset, source, queryConfig, limit, sourceSchema) } else { val instantFunction = InstantFunction.double(function, funcParams) source.map { rv => @@ -158,7 +160,8 @@ final case class ScalarOperationMapper(operator: BinaryOperator, val operatorFunction = BinaryOperatorFunction.factoryMethod(operator) - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { @@ -200,7 +203,8 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, } } - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala index b4432e3832..654c27f666 100644 --- a/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala +++ b/query/src/main/scala/filodb/query/exec/SelectRawPartitionsExec.scala @@ -9,7 +9,8 @@ import filodb.core.{DatasetRef, Types} import filodb.core.metadata.{Column, Dataset} import filodb.core.query.{ColumnFilter, RangeVector, ResultSchema} import filodb.core.store._ -import filodb.query.QueryConfig +import filodb.query.{Query, QueryConfig} +import filodb.query.exec.rangefn.RangeFunction object SelectRawPartitionsExec { import Column.ColumnType._ @@ -18,8 +19,8 @@ object SelectRawPartitionsExec { def histMaxColumn(dataset: Dataset, colIDs: Seq[Types.ColumnId]): Option[Int] = { colIDs.find { id => dataset.dataColumns(id).columnType == HistogramColumn } .flatMap { histColID => - dataset.dataColumns.find { c => c.name == "max" && c.columnType == DoubleColumn } - }.map(_.id) + dataset.dataColumns.find(c => c.name == "max" && c.columnType == DoubleColumn).map(_.id) + } } } @@ -38,18 +39,39 @@ final case class SelectRawPartitionsExec(id: String, colIds: Seq[Types.ColumnId]) extends LeafExecPlan { import SelectRawPartitionsExec._ - require(colIds.nonEmpty) - protected[filodb] def schemaOfDoExecute(dataset: Dataset): ResultSchema = { - val numRowKeyCols = colIds.zip(dataset.rowKeyIDs).takeWhile { case (a, b) => a == b }.length + require(dataset.rowKeyIDs.forall(rk => !colIds.contains(rk)), + "User selected columns should not include timestamp (row-key); it will be auto-prepended") + + val selectedColIds = selectColIds(dataset) + val numRowKeyCols = 1 // hardcoded since a future PR will indeed fix this to 1 timestamp column // Add the max column to the schema together with Histograms for max computation -- just in case it's needed // But make sure the max column isn't already included - histMaxColumn(dataset, colIds).filter { mId => !(colIds contains mId) } + histMaxColumn(dataset, selectedColIds).filter { mId => !(colIds contains mId) } .map { maxColId => - ResultSchema(dataset.infosFromIDs(colIds :+ maxColId), numRowKeyCols, colIDs = (colIds :+ maxColId)) + ResultSchema(dataset.infosFromIDs(selectedColIds :+ maxColId), + numRowKeyCols, colIDs = (selectedColIds :+ maxColId)) }.getOrElse { - ResultSchema(dataset.infosFromIDs(colIds), numRowKeyCols, colIDs = colIds) + ResultSchema(dataset.infosFromIDs(selectedColIds), numRowKeyCols, colIDs = selectedColIds) + } + } + + private def selectColIds(dataset: Dataset) = { + dataset.rowKeyIDs ++ { + if (colIds.nonEmpty) { + // query is selecting specific columns + colIds + } else if (!dataset.options.hasDownsampledData) { + // needs to select raw data + colIds ++ dataset.colIDs(dataset.options.valueColumn).get + } else { + // need to select column based on range function + val colNames = rangeVectorTransformers.find(_.isInstanceOf[PeriodicSamplesMapper]).map { p => + RangeFunction.downsampleColsFromRangeFunction(dataset, p.asInstanceOf[PeriodicSamplesMapper].functionId) + }.getOrElse(Seq(dataset.options.valueColumn)) + colIds ++ dataset.colIDs(colNames: _*).get + } } } @@ -58,10 +80,14 @@ final case class SelectRawPartitionsExec(id: String, queryConfig: QueryConfig) (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = { - require(colIds.indexOfSlice(dataset.rowKeyIDs) == 0) + require(dataset.rowKeyIDs.forall(rk => !colIds.contains(rk)), + "User selected columns should not include timestamp (row-key); it will be auto-prepended") val partMethod = FilteredPartitionScan(ShardSplit(shard), filters) - source.rangeVectors(dataset, colIds, partMethod, chunkMethod) + val selectCols = selectColIds(dataset) + Query.qLogger.debug(s"queryId=$id on dataset=${dataset.ref} shard=$shard" + + s" is configured to use column=$selectCols to serve downsampled results") + source.rangeVectors(dataset, selectCols, partMethod, chunkMethod) } protected def args: String = s"shard=$shard, chunkMethod=$chunkMethod, filters=$filters, colIDs=$colIds" diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index 6cd87b09cf..bc5c97b214 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -89,7 +89,8 @@ final case class StitchRvsExec(id: String, */ final case class StitchRvsMapper() extends RangeVectorTransformer { - def apply(source: Observable[RangeVector], + def apply(dataset: Dataset, + source: Observable[RangeVector], queryConfig: QueryConfig, limit: Int, sourceSchema: ResultSchema): Observable[RangeVector] = { diff --git a/query/src/main/scala/filodb/query/exec/TransientRow.scala b/query/src/main/scala/filodb/query/exec/TransientRow.scala index 1e80a7c633..1a1768156b 100644 --- a/query/src/main/scala/filodb/query/exec/TransientRow.scala +++ b/query/src/main/scala/filodb/query/exec/TransientRow.scala @@ -43,7 +43,8 @@ final class TransientRow(var timestamp: Long, var value: Double) extends Mutable def getBoolean(columnNo: Int): Boolean = throw new IllegalArgumentException() def getInt(columnNo: Int): Int = throw new IllegalArgumentException() def getLong(columnNo: Int): Long = if (columnNo == 0) timestamp else throw new IllegalArgumentException() - def getDouble(columnNo: Int): Double = if (columnNo == 1) value else throw new IllegalArgumentException() + def getDouble(columnNo: Int): Double = if (columnNo == 1) value + else throw new IllegalArgumentException(s"Invalid col $columnNo") def getFloat(columnNo: Int): Float = throw new IllegalArgumentException() def getString(columnNo: Int): String = throw new IllegalArgumentException() def getAny(columnNo: Int): Any = throw new IllegalArgumentException() 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 b19a0b3353..37f4cb94a5 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -222,6 +222,80 @@ class SumAndMaxOverTimeFuncHD(maxColID: Int) extends ChunkedRangeFunction[Transi } } +/** + * Computes Average Over Time using sum and count columns. + * Used in when calculating avg_over_time using downsampled data + */ +class AvgWithSumAndCountOverTimeFuncD(countColId: Int) extends ChunkedRangeFunction[TransientRow] { + private val sumFunc = new SumOverTimeChunkedFunctionD + private val countFunc = new SumOverTimeChunkedFunctionD + + override final def reset(): Unit = { + sumFunc.reset() + countFunc.reset() + } + + final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { + sampleToEmit.setValues(endTimestamp, sumFunc.sum / countFunc.sum) + } + + import BinaryVector.BinaryVectorPtr + + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { + // Do BinarySearch for start/end pos only once for both columns == WIN! + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = Math.min(tsReader.ceilingIndex(tsVector, endTime), info.numRows - 1) + + // At least one sample is present + if (startRowNum <= endRowNum) { + sumFunc.addTimeChunks(valueVector, valueReader, startRowNum, endRowNum) + + // Get valueVector/reader for count column + val countVectPtr = info.vectorPtr(countColId) + countFunc.addTimeChunks(countVectPtr, bv.DoubleVector(countVectPtr), startRowNum, endRowNum) + } + } +} + +/** + * Computes Average Over Time using sum and count columns. + * Used in when calculating avg_over_time using downsampled data + */ +class AvgWithSumAndCountOverTimeFuncL(countColId: Int) extends ChunkedRangeFunction[TransientRow] { + private val sumFunc = new SumOverTimeChunkedFunctionL + private val countFunc = new CountOverTimeChunkedFunction + + override final def reset(): Unit = { + sumFunc.reset() + countFunc.reset() + } + + final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { + sampleToEmit.setValues(endTimestamp, sumFunc.sum / countFunc.count) + } + + import BinaryVector.BinaryVectorPtr + + final def addChunks(tsVector: BinaryVectorPtr, tsReader: bv.LongVectorDataReader, + valueVector: BinaryVectorPtr, valueReader: VectorDataReader, + startTime: Long, endTime: Long, info: ChunkSetInfo, queryConfig: QueryConfig): Unit = { + // Do BinarySearch for start/end pos only once for both columns == WIN! + val startRowNum = tsReader.binarySearch(tsVector, startTime) & 0x7fffffff + val endRowNum = Math.min(tsReader.ceilingIndex(tsVector, endTime), info.numRows - 1) + + // At least one sample is present + if (startRowNum <= endRowNum) { + sumFunc.addTimeChunks(valueVector, valueReader, startRowNum, endRowNum) + + // Get valueVector/reader for count column + val cntVectPtr = info.vectorPtr(countColId) + countFunc.addTimeChunks(cntVectPtr, bv.DoubleVector(cntVectPtr), startRowNum, endRowNum) + } + } +} + class CountOverTimeFunction(var count: Double = Double.NaN) extends RangeFunction { override def addedToWindow(row: TransientRow, window: Window): Unit = { if (!JLDouble.isNaN(row.value)) { 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 ec36706547..251627a9bf 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -1,12 +1,13 @@ package filodb.query.exec.rangefn import filodb.core.metadata.Column.ColumnType +import filodb.core.metadata.Dataset import filodb.core.store.ChunkSetInfo import filodb.memory.format.{vectors => bv, _} import filodb.memory.format.BinaryVector.BinaryVectorPtr import filodb.query.{QueryConfig, RangeFunctionId} -import filodb.query.exec._ import filodb.query.RangeFunctionId._ +import filodb.query.exec._ /** * Container for samples within a window of samples @@ -225,70 +226,113 @@ trait ChunkedLongRangeFunction extends TimeRangeFunction[TransientRow] { object RangeFunction { type RangeFunctionGenerator = () => BaseRangeFunction + + def downsampleColsFromRangeFunction(dataset: Dataset, f: Option[RangeFunctionId]): Seq[String] = { + f match { + case None => Seq("avg") + case Some(Rate) => Seq(dataset.options.valueColumn) + case Some(Irate) => Seq(dataset.options.valueColumn) + case Some(Increase) => Seq(dataset.options.valueColumn) + case Some(Resets) => Seq(dataset.options.valueColumn) + case Some(CountOverTime) => Seq("count") + case Some(Changes) => Seq("avg") + case Some(Delta) => Seq("avg") + case Some(Idelta) => Seq("avg") + case Some(Deriv) => Seq("avg") + case Some(HoltWinters) => Seq("avg") + case Some(PredictLinear) => Seq("avg") + case Some(SumOverTime) => Seq("sum") + case Some(AvgOverTime) => Seq("sum", "count") + case Some(StdDevOverTime) => Seq("avg") + case Some(StdVarOverTime) => Seq("avg") + case Some(QuantileOverTime) => Seq("avg") + case Some(MinOverTime) => Seq("min") + case Some(MaxOverTime) => Seq("max") + } + } /** * Returns a (probably new) instance of RangeFunction given the func ID and column type */ - def apply(func: Option[RangeFunctionId], + def apply(dataset: Dataset, + func: Option[RangeFunctionId], columnType: ColumnType, config: QueryConfig, funcParams: Seq[Any] = Nil, maxCol: Option[Int] = None, useChunked: Boolean): BaseRangeFunction = - generatorFor(func, columnType, config, funcParams, maxCol, useChunked)() + generatorFor(dataset, func, columnType, config, funcParams, maxCol, useChunked)() /** * Given a function type and column type, returns a RangeFunctionGenerator */ - def generatorFor(func: Option[RangeFunctionId], + def generatorFor(dataset: Dataset, + func: Option[RangeFunctionId], columnType: ColumnType, config: QueryConfig, funcParams: Seq[Any] = Nil, maxCol: Option[Int] = None, - useChunked: Boolean = true): RangeFunctionGenerator = + useChunked: Boolean = true): RangeFunctionGenerator = { if (useChunked) columnType match { - case ColumnType.DoubleColumn => doubleChunkedFunction(func, config, funcParams) - case ColumnType.LongColumn => longChunkedFunction(func, funcParams) - case ColumnType.TimestampColumn => longChunkedFunction(func, funcParams) + case ColumnType.DoubleColumn => doubleChunkedFunction(dataset, func, config, funcParams) + case ColumnType.LongColumn => longChunkedFunction(dataset, func, funcParams) + case ColumnType.TimestampColumn => longChunkedFunction(dataset, func, funcParams) case ColumnType.HistogramColumn => histChunkedFunction(func, funcParams, maxCol) - case other: ColumnType => throw new IllegalArgumentException(s"Column type $other not supported") + case other: ColumnType => throw new IllegalArgumentException(s"Column type $other not supported") } else { iteratingFunction(func, funcParams) } + } /** * Returns a function to generate a ChunkedRangeFunction for Long columns */ - def longChunkedFunction(func: Option[RangeFunctionId], - funcParams: Seq[Any] = Nil): RangeFunctionGenerator = func match { - case None => () => new LastSampleChunkedFunctionL - case Some(CountOverTime) => () => new CountOverTimeChunkedFunction() - case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionL - case Some(AvgOverTime) => () => new AvgOverTimeChunkedFunctionL - case Some(MinOverTime) => () => new MinOverTimeChunkedFunctionL - case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionL - case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionL - case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionL - case _ => iteratingFunction(func, funcParams) + def longChunkedFunction(dataset: Dataset, + func: Option[RangeFunctionId], + funcParams: Seq[Any] = Nil): RangeFunctionGenerator = { + func match { + case None => () => new LastSampleChunkedFunctionL + case Some(CountOverTime) => () => if (dataset.options.hasDownsampledData) new SumOverTimeChunkedFunctionL + else new CountOverTimeChunkedFunction() + case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionL + case Some(AvgOverTime) => () => if (dataset.options.hasDownsampledData) { + val cntCol = dataset.dataColId("count") + new AvgWithSumAndCountOverTimeFuncL(cntCol) + } + else new AvgOverTimeChunkedFunctionL + case Some(MinOverTime) => () => new MinOverTimeChunkedFunctionL + case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionL + case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionL + case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionL + case _ => iteratingFunction(func, funcParams) + } } /** * Returns a function to generate a ChunkedRangeFunction for Double columns */ - def doubleChunkedFunction(func: Option[RangeFunctionId], + def doubleChunkedFunction(dataset: Dataset, + func: Option[RangeFunctionId], config: QueryConfig, - funcParams: Seq[Any] = Nil): RangeFunctionGenerator = func match { - case None => () => new LastSampleChunkedFunctionD - case Some(Rate) if config.has("faster-rate") => () => new ChunkedRateFunction - case Some(Increase) if config.has("faster-rate") => () => new ChunkedIncreaseFunction - case Some(Delta) if config.has("faster-rate") => () => new ChunkedDeltaFunction - case Some(CountOverTime) => () => new CountOverTimeChunkedFunctionD() - case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionD - case Some(AvgOverTime) => () => new AvgOverTimeChunkedFunctionD - case Some(MinOverTime) => () => new MinOverTimeChunkedFunctionD - case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionD - case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionD - case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionD - case _ => iteratingFunction(func, funcParams) + funcParams: Seq[Any] = Nil): RangeFunctionGenerator = { + func match { + case None => () => new LastSampleChunkedFunctionD + case Some(Rate) if config.has("faster-rate") => () => new ChunkedRateFunction + case Some(Increase) if config.has("faster-rate") => () => new ChunkedIncreaseFunction + case Some(Delta) if config.has("faster-rate") => () => new ChunkedDeltaFunction + case Some(CountOverTime) => () => if (dataset.options.hasDownsampledData) new SumOverTimeChunkedFunctionD + else new CountOverTimeChunkedFunctionD() + case Some(SumOverTime) => () => new SumOverTimeChunkedFunctionD + case Some(AvgOverTime) => () => if (dataset.options.hasDownsampledData) { + val cntCol = dataset.dataColId("count") + new AvgWithSumAndCountOverTimeFuncD(cntCol) + } + else new AvgOverTimeChunkedFunctionD + case Some(MinOverTime) => () => new MinOverTimeChunkedFunctionD + case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionD + case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionD + case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionD + case _ => iteratingFunction(func, funcParams) + } } def histChunkedFunction(func: Option[RangeFunctionId], diff --git a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala index 9029f3122a..45f53a4b17 100644 --- a/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/AggrOverRangeVectorsSpec.scala @@ -8,7 +8,7 @@ import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.concurrent.ScalaFutures -import filodb.core.{MachineMetricsData => MMD} +import filodb.core.{MetricsTestData, MachineMetricsData => MMD} import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} @@ -22,7 +22,7 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val tvSchema = ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1) val histSchema = ResultSchema(MMD.histDataset.infosFromIDs(Seq(0, 3)), 1) - val histMaxSchema = ResultSchema(MMD.histMaxDS.infosFromIDs(Seq(0, 4, 3)), 1, colIDs=Seq(0, 4, 3)) + val histMaxSchema = ResultSchema(MMD.histMaxDS.infosFromIDs(Seq(0, 4, 3)), 1, colIDs = Seq(0, 4, 3)) it ("should work without grouping") { val ignoreKey = CustomRangeVectorKey( @@ -254,8 +254,10 @@ class AggrOverRangeVectorsSpec extends RawDataWindowingSpec with ScalaFutures { val agg = RowAggregator(AggregationOperator.Avg, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Avg, Nil, Nil, Nil) - val mapped1 = aggMR(Observable.fromIterable(Seq(toRv(s1))), queryConfig, 1000, tvSchema) - val mapped2 = aggMR(Observable.fromIterable(Seq(toRv(s2))), queryConfig, 1000, tvSchema) + val mapped1 = aggMR(MetricsTestData.timeseriesDataset, Observable.fromIterable(Seq(toRv(s1))), + queryConfig, 1000, tvSchema) + val mapped2 = aggMR(MetricsTestData.timeseriesDataset, Observable.fromIterable(Seq(toRv(s2))), + queryConfig, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped1 ++ mapped2, rv=>rv.key) val result4 = resultObs4.toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index 3b996de88a..ec1a39fd80 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -11,6 +11,7 @@ import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures +import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} @@ -183,7 +184,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleNodeCpu), + queryConfig, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue @@ -262,7 +264,8 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val agg = RowAggregator(AggregationOperator.Sum, Nil, tvSchema) val aggMR = AggregateMapReduce(AggregationOperator.Sum, Nil, Nil, Seq("instance", "job")) - val mapped = aggMR(Observable.fromIterable(sampleNodeCpu), queryConfig, 1000, tvSchema) + val mapped = aggMR(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleNodeCpu), + queryConfig, 1000, tvSchema) val resultObs4 = RangeVectorAggregator.mapReduce(agg, true, mapped, rv=>rv.key) val samplesRhs = resultObs4.toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index 766b0b6372..e92bf14d29 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -217,7 +217,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -254,7 +255,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance", "job"), Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -291,7 +293,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Seq("instance"), Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -327,7 +330,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group")) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -362,7 +366,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LAND, Nil, Seq("group", "job")) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -469,7 +474,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs = QueryResult("someId", null, canaryPlusOne.map(rv => SerializableRangeVector(rv, schema))) @@ -529,7 +535,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializableRangeVector(rv, schema))) @@ -600,7 +607,8 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures BinaryOperator.LOR, Nil, Nil) - val canaryPlusOne = scalarOpMapper(Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). + val canaryPlusOne = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleCanary), queryConfig, 1000, resultSchema). toListL.runAsync.futureValue // scalastyle:off val lhs1 = QueryResult("someId", null, sampleHttpRequests.map(rv => SerializableRangeVector(rv, schema))) diff --git a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala index 7676b926c8..4538e26dc2 100644 --- a/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/HistogramQuantileMapperSpec.scala @@ -10,6 +10,7 @@ import org.scalatest.concurrent.ScalaFutures import filodb.core.metadata.Column.ColumnType import filodb.core.query._ +import filodb.core.MetricsTestData import filodb.memory.format.ZeroCopyUTF8String import filodb.query.QueryConfig @@ -17,8 +18,8 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture val config = ConfigFactory.load("application_test.conf").getConfig("filodb") val queryConfig = new QueryConfig(config.getConfig("query")) - import ZeroCopyUTF8String._ import HistogramQuantileMapper._ + import ZeroCopyUTF8String._ private def genHistBuckets(histKey: Map[ZeroCopyUTF8String, ZeroCopyUTF8String]): Array[CustomRangeVectorKey] = { val numBuckets = 8 @@ -56,7 +57,8 @@ class HistogramQuantileMapperSpec extends FunSpec with Matchers with ScalaFuture expectedResult: Seq[(Map[ZeroCopyUTF8String, ZeroCopyUTF8String], Seq[(Int, Double)])]): Unit = { val hqMapper = HistogramQuantileMapper(Seq(q)) - val result = hqMapper.apply(Observable.fromIterable(histRvs), + val result = hqMapper.apply(MetricsTestData.timeseriesDataset, + Observable.fromIterable(histRvs), queryConfig, 10, new ResultSchema(Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)), 1)) .toListL.runAsync.futureValue diff --git a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala index 03ec1a07f8..c2120a1e9c 100644 --- a/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PeriodicSamplesMapperSpec.scala @@ -30,7 +30,8 @@ class PeriodicSamplesMapperSpec extends FunSpec with Matchers with ScalaFutures it("should return value present at time - staleSampleAfterMs") { val periodicSamplesVectorFnMapper = exec.PeriodicSamplesMapper(100000L, 100000, 600000L, None, None) - val resultObs = periodicSamplesVectorFnMapper(Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema) + val resultObs = periodicSamplesVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(Seq(rv)), queryConfig, 1000, resultSchema) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map (r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN)) diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index 73d943a367..36d16808b2 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -93,7 +93,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunkScan, Seq(0, 1)) + timeseriesDataset.ref, 0, filters, AllChunkScan, Nil) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -113,7 +113,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val endTime = now - (numRawSamples-10) * reportingInterval val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, TimeRangeChunkScan(startTime, endTime), Seq(0, 1)) + filters, TimeRangeChunkScan(startTime, endTime), Nil) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -130,7 +130,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture // read from an interval of 100000ms, resulting in 11 samples val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, TimeRangeChunkScan(100000L, 150000L), Seq(0, 4)) + filters, TimeRangeChunkScan(100000L, 150000L), Seq(4)) val resp = execPlan.execute(memStore, MMD.dataset1, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -144,7 +144,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, - filters, TimeRangeChunkScan(100000L, 150000L), Seq(0, 3)) + filters, TimeRangeChunkScan(100000L, 150000L), Seq(3)) val resp = execPlan.execute(memStore, MMD.histDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryResult] @@ -160,7 +160,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, AllChunkScan, Seq(0, 1)) + filters, AllChunkScan, Nil) val start = now - numRawSamples * reportingInterval - 100 // reduce by 100 to not coincide with reporting intervals val step = 20000 val end = now - (numRawSamples-100) * reportingInterval @@ -191,7 +191,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("series", Filter.Equals("Series 1".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, MMD.dataset1.ref, 0, - filters, AllChunkScan, Seq(0, 4)) + filters, AllChunkScan, Seq(4)) // Raw data like 101000, 111000, .... val start = 105000L @@ -211,7 +211,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture import ZeroCopyUTF8String._ val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("id1", now, numRawSamples, dummyDispatcher, MMD.histDataset.ref, 0, - filters, AllChunkScan, Seq(0, 3)) + filters, AllChunkScan, Seq(3)) val start = 105000L val step = 20000L @@ -233,7 +233,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture it("should sum Histogram records with max correctly") { val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("hMax", now, numRawSamples, dummyDispatcher, MMD.histMaxDS.ref, 0, - filters, AllChunkScan, Seq(0, 4)) + filters, AllChunkScan, Seq(4)) val start = 105000L val step = 20000L @@ -275,7 +275,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture it("should extract Histogram with max using Last/None function correctly") { val filters = Seq(ColumnFilter("dc", Filter.Equals("0".utf8))) val execPlan = SelectRawPartitionsExec("hMax", now, numRawSamples, dummyDispatcher, MMD.histMaxDS.ref, 0, - filters, AllChunkScan, Seq(0, 4)) + filters, AllChunkScan, Seq(4)) val start = 105000L val step = 20000L @@ -305,7 +305,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), ColumnFilter("job", Filter.Equals("myCoolService".utf8))) val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, timeseriesDataset.ref, 0, - filters, AllChunkScan, Seq(0, 1)) + filters, AllChunkScan, Nil) val resultSchema = execPlan.schema(timeseriesDataset) resultSchema.isTimeSeries shouldEqual true resultSchema.numRowKeyColumns shouldEqual 1 @@ -317,14 +317,14 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture it("should produce correct schema for histogram RVs with and without max column") { // Histogram dataset, no max column val noMaxPlan = SelectRawPartitionsExec("someQueryId", System.currentTimeMillis, 100, dummyDispatcher, - MMD.histDataset.ref, 0, Nil, AllChunkScan, Seq(0, 3)) + MMD.histDataset.ref, 0, Nil, AllChunkScan, Seq(3)) val expected1 = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("h", HistogramColumn)), 1, colIDs = Seq(0, 3)) noMaxPlan.schemaOfDoExecute(MMD.histDataset) shouldEqual expected1 // Histogram dataset with max column - should add max to schema automatically val maxPlan = SelectRawPartitionsExec("someQueryId", System.currentTimeMillis, 100, dummyDispatcher, - MMD.histMaxDS.ref, 0, Nil, AllChunkScan, Seq(0, 4)) + MMD.histMaxDS.ref, 0, Nil, AllChunkScan, Seq(4)) val expected2 = ResultSchema(Seq(ColumnInfo("timestamp", TimestampColumn), ColumnInfo("h", HistogramColumn), ColumnInfo("max", DoubleColumn)), 1, colIDs = Seq(0, 4, 3)) @@ -364,7 +364,7 @@ class SelectRawPartitionsExecSpec extends FunSpec with Matchers with ScalaFuture // Query returns n ("numRawSamples") samples - Applying Limit (n-1) to fail the query execution // with ResponseTooLargeException val execPlan = SelectRawPartitionsExec("someQueryId", now, numRawSamples - 1, dummyDispatcher, - timeseriesDataset.ref, 0, filters, AllChunkScan, Seq(0, 1)) + timeseriesDataset.ref, 0, filters, AllChunkScan, Nil) val resp = execPlan.execute(memStore, timeseriesDataset, queryConfig).runAsync.futureValue val result = resp.asInstanceOf[QueryError] diff --git a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala index feecd07a35..c8a0d48634 100644 --- a/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/WindowIteratorSpec.scala @@ -1,5 +1,6 @@ package filodb.query.exec +import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType import filodb.query.RangeFunctionId import filodb.query.exec.rangefn.{RangeFunction, RawDataWindowingSpec} @@ -133,9 +134,10 @@ class WindowIteratorSpec extends RawDataWindowingSpec { it ("should ignore out of order samples for RateFunction") { val rawRows = counterSamples.map(s => new TransientRow(s._1, s._2)) - val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1538416154000L, 20000, 1538416649000L,20000, - RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, - queryConfig) + val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, + 1538416154000L, 20000, 1538416649000L, 20000, + RangeFunction(MetricsTestData.timeseriesDataset, Some(RangeFunctionId.Rate), + ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach{ v => // if out of order samples are not removed, counter correction causes rate to spike up to very high value v.value should be < 10000d @@ -160,7 +162,8 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val end = 1000L val step = 5 val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, start, step, - end, 0, RangeFunction(None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) + end, 0, RangeFunction(MetricsTestData.timeseriesDataset, + None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) val result = slidingWinIterator.map(v => (v.timestamp, v.value)).toSeq result.map(_._1) shouldEqual (start to end).by(step) result.foreach{ v => @@ -194,14 +197,16 @@ class WindowIteratorSpec extends RawDataWindowingSpec { 750000->17.0 ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 1100000L, 100000, - RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) // NOTE: dum_over_time sliding iterator does not handle the NaN at the end correctly! // slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList shouldEqual windowResults slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 1100000L, 100000, - RangeFunction(Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.SumOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -227,7 +232,8 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val rawRows = samples.map(s => new TransientRow(s._1, s._2)) val slidingWinIterator = new SlidingWindowIterator(rawRows.iterator, 1548191496000L, 15000, 1548191796000L, 300000, - RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.foreach { v => windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) @@ -235,8 +241,8 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val rv = timeValueRV(samples) val chunkedIt = new ChunkedWindowIteratorD(rv, 1548191496000L, 15000, 1548191796000L, 300000, - RangeFunction(Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, queryConfig, - useChunked = true).asChunkedD, queryConfig) + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.Rate), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.foreach { v => windowResults.find(a => a._1 == v.timestamp).foreach(b => v.value shouldEqual b._2 +- 0.0000000001) } @@ -298,14 +304,15 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val slidingWinIterator = new SlidingWindowIterator(rv.rows, 1540845090000L, 15000, 1540855905000L, 0, - RangeFunction(None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, + RangeFunction(MetricsTestData.timeseriesDataset, + None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults val chunkedWinIt = new ChunkedWindowIteratorD(rv, 1540845090000L, 15000, 1540855905000L, queryConfig.staleSampleAfterMs, - RangeFunction(None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, - queryConfig) + RangeFunction(MetricsTestData.timeseriesDataset, + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -326,14 +333,16 @@ class WindowIteratorSpec extends RawDataWindowingSpec { val slidingWinIterator = new SlidingWindowIterator(rv.rows, 100000L, 100000, 600000L, 0, - RangeFunction(None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, + RangeFunction(MetricsTestData.timeseriesDataset, + None, ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults // ChunkedWindowIterator requires window to be staleSampleAfterMs + 1 when window of SlidingWindowIterator is 0 val chunkedWinIt = new ChunkedWindowIteratorD(rv, 100000L, 100000, 600000L, queryConfig.staleSampleAfterMs + 1, - RangeFunction(None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + RangeFunction(MetricsTestData.timeseriesDataset, + None, ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedWinIt.map(r => (r.getLong(0), r.getDouble(1))).toList.filter(!_._2.isNaN) shouldEqual windowResults } @@ -361,12 +370,14 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.AvgOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } @@ -394,17 +405,64 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.CountOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } + + it("should calculate query results from downsampled data") { + // min, max, sum, count, avg + val samples = Seq( + (100000L, 2d, 5d, 20d, 5d, 2.8d), + (153000L, 1d, 6d, 18d, 3d, 1.4d), + (250000L, 3d, 7d, 21d, 5d, 5d), + (270000L, 2d, 10d, 22d, 4d, 6d), + (280000L, 1.5d, 2d, 10d, 6d, 1.75d), + (360000L, 0.6d, 7d, 23d, 7d, 2d), + (430000L, 7d, 10d, 60d, 5d, 8d), + (690000L, 1.8d, 5d, 25d, 7d, 3d), + (700000L, 4.9d, 12d, 80d, 10d, 10d), + (710000L, 0.1d, 3d, 10d, 10d, 1d) + ) + val rvAvg = timeValueRvDownsample(samples, Array(0, 3, 4)) + + val avgWindowResults = Seq( + 150000 -> 4.0, + 250000 -> 4.875, + 350000 -> 3.2, + 450000 -> 6.916666666666667, + 750000 -> 4.2592592592592595 + ) + val chunkedItAvg = new ChunkedWindowIteratorD(rvAvg, 50000L, 100000, 750000L, 100000, + RangeFunction(MetricsTestData.downsampleDataset, Some(RangeFunctionId.AvgOverTime), + ColumnType.DoubleColumn, queryConfig, + useChunked = true).asChunkedD, queryConfig) + chunkedItAvg.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual avgWindowResults + + val rvCnt = timeValueRvDownsample(samples, Array(0, 4)) + + val countWindowResults = Seq( + 150000 -> 5.0, + 250000 -> 8.0, + 350000 -> 10.0, + 450000 -> 12.0, + 750000 -> 27.0 + ) + val chunkedItCnt = new ChunkedWindowIteratorD(rvCnt, 50000L, 100000, 750000L, 100000, + RangeFunction(MetricsTestData.downsampleDataset, Some(RangeFunctionId.CountOverTime), + ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) + chunkedItCnt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual countWindowResults + } + it("should calculate MinOverTime correctly even for windows with no values") { val samples = Seq( 100000L -> 1d, @@ -428,12 +486,14 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.MinOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults @@ -462,12 +522,14 @@ class WindowIteratorSpec extends RawDataWindowingSpec { ) val slidingWinIterator = new SlidingWindowIterator(rv.rows, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = false).asSliding, queryConfig) slidingWinIterator.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults val chunkedIt = new ChunkedWindowIteratorD(rv, 50000L, 100000, 700000L, 100000, - RangeFunction(Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, queryConfig, + RangeFunction(MetricsTestData.timeseriesDataset, + Some(RangeFunctionId.MaxOverTime), ColumnType.DoubleColumn, queryConfig, useChunked = true).asChunkedD, queryConfig) chunkedIt.map(r => (r.getLong(0), r.getDouble(1))).filter(!_._2.isNaN).toList shouldEqual windowResults } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index d1ab8464e8..da4e32ad39 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -22,10 +22,13 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesDataset.blockMetaSize, true) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) + protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesDataset.blockMetaSize, true) protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDataset, storeConf) + protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, None, downsampleDataset.blockMetaSize, true) + protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleDataset, storeConf) + override def afterAll(): Unit = { blockStore.releaseBlocks() } @@ -54,6 +57,19 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll RawDataRangeVector(null, part, AllChunkScan, Array(0, 1)) } + def timeValueRvDownsample(tuples: Seq[(Long, Double, Double, Double, Double, Double)], + colIds: Array[Int]): RawDataRangeVector = { + val part = TimeSeriesPartitionSpec.makePart(0, downsampleDataset, bufferPool = tsBufferPool2) + val readers = tuples.map { case (ts, d1, d2, d3, d4, d5) => + TupleRowReader((Some(ts), Some(d1), Some(d2), Some(d3), Some(d4), Some(d5))) + } + readers.foreach { row => part.ingest(row, ingestBlockHolder2) } + // Now flush and ingest the rest to ensure two separate chunks + part.switchBuffers(ingestBlockHolder2, encode = true) + // part.encodeAndReleaseBuffers(ingestBlockHolder) + RawDataRangeVector(null, part, AllChunkScan, colIds) + } + def timeValueRV(data: Seq[Double], startTS: Long = defaultStartTS): RawDataRangeVector = { val tuples = data.zipWithIndex.map { case (d, t) => (startTS + t * pubFreq, d) } timeValueRV(tuples) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala index b7ac115508..33d6ddd5eb 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/BinaryOperatorSpec.scala @@ -214,7 +214,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val binaryOpMapper = exec.ScalarOperationMapper(BinaryOperator.ADD, scalar, true) - val resultObs = binaryOpMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + val resultObs = binaryOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -229,7 +230,8 @@ class BinaryOperatorSpec extends FunSpec with Matchers with ScalaFutures { private def applyBinaryOperationAndAssertResult(samples: Array[RangeVector], expectedVal: Array[Iterator[Double]], binOp: BinaryOperator, scalar: Double, scalarOnLhs: Boolean): Unit = { val scalarOpMapper = exec.ScalarOperationMapper(binOp, scalar, scalarOnLhs) - val resultObs = scalarOpMapper(Observable.fromIterable(samples), queryConfig, 1000, resultSchema) + val resultObs = scalarOpMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(samples), queryConfig, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index b11a4a51f2..9ab285ff41 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -161,7 +161,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // sort_desc the[UnsupportedOperationException] thrownBy { val miscellaneousVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.SortDesc) - miscellaneousVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + miscellaneousVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "SortDesc not supported." } @@ -169,58 +170,65 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // clamp_max the[IllegalArgumentException] thrownBy { val instantVectorFnMapper1 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMax) - instantVectorFnMapper1(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper1(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Cannot use ClampMax without providing a upper limit of max." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper2 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMax, Seq("hi")) - instantVectorFnMapper2(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper2(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Cannot use ClampMax without providing a upper limit of max as a Number." // clamp_min the[IllegalArgumentException] thrownBy { val instantVectorFnMapper3 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMin) - instantVectorFnMapper3(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper3(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Cannot use ClampMin without providing a lower limit of min." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper4 = exec.InstantVectorFunctionMapper(InstantFunctionId.ClampMin, Seq("hi")) - instantVectorFnMapper4(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper4(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Cannot use ClampMin without providing a lower limit of min as a Number." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Sqrt, Seq(1)) - instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper5(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: No additional parameters required for the instant function." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Round, Seq("hi")) - instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper5(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: to_nearest optional parameter should be a Number." the[IllegalArgumentException] thrownBy { val instantVectorFnMapper5 = exec.InstantVectorFunctionMapper(InstantFunctionId.Round, Seq(1, 2)) - instantVectorFnMapper5(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + instantVectorFnMapper5(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) } should have message "requirement failed: Only one optional parameters allowed for Round." // histogram quantile the[IllegalArgumentException] thrownBy { val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile) - ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + ivMapper(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) } should have message "requirement failed: Quantile (between 0 and 1) required for histogram quantile" the[IllegalArgumentException] thrownBy { val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramQuantile, Seq("b012")) - ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + ivMapper(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) } should have message "requirement failed: histogram_quantile parameter must be a number" // histogram bucket the[IllegalArgumentException] thrownBy { val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket) - ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + ivMapper(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) } should have message "requirement failed: Bucket/le required for histogram bucket" the[IllegalArgumentException] thrownBy { val ivMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.HistogramBucket, Seq("b012")) - ivMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) + ivMapper(MetricsTestData.timeseriesDataset, Observable.fromIterable(sampleBase), queryConfig, 1000, histSchema) } should have message "requirement failed: histogram_bucket parameter must be a number" } @@ -228,7 +236,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { // ceil val expectedVal = sampleBase.map(_.rows.map(v => scala.math.floor(v.getDouble(1)))) val instantVectorFnMapper = exec.InstantVectorFunctionMapper(InstantFunctionId.Ceil) - val resultObs = instantVectorFnMapper(Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) + val resultObs = instantVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { @@ -291,7 +300,8 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { instantFunctionId: InstantFunctionId, funcParams: Seq[Any] = Nil, schema: ResultSchema = resultSchema): Unit = { val instantVectorFnMapper = exec.InstantVectorFunctionMapper(instantFunctionId, funcParams) - val resultObs = instantVectorFnMapper(Observable.fromIterable(samples), queryConfig, 1000, schema) + val resultObs = instantVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(samples), queryConfig, 1000, schema) val result = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) expectedVal.zip(result).foreach { case (ex, res) => { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala index 274375b387..969350c562 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LabelReplaceSpec.scala @@ -74,7 +74,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1 new Label Value $2", "instance", "(.*):90(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -120,7 +121,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instanceNew", "$1-$1", "instance", "(.*)\\d") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -161,7 +163,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("instance", "$1", "instance", "(.*)9") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(sampleWithKey), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -184,13 +187,15 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "Invalid Regular Expression for label_replace" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("instance", "$1")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "requirement failed: " + "Cannot use LabelReplace without function parameters: " + "instant-vector, dst_label string, replacement string, src_label string, regex string" @@ -198,7 +203,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, Seq("$instance", "$1", "instance", "(.*)9(")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "requirement failed: Invalid destination label name" } @@ -212,7 +218,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -238,7 +245,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "destination-value-$1", "src", "value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -264,7 +272,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "$1-value-$2 $3$67", "src", "(.*)-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -290,7 +299,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", "source-value-(.*)") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -316,7 +326,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "nonexistent-src", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -342,7 +353,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "value-$1", "src", "dummy-regex") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -366,7 +378,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "dst", ".*") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -390,7 +403,8 @@ class LabelReplaceSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("src", "", "", "") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelReplace, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala index 74e728ffdc..8489bb9dff 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/LableJoinSpec.scala @@ -101,7 +101,8 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -132,7 +133,8 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src3", "src1") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -162,7 +164,8 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "", "emptysrc", "emptysrc1", "emptysrc2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -193,7 +196,8 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-", "src", "src1", "src2") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -223,7 +227,8 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { val funcParams = Seq("dst", "-") val labelVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams) - val resultObs = labelVectorFnMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultObs = labelVectorFnMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) val resultLabelValues = resultObs.toListL.runAsync.futureValue.map(_.key.labelValues) val resultRows = resultObs.toListL.runAsync.futureValue.map(_.rows.map(_.getDouble(1))) @@ -248,19 +253,22 @@ class LableJoinSpec extends FunSpec with Matchers with ScalaFutures { the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams1) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "requirement failed: Invalid source label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, funcParams2) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "requirement failed: Invalid destination label name in label_join()" the[IllegalArgumentException] thrownBy { val miscellaneousFunctionMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.LabelJoin, Seq("dst")) - miscellaneousFunctionMapper(Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + miscellaneousFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) } should have message "requirement failed: expected at least 3 argument(s) in call to label_join" } } From 4ea611fb51ce96299b5fed9240e42819a28d0a9d Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Tue, 6 Aug 2019 13:23:42 -0700 Subject: [PATCH 09/28] feat(query, coordinator): routing queries based on failure time ranges across pods/cluster (#448) Route queries to remote pod/cluster by creating PromQlExec plan when local failure is present for query time range and merge remote query results with local --- cli/src/main/scala/filodb.cli/CliMain.scala | 19 +- .../scala/filodb.coordinator/QueryActor.scala | 9 +- .../client/QueryCommands.scala | 54 +-- .../filodb.coordinator/client/QueryOps.scala | 7 +- .../client/Serializer.scala | 3 +- .../queryengine/Utils.scala | 4 +- .../queryengine2/FailureProvider.scala | 47 +++ .../queryengine2/QueryEngine.scala | 103 +++++- .../queryengine2/RoutingPlanner.scala | 237 +++++++++++++ .../ClusterRecoverySpec.scala | 5 +- .../NodeCoordinatorActorSpec.scala | 38 ++- .../client/SerializationSpec.scala | 23 +- .../queryengine2/FailureProviderSpec.scala | 203 +++++++++++ .../queryengine2/QueryEngineSpec.scala | 316 +++++++++++++++++- .../scala/filodb.core/SpreadProvider.scala | 9 + .../filodb/http/PrometheusApiRoute.scala | 40 +-- .../filodb/http/PrometheusApiRouteSpec.scala | 5 +- .../downsample/GaugeDownsampleValidator.scala | 4 +- .../filodb.jmh/HistogramQueryBenchmark.scala | 16 +- .../filodb.jmh/QueryAndIngestBenchmark.scala | 7 +- .../QueryHiCardInMemoryBenchmark.scala | 8 +- .../filodb.jmh/QueryInMemoryBenchmark.scala | 20 +- .../filodb.jmh/QueryOnDemandBenchmark.scala | 8 +- project/FiloBuild.scala | 4 + .../prometheus/query/PrometheusModel.scala | 24 +- .../scala/filodb/query/PromCirceSupport.scala | 20 ++ .../filodb/query/PromQlInvocationParams.scala | 5 + .../filodb/query/PromQueryResponse.scala | 22 ++ .../main/scala/filodb/query/QueryConfig.scala | 4 +- .../scala/filodb/query/QueryOptions.scala | 47 +++ .../main/scala/filodb/query/ResultTypes.scala | 4 +- .../scala/filodb/query/exec/ExecPlan.scala | 2 +- .../query/exec/InProcessPlanDispatcher.scala | 59 ++++ .../filodb/query/exec/PlanDispatcher.scala | 6 +- .../scala/filodb/query/exec/PromQlExec.scala | 131 ++++++++ .../filodb/query/exec/StitchRvsExec.scala | 2 +- .../query/exec/BinaryJoinExecSpec.scala | 4 +- .../query/exec/BinaryJoinGroupingSpec.scala | 5 +- .../exec/BinaryJoinSetOperatorSpec.scala | 6 +- .../exec/InProcessPlanDispatcherSpec.scala | 122 +++++++ .../filodb/query/exec/MetadataExecSpec.scala | 4 +- .../filodb/query/exec/PromQlExecSpec.scala | 41 +++ .../exec/SelectRawPartitionsExecSpec.scala | 6 +- .../standalone/StandaloneMultiJvmSpec.scala | 12 +- 44 files changed, 1484 insertions(+), 231 deletions(-) create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryengine2/FailureProvider.scala create mode 100644 coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala create mode 100644 coordinator/src/test/scala/filodb.coordinator/queryengine2/FailureProviderSpec.scala create mode 100644 core/src/main/scala/filodb.core/SpreadProvider.scala create mode 100644 query/src/main/scala/filodb/query/PromCirceSupport.scala create mode 100644 query/src/main/scala/filodb/query/PromQlInvocationParams.scala create mode 100644 query/src/main/scala/filodb/query/PromQueryResponse.scala create mode 100644 query/src/main/scala/filodb/query/QueryOptions.scala create mode 100644 query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala create mode 100644 query/src/main/scala/filodb/query/exec/PromQlExec.scala create mode 100644 query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala create mode 100644 query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala diff --git a/cli/src/main/scala/filodb.cli/CliMain.scala b/cli/src/main/scala/filodb.cli/CliMain.scala index 08fa757d7c..d1cb249102 100644 --- a/cli/src/main/scala/filodb.cli/CliMain.scala +++ b/cli/src/main/scala/filodb.cli/CliMain.scala @@ -12,7 +12,8 @@ import monix.reactive.Observable import filodb.coordinator._ import filodb.coordinator.client._ -import filodb.coordinator.client.QueryCommands.{SpreadChange, SpreadProvider, StaticSpreadProvider} +import filodb.coordinator.client.QueryCommands.StaticSpreadProvider +import filodb.coordinator.queryengine2.{PromQlQueryParams, TsdbQueryParams, UnavailablePromQlQueryParams} import filodb.core._ import filodb.core.metadata.Column import filodb.memory.format.RowReader @@ -211,27 +212,29 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { timeParams: TimeRangeParams, options: QOptions): Unit = { val logicalPlan = Parser.metadataQueryToLogicalPlan(query, timeParams) - executeQuery2(client, dataset, logicalPlan, options) + executeQuery2(client, dataset, logicalPlan, options, UnavailablePromQlQueryParams) } def parseLabelValuesQuery(client: LocalClient, labelNames: Seq[String], constraints: Map[String, String], dataset: String, timeParams: TimeRangeParams, options: QOptions): Unit = { val logicalPlan = LabelValues(labelNames, constraints, 3.days.toMillis) - executeQuery2(client, dataset, logicalPlan, options) + executeQuery2(client, dataset, logicalPlan, options, UnavailablePromQlQueryParams) } def parsePromQuery2(client: LocalClient, query: String, dataset: String, timeParams: TimeRangeParams, options: QOptions): Unit = { val logicalPlan = Parser.queryRangeToLogicalPlan(query, timeParams) - executeQuery2(client, dataset, logicalPlan, options) + executeQuery2(client, dataset, logicalPlan, options, PromQlQueryParams(query,timeParams.start, timeParams.step, + timeParams.end)) } - def executeQuery2(client: LocalClient, dataset: String, plan: LogicalPlan, options: QOptions): Unit = { + def executeQuery2(client: LocalClient, dataset: String, plan: LogicalPlan, options: QOptions, tsdbQueryParams: + TsdbQueryParams): Unit = { val ref = DatasetRef(dataset) val spreadProvider: Option[SpreadProvider] = options.spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) - val qOpts = QueryCommands.QueryOptions(spreadProvider, options.sampleLimit) + val qOpts = QueryOptions(spreadProvider, options.sampleLimit) .copy(queryTimeoutSecs = options.timeout.toSeconds.toInt, shardOverrides = options.shardOverrides) println(s"Sending query command to server for $ref with options $qOpts...") @@ -239,7 +242,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { options.everyN match { case Some(intervalSecs) => val fut = Observable.intervalAtFixedRate(intervalSecs.seconds).foreach { n => - client.logicalPlan2Query(ref, plan, qOpts) match { + client.logicalPlan2Query(ref, plan, tsdbQueryParams, qOpts) match { case QueryResult(_, schema, result) => result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case err: QueryError => throw new ClientException(err) } @@ -250,7 +253,7 @@ object CliMain extends ArgMain[Arguments] with FilodbClusterNode { while (!fut.isCompleted) { Thread sleep 1000 } case None => try { - client.logicalPlan2Query(ref, plan, qOpts) match { + client.logicalPlan2Query(ref, plan, tsdbQueryParams, qOpts) match { case QueryResult(_, schema, result) => println(s"Number of Range Vectors: ${result.size}") result.take(options.limit).foreach(rv => println(rv.prettyPrint())) case QueryError(_,ex) => println(s"QueryError: ${ex.getClass.getSimpleName} ${ex.getMessage}") diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index aeedf03e4d..d5cf1e848f 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -11,7 +11,7 @@ import net.ceedubs.ficus.Ficus._ import net.ceedubs.ficus.readers.ValueReader import scala.util.control.NonFatal -import filodb.coordinator.queryengine2.QueryEngine +import filodb.coordinator.queryengine2.{EmptyFailureProvider, QueryEngine} import filodb.core._ import filodb.core.memstore.{FiloSchedulers, MemStore, TermInfo} import filodb.core.metadata.Dataset @@ -77,7 +77,8 @@ final class QueryActor(memStore: MemStore, }.getOrElse { x: Seq[ColumnFilter] => Seq(SpreadChange(defaultSpread)) } val functionalSpreadProvider = FunctionalSpreadProvider(spreadFunc) - val queryEngine2 = new QueryEngine(dataset, shardMapFunc) + val queryEngine2 = new QueryEngine(dataset, shardMapFunc, + EmptyFailureProvider, functionalSpreadProvider) val queryConfig = new QueryConfig(config.getConfig("filodb.query")) val numSchedThreads = Math.ceil(config.getDouble("filodb.query.threads-factor") * sys.runtime.availableProcessors) val queryScheduler = Scheduler.fixedPool(s"$QuerySchedName-${dataset.ref}", numSchedThreads.toInt) @@ -121,7 +122,7 @@ final class QueryActor(memStore: MemStore, // This is for CLI use only. Always prefer clients to materialize logical plan lpRequests.increment try { - val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, getSpreadProvider(q.queryOptions)) + val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, q.tsdbQueryParams) self forward execPlan } catch { case NonFatal(ex) => @@ -133,7 +134,7 @@ final class QueryActor(memStore: MemStore, private def processExplainPlanQuery(q: ExplainPlan2Query, replyTo: ActorRef) = { try { - val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, getSpreadProvider(q.queryOptions)) + val execPlan = queryEngine2.materialize(q.logicalPlan, q.queryOptions, q.tsdbQueryParams) replyTo ! execPlan } catch { case NonFatal(ex) => diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala index 0df6600d1c..4a676f8361 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryCommands.scala @@ -1,7 +1,8 @@ package filodb.coordinator.client -import filodb.core.query.{ColumnFilter, Filter} -import filodb.query.{LogicalPlan => LogicalPlan2, QueryCommand} +import filodb.coordinator.queryengine2.TsdbQueryParams +import filodb.core.query.ColumnFilter +import filodb.query.{LogicalPlan => LogicalPlan2, QueryCommand, QueryOptions} object QueryCommands { import filodb.core._ @@ -33,11 +34,8 @@ object QueryCommands { submitTime: Long = System.currentTimeMillis()) extends QueryCommand - final case class SpreadChange(time: Long = 0L, spread: Int = 1) - trait SpreadProvider { - def spreadFunc(filter: Seq[ColumnFilter]): Seq[SpreadChange] - } + final case class StaticSpreadProvider(spreadChange: SpreadChange = SpreadChange()) extends SpreadProvider { def spreadFunc(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { @@ -57,48 +55,6 @@ object QueryCommands { } } - /** - * This class provides general query processing parameters - * @param spreadFunc a function that returns chronologically ordered spread changes for the filter - */ - final case class QueryOptions(spreadProvider: Option[SpreadProvider] = None, - parallelism: Int = 16, - queryTimeoutSecs: Int = 30, - sampleLimit: Int = 1000000, - shardOverrides: Option[Seq[Int]] = None) - - object QueryOptions { - def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): QueryOptions = - QueryOptions(spreadProvider = constSpread, sampleLimit = sampleLimit) - - /** - * Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values - * present in the spreadMap to specific spread values, with a default if the filter/value not present in the map - */ - def simpleMapSpreadFunc(keyName: String, - spreadMap: collection.mutable.Map[collection.Map[String, String], Int], - defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = { - filters: Seq[ColumnFilter] => - filters.collectFirst { - case ColumnFilter(key, Filter.Equals(filtVal: String)) if key == keyName => filtVal - }.map { tagValue => - Seq(SpreadChange(spread = spreadMap.getOrElse(collection.mutable.Map(keyName->tagValue), defaultSpread))) - }.getOrElse(Seq(SpreadChange(defaultSpread))) - } - - import collection.JavaConverters._ - - def simpleMapSpreadFunc(keyName: String, - spreadMap: java.util.Map[java.util.Map[String, String], Integer], - defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = { - val spreadAssignment: collection.mutable.Map[collection.Map[String, String], Int]= spreadMap.asScala.map { - case (d, v) => d.asScala -> v.toInt - } - - simpleMapSpreadFunc(keyName, spreadAssignment, defaultSpread) - } - } - /** * Executes a query using a LogicalPlan and returns the result as one message to the client. * Depends on queryOptions, the query will fan out to multiple nodes and shards as needed to gather @@ -110,11 +66,13 @@ object QueryCommands { */ final case class LogicalPlan2Query(dataset: DatasetRef, logicalPlan: LogicalPlan2, + tsdbQueryParams: TsdbQueryParams, queryOptions: QueryOptions = QueryOptions(), submitTime: Long = System.currentTimeMillis()) extends QueryCommand final case class ExplainPlan2Query(dataset: DatasetRef, logicalPlan: LogicalPlan2, + tsdbQueryParams: TsdbQueryParams, queryOptions: QueryOptions = QueryOptions(), submitTime: Long = System.currentTimeMillis()) extends QueryCommand // Error responses from query diff --git a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala index 1d62830ad9..a90c808516 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/QueryOps.scala @@ -4,9 +4,9 @@ import scala.concurrent.duration._ import com.typesafe.scalalogging.StrictLogging +import filodb.coordinator.queryengine2.TsdbQueryParams import filodb.core._ -import filodb.query.{LogicalPlan => LogicalPlan2} -import filodb.query.{QueryResponse => QueryResponse2} +import filodb.query.{LogicalPlan => LogicalPlan2, QueryOptions, QueryResponse => QueryResponse2} trait QueryOps extends ClientBase with StrictLogging { import QueryCommands._ @@ -53,8 +53,9 @@ trait QueryOps extends ClientBase with StrictLogging { */ def logicalPlan2Query(dataset: DatasetRef, plan: LogicalPlan2, + tsdbQueryParams: TsdbQueryParams, options: QueryOptions = QueryOptions()): QueryResponse2 = { - val qCmd = LogicalPlan2Query(dataset, plan, options) + val qCmd = LogicalPlan2Query(dataset, plan, tsdbQueryParams, options) // NOTE: It's very important to extend the query timeout for the ask itself, because the queryTimeoutSecs is // the internal FiloDB scatter-gather timeout. We need additional time for the proper error to get transmitted // back in case of internal timeouts. diff --git a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala index 530bf4e7c0..74647844f3 100644 --- a/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala +++ b/coordinator/src/main/scala/filodb.coordinator/client/Serializer.scala @@ -10,6 +10,7 @@ import filodb.core.binaryrecord2.{RecordSchema => RecordSchema2} import filodb.core.metadata.Column import filodb.core.query.ColumnInfo import filodb.memory.format.ZeroCopyUTF8String +import filodb.query.QueryOptions /** * Register commonly used classes for efficient Kryo serialization. If this is not done then Kryo might have to @@ -91,7 +92,7 @@ class KryoInit { kryo.register(classOf[ShardSplit]) kryo.register(classOf[QueryCommands.BadQuery]) - kryo.register(classOf[QueryCommands.QueryOptions]) + kryo.register(classOf[QueryOptions]) kryo.register(classOf[QueryCommands.FilteredPartitionQuery]) } } diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala index 5687cde7e3..214aeb255b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine/Utils.scala @@ -12,11 +12,12 @@ import monix.reactive.Observable import org.scalactic._ import filodb.coordinator.ShardMapper -import filodb.core.{ErrorResponse, Types} +import filodb.core.{ErrorResponse, SpreadProvider, Types} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} import filodb.core.store._ +import filodb.query.QueryOptions final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) extends Exception(s"From [$source] - $resp") @@ -26,7 +27,6 @@ final case class ChildErrorResponse(source: ActorRef, resp: ErrorResponse) exten * Logical -> Physical Plan conversion and implementing the Distribute* physical primitives */ object Utils extends StrictLogging { - import filodb.coordinator.client.QueryCommands._ import TrySugar._ import filodb.coordinator.client.QueryCommands._ diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/FailureProvider.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/FailureProvider.scala new file mode 100644 index 0000000000..fbdaf4efab --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/FailureProvider.scala @@ -0,0 +1,47 @@ +package filodb.coordinator.queryengine2 + +import filodb.core.DatasetRef + +/** + * A provider to get failure ranges. Query engine can use failure ranges while preparing physical + * plan to reroute or skip failure ranges. Ranges are based on dataset and over all clusters. + * Provider will filter failure ranges by current cluster and its replicas. Failures which do not + * belong to current cluster or its replica, will be skipped. + */ +trait FailureProvider { + def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] +} + +object EmptyFailureProvider extends FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = Nil +} + +/** + * Time range. + * + * @param startInMillis epoch time in millis. + * @param endInMillis epoch time in millis. + */ +case class TimeRange(startInMillis: Long, endInMillis: Long) + +/** + * Failure details. + * + * @param clusterName cluster name. + * @param datasetRef Dataset reference for database and dataset. + * @param timeRange time range. + */ +case class FailureTimeRange(clusterName: String, datasetRef: DatasetRef, timeRange: TimeRange, + isRemote: Boolean) + +/** + * For rerouting queries for failure ranges, Route trait will offer more context in the form of corrective + * ranges for queries or alternative dispatchers. + * A local route indicates a non-failure range on local cluster. A remote route indicates a non-failure + * range on remote cluster. + */ +trait Route + +case class LocalRoute(timeRange: Option[TimeRange]) extends Route + +case class RemoteRoute(timeRange: Option[TimeRange]) extends Route \ No newline at end of file diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 0b7ac6255a..567ade399b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -3,17 +3,18 @@ package filodb.coordinator.queryengine2 import java.util.UUID import java.util.concurrent.ThreadLocalRandom -import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration import akka.actor.ActorRef +import com.typesafe.config.{Config, ConfigFactory} import com.typesafe.scalalogging.StrictLogging import kamon.Kamon import monix.eval.Task +import monix.execution.Scheduler import filodb.coordinator.ShardMapper -import filodb.coordinator.client.QueryCommands.{QueryOptions, SpreadProvider, StaticSpreadProvider} -import filodb.core.Types +import filodb.coordinator.client.QueryCommands.StaticSpreadProvider +import filodb.core.{SpreadProvider, Types} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.metadata.Dataset import filodb.core.query.{ColumnFilter, Filter} @@ -22,13 +23,23 @@ import filodb.prometheus.ast.Vectors.PromMetricLabel import filodb.query.{exec, _} import filodb.query.exec._ + +trait TsdbQueryParams +case class PromQlQueryParams(promQl: String, start: Long, step: Long, end: Long, + spread: Option[Int] = None, processFailure: Boolean = true) extends TsdbQueryParams + +object UnavailablePromQlQueryParams extends TsdbQueryParams + /** * FiloDB Query Engine is the facade for execution of FiloDB queries. * It is meant for use inside FiloDB nodes to execute materialized * ExecPlans as well as from the client to execute LogicalPlans. */ class QueryEngine(dataset: Dataset, - shardMapperFunc: => ShardMapper) + shardMapperFunc: => ShardMapper, + failureProvider: FailureProvider, + spreadProvider: SpreadProvider = StaticSpreadProvider(), + queryEngineConfig: Config = ConfigFactory.empty()) extends StrictLogging { /** @@ -46,7 +57,7 @@ class QueryEngine(dataset: Dataset, * It sends the ExecPlan to the destination where it will be executed. */ def dispatchExecPlan(execPlan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = { val currentSpan = Kamon.currentSpan() Kamon.withSpan(currentSpan) { @@ -54,14 +65,89 @@ class QueryEngine(dataset: Dataset, } } + /** + * Converts Routes to ExecPlan + */ + private def routeExecPlanMapper(routes: Seq[Route], rootLogicalPlan: LogicalPlan, queryId: String, submitTime: Long, + options: QueryOptions, spreadProvider: SpreadProvider, lookBackTime: Long, + tsdbQueryParams: TsdbQueryParams): ExecPlan = { + + val execPlans : Seq[ExecPlan]= routes.map { route => + route match { + case route: LocalRoute => if (!route.timeRange.isDefined) + generateLocalExecPlan(rootLogicalPlan, queryId, submitTime, options, spreadProvider) + else + generateLocalExecPlan(QueryRoutingPlanner.copyWithUpdatedTimeRange(rootLogicalPlan, + route.asInstanceOf[LocalRoute].timeRange.get, lookBackTime), queryId, submitTime, options, spreadProvider) + case route: RemoteRoute => + val timeRange = route.timeRange.get + val queryParams = tsdbQueryParams.asInstanceOf[PromQlQueryParams] + val endpoint = queryEngineConfig.isEmpty() match { + case false => queryEngineConfig.getString("routing.buddy.http.endpoint") + case _ => "" + } + + val promQlInvocationParams = PromQlInvocationParams(endpoint, queryParams.promQl, (timeRange.startInMillis + /1000), queryParams.step, (timeRange.endInMillis / 1000), queryParams.spread, false) + logger.debug("PromQlExec params:" + promQlInvocationParams) + PromQlExec(queryId, InProcessPlanDispatcher(dataset), dataset.ref, promQlInvocationParams, submitTime) + } + } + + if (execPlans.size == 1) + execPlans.head + else + // Stitch RemoteExec plan results with local using InProcessorDispatcher + // Sort to move RemoteExec in end as it does not have schema + StitchRvsExec(queryId, InProcessPlanDispatcher(dataset), + execPlans.sortWith((x, y) => !x.isInstanceOf[PromQlExec])) + } + /** * Converts a LogicalPlan to the ExecPlan */ def materialize(rootLogicalPlan: LogicalPlan, - options: QueryOptions, spreadProvider: SpreadProvider = StaticSpreadProvider()): ExecPlan = { + options: QueryOptions, + tsdbQueryParams: TsdbQueryParams): ExecPlan = { val queryId = UUID.randomUUID().toString + val submitTime = System.currentTimeMillis() + val querySpreadProvider = options.spreadProvider.getOrElse(spreadProvider) + + if (!QueryRoutingPlanner.isPeriodicSeriesPlan(rootLogicalPlan) || // It is a raw data query + !tsdbQueryParams.isInstanceOf[PromQlQueryParams] || // We don't know the promql issued (unusual) + (tsdbQueryParams.isInstanceOf[PromQlQueryParams] && + !tsdbQueryParams.asInstanceOf[PromQlQueryParams].processFailure) || // This is a query that was part of + // failure routing + !QueryRoutingPlanner.hasSingleTimeRange(rootLogicalPlan)) // Sub queries have different time ranges (unusual) + { + generateLocalExecPlan(rootLogicalPlan, queryId, submitTime, options, querySpreadProvider) + } else { + val periodicSeriesTime = QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(logicalPlan = rootLogicalPlan) + val lookBackTime: Long = QueryRoutingPlanner.getRawSeriesStartTime(rootLogicalPlan). + map(periodicSeriesTime.startInMillis - _).get + + val routingTime = TimeRange(periodicSeriesTime.startInMillis - lookBackTime, periodicSeriesTime.endInMillis) + val failures = + failureProvider.getFailures(dataset.ref, routingTime) + .sortWith(_.timeRange.startInMillis < _.timeRange.startInMillis) + + if (failures.isEmpty) { + generateLocalExecPlan(rootLogicalPlan, queryId, submitTime, options, querySpreadProvider) + } else { + val routes = QueryRoutingPlanner.plan(failures, periodicSeriesTime, lookBackTime, + tsdbQueryParams.asInstanceOf[PromQlQueryParams].step * 1000) + logger.debug("Routes:" + routes) + routeExecPlanMapper(routes, rootLogicalPlan, queryId, submitTime, options, querySpreadProvider, lookBackTime, + tsdbQueryParams) + } + } + } - val materialized = walkLogicalPlanTree(rootLogicalPlan, queryId, System.currentTimeMillis(), + private def generateLocalExecPlan(logicalPlan: LogicalPlan, + queryId: String, + submitTime: Long, + options: QueryOptions, spreadProvider: SpreadProvider) = { + val materialized = walkLogicalPlanTree(logicalPlan, queryId, System.currentTimeMillis(), options, spreadProvider) match { case PlanResult(Seq(justOne), stitch) => @@ -79,8 +165,9 @@ class QueryEngine(dataset: Dataset, } } logger.debug(s"Materialized logical plan for dataset=${dataset.ref}:" + - s" $rootLogicalPlan to \n${materialized.printTree()}") + s" $logicalPlan to \n${materialized.printTree()}") materialized + } val shardColumns = dataset.options.shardKeyColumns.sorted diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala new file mode 100644 index 0000000000..295b1ef5e8 --- /dev/null +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala @@ -0,0 +1,237 @@ +package filodb.coordinator.queryengine2 + +import com.typesafe.scalalogging.StrictLogging + +import filodb.query._ + +/** + * Planner for routing based on failure ranges for a given LogicalPlan. + */ +trait RoutingPlanner extends StrictLogging { + def plan(failure: Seq[FailureTimeRange], time: TimeRange, lookbackTime: Long, step: Long): Seq[Route] +} + +object QueryRoutingPlanner extends RoutingPlanner { + + /** + * Remove smaller FailureTimeRange when more than one FailureTimeRanges have overlapping times + */ + private def removeSmallerOverlappingFailures(failures: Seq[FailureTimeRange]): Seq[FailureTimeRange] = { + + failures.sortWith(_.timeRange.startInMillis < _.timeRange.startInMillis). + foldLeft(Seq[FailureTimeRange]()) { (buildList, tail) => + buildList match { + case Nil => Seq(tail) + case head :+ value => + if (value.timeRange.endInMillis >= tail.timeRange.startInMillis) { + // Remove larger overlapping interval + if ((value.timeRange.endInMillis - value.timeRange.startInMillis) < + (tail.timeRange.endInMillis - tail.timeRange.startInMillis)) { + buildList.dropRight(1) :+ tail + } + else { + buildList + } + + } else { + buildList :+ tail + } + } + } + } + + /** + * + * @param failures seq of FailureTimeRanges + * @param time query start and end time + * @param lookbackTime query lookbackTime + * @param step query step + * @return seq of routes + */ + def plan(failures: Seq[FailureTimeRange], time: TimeRange, lookbackTime: Long, step: Long): Seq[Route] = { + + val nonOverlappingFailures = removeSmallerOverlappingFailures(failures) + if ((nonOverlappingFailures.last.timeRange.endInMillis < time.startInMillis) || + (nonOverlappingFailures.head.timeRange.startInMillis > time.endInMillis)) { + Seq(LocalRoute(None)) // No failure in this time range + } + logger.info("Logical plan time:" + time) + + // Recursively split query into local and remote routes starting from first FailureTimeRange + splitQueryTime(nonOverlappingFailures, 0, time.startInMillis - lookbackTime, time.endInMillis, lookbackTime, + step) + } + + /** + * Recursively generate Local and Remote Routes by splitting query time based on failures + * + * @param failure seq of FailureTimeRanges + * @param index index of FailureTimeRange which has to be processed + * @param start start time for route + * @param end end time for route + * @return seq of Routes + */ + private def splitQueryTime(failure: Seq[FailureTimeRange], index: Int, start: Long, end: Long, + lookbackTime: Long, step: Long): Seq[Route] = { + + if (index >= failure.length) + return Nil + + val startWithLookBack = if (index == 0) + start + lookbackTime + else + start + + if (startWithLookBack > end) + return Nil + // traverse query range time from left to right , break at failure start + var i = index + 1 + + if (!failure(index).isRemote) { + // Handle local failure + // Traverse till we get a remote failure to minimize number of queries + while ((i < failure.length) && (!failure(i).isRemote)) + i = i + 1 + // need further splitting + if (i < failure.length) { + val lastSampleTime = getLastSampleTimeWithStep(startWithLookBack, + failure(i).timeRange.startInMillis, step) + // Query from current start time till next remote failure starts should be executed remotely + // Routes should have Periodic series time so add lookbackTime + RemoteRoute(Some(TimeRange(startWithLookBack, lastSampleTime))) +: + splitQueryTime(failure, i, lastSampleTime + step, end, + lookbackTime, step) // Process remaining query + } else { + // Last failure so no further splitting required + Seq(RemoteRoute(Some(TimeRange(startWithLookBack , end)))) + } + + } else { + // Iterate till we get a local failure + while ((i < failure.length) && (failure(i).isRemote)) + i = i + 1 + if (i < failure.length) { + val lastSampleTime = getLastSampleTimeWithStep(startWithLookBack, + failure(i).timeRange.startInMillis, step ) + // Query from current start time till next local failure starts should be executed locally + LocalRoute(Some(TimeRange(startWithLookBack, lastSampleTime))) +: + splitQueryTime(failure, i, lastSampleTime + step, end, lookbackTime, step) + } + else { + Seq(LocalRoute(Some(TimeRange(startWithLookBack, end)))) + } + } + } + + private def getLastSampleTimeWithStep( start: Long, end: Long, step: Long): Long = { + var ctr = 0 + var currentTime = start + var nextTime = start + while (nextTime < end) { + ctr = ctr + 1 + currentTime = nextTime + nextTime = start + (step * ctr) + } + currentTime + } + + /** + * Check whether logical plan has a PeriodicSeriesPlan + */ + def isPeriodicSeriesPlan(logicalPlan: LogicalPlan): Boolean = { + if (logicalPlan.isInstanceOf[RawSeriesPlan] || logicalPlan.isInstanceOf[MetadataQueryPlan]) { + false + } else + true + } + + /** + * Check whether all child logical plans have same start and end time + */ + def hasSingleTimeRange(logicalPlan: LogicalPlan): Boolean = { + if (logicalPlan.isInstanceOf[BinaryJoin]) { + val binaryJoin = logicalPlan.asInstanceOf[BinaryJoin] + val lhsTime = getPeriodicSeriesTimeFromLogicalPlan(binaryJoin.lhs) + val rhsTime = getPeriodicSeriesTimeFromLogicalPlan(binaryJoin.rhs) + ((lhsTime.startInMillis == rhsTime.startInMillis) && (lhsTime.endInMillis == rhsTime.endInMillis)) + } else + true + } + + /** + * Retrieve start and end time from LogicalPlan + * NOTE: Plan should be PeriodicSeriesPlan + */ + def getPeriodicSeriesTimeFromLogicalPlan(logicalPlan: LogicalPlan): TimeRange = { + logicalPlan match { + case lp: PeriodicSeries => TimeRange(lp.start, lp.end) + case lp: PeriodicSeriesWithWindowing => TimeRange(lp.start, lp.end) + case lp: ApplyInstantFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + case lp: Aggregate => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + case lp: BinaryJoin => getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) // can assume lhs & rhs have same time + case lp: ScalarVectorBinaryOperation => getPeriodicSeriesTimeFromLogicalPlan(lp.vector) + case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + case _ => throw new BadQueryException("Invalid logical plan") + } + } + + /** + * Used to change start and end time(TimeRange) of LogicalPlan + * NOTE: Plan should be PeriodicSeriesPlan + */ + def copyWithUpdatedTimeRange(logicalPlan: LogicalPlan, timeRange: TimeRange, + lookBackTime: Long): PeriodicSeriesPlan = { + logicalPlan match { + case lp: PeriodicSeries => lp.copy(start = timeRange.startInMillis, end = timeRange.endInMillis, + rawSeries = copyRawSeriesWithUpdatedTimeRange(lp.rawSeries, timeRange, lookBackTime)) + case lp: PeriodicSeriesWithWindowing => lp.copy(start = timeRange.startInMillis, end = + timeRange.endInMillis, rawSeries = copyRawSeriesWithUpdatedTimeRange(lp.rawSeries, timeRange, lookBackTime)) + case lp: ApplyInstantFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + case lp: Aggregate => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + case lp: BinaryJoin => lp.copy(lhs = copyWithUpdatedTimeRange(lp.lhs, timeRange, lookBackTime), rhs = + copyWithUpdatedTimeRange(lp.rhs, timeRange, lookBackTime)) + case lp: ScalarVectorBinaryOperation => lp.copy(vector = copyWithUpdatedTimeRange(lp.vector, timeRange, + lookBackTime)) + case lp: ApplyMiscellaneousFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, + lookBackTime)) + case _ => throw new UnsupportedOperationException("Logical plan not supportred for copy") + } + } + + /** + * Used to change rangeSelector of RawSeriesPlan + */ + def copyRawSeriesWithUpdatedTimeRange(rawSeriesPlan: RawSeriesPlan, timeRange: TimeRange, lookBackTime: Long): + RawSeries = { + + rawSeriesPlan match { + case rs: RawSeries => rs.rangeSelector match { + case is: IntervalSelector => rs.copy(rangeSelector = is.copy(timeRange.startInMillis - lookBackTime, + timeRange.endInMillis)) + case _ => throw new UnsupportedOperationException("Copy supported only for IntervalSelector") + } + case _ => throw new UnsupportedOperationException("Copy supported only for RawSeries") + } + } + + /** + * Retrieve start time of Raw Series + * NOTE: Plan should be PeriodicSeriesPlan + */ + def getRawSeriesStartTime(logicalPlan: LogicalPlan): Option[Long] = { + logicalPlan match { + case lp: RawSeries => lp.rangeSelector match { + case rs: IntervalSelector => Some(rs.from) + case _ => None + } + case lp: PeriodicSeries => getRawSeriesStartTime(lp.rawSeries) + case lp: PeriodicSeriesWithWindowing => getRawSeriesStartTime(lp.rawSeries) + case lp: ApplyInstantFunction => getRawSeriesStartTime(lp.vectors) + case lp: Aggregate => getRawSeriesStartTime(lp.vectors) + case lp: BinaryJoin => getRawSeriesStartTime(lp.lhs) // can assume lhs & rhs have same time + case lp: ScalarVectorBinaryOperation => getRawSeriesStartTime(lp.vector) + case lp: ApplyMiscellaneousFunction => getRawSeriesStartTime(lp.vectors) + case _ => None + } + } +} diff --git a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala index 67c65289e7..61fcdcee89 100644 --- a/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala +++ b/coordinator/src/multi-jvm/scala/filodb.coordinator/ClusterRecoverySpec.scala @@ -2,14 +2,13 @@ package filodb.coordinator import scala.concurrent.Future import scala.concurrent.duration._ - import akka.actor.ActorRef import akka.pattern.ask import akka.remote.testkit.MultiNodeConfig import akka.util.Timeout import com.typesafe.config.ConfigFactory +import filodb.coordinator.queryengine2.UnavailablePromQlQueryParams import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core._ import filodb.core.metadata.Column.ColumnType import filodb.core.query.ColumnInfo @@ -144,7 +143,7 @@ abstract class ClusterRecoverySpec extends ClusterSpec(ClusterRecoverySpecConfig val q2 = LogicalPlan2Query(dataset6.ref, PeriodicSeriesWithWindowing( RawSeries(AllChunksSelector, Nil, Seq("AvgTone")), - 100L, 1000L, 100L, window=1000L, function=RangeFunctionId.CountOverTime), qOpt) + 100L, 1000L, 100L, window = 1000L, function = RangeFunctionId.CountOverTime), UnavailablePromQlQueryParams, qOpt) coordinatorActor ! q2 expectMsgPF(10.seconds.dilated) { case QueryResult(_, schema, vectors) => diff --git a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala index 699c7e7642..4ed9c94eea 100644 --- a/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/NodeCoordinatorActorSpec.scala @@ -3,15 +3,14 @@ package filodb.coordinator import java.net.InetAddress import scala.concurrent.duration._ - import akka.actor.{Actor, ActorRef, AddressFromURIString, PoisonPill, Props} import akka.pattern.gracefulStop import akka.util.Timeout import com.typesafe.config.ConfigFactory +import filodb.coordinator.queryengine2.UnavailablePromQlQueryParams import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core._ import filodb.core.memstore.TimeSeriesMemStore import filodb.core.metadata.{Column, Dataset} @@ -135,7 +134,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew it("should return UnknownDataset if attempting to query before ingestion set up") { val ref = MachineMetricsData.dataset1.ref - val q1 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "Series 1"), Seq("min"))) + val q1 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "Series 1"), + Seq("min")), UnavailablePromQlQueryParams) probe.send(coordinatorActor, q1) probe.expectMsg(UnknownDataset) } @@ -148,7 +148,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew memStore.refreshIndexForTesting(dataset1.ref) // Query existing partition: Series 1 - val q1 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "Series 1"), Seq("min")), qOpt) + val q1 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "Series 1"), + Seq("min")),UnavailablePromQlQueryParams, qOpt) probe.send(coordinatorActor, q1) val info1 = probe.expectMsgPF(3.seconds.dilated) { @@ -159,7 +160,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew } // Query nonexisting partition - val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "NotSeries"), Seq("min")), qOpt) + val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, filters("series" -> "NotSeries"), + Seq("min")), UnavailablePromQlQueryParams, qOpt) probe.send(coordinatorActor, q2) val info2 = probe.expectMsgPF(3.seconds.dilated) { case QueryResult(_, schema, Nil) => @@ -173,7 +175,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val from = to - 50 val qParams = TimeStepParams(from, 10, to) val logPlan = Parser.queryRangeToLogicalPlan("topk(a1b, series_1)", qParams) - val q1 = LogicalPlan2Query(ref, logPlan, qOpt) + val q1 = LogicalPlan2Query(ref, logPlan, UnavailablePromQlQueryParams, qOpt) probe.send(coordinatorActor, q1) probe.expectMsgClass(classOf[QueryError]) } @@ -187,10 +189,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew // Try a filtered partition query 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) + val q2 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Avg, PeriodicSeries(RawSeries(AllChunksSelector, + multiFilter, Seq("min")), 120000L, 10000L, 130000L)), UnavailablePromQlQueryParams, qOpt) memStore.refreshIndexForTesting(dataset1.ref) probe.send(coordinatorActor, q2) probe.expectMsgPF() { @@ -204,7 +204,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val q3 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Avg, PeriodicSeries( - RawSeries(AllChunksSelector, multiFilter, Seq("count")), 120000L, 10000L, 130000L)), qOpt) + RawSeries(AllChunksSelector, multiFilter, Seq("count")), 120000L, 10000L, 130000L)), + UnavailablePromQlQueryParams, qOpt) probe.send(coordinatorActor, q3) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => @@ -217,8 +218,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val q4 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Avg, PeriodicSeries( - RawSeries(AllChunksSelector, filters("series" -> "foobar"), Seq("min")), - 120000L, 10000L, 130000L)), qOpt) + RawSeries(AllChunksSelector, filters("series" -> "foobar"), Seq("min")), 120000L, + 10000L, 130000L)), UnavailablePromQlQueryParams, qOpt) probe.send(coordinatorActor, q4) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => @@ -241,7 +242,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val q2 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Avg, PeriodicSeries( - RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), qOpt) + RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 130000L)), + UnavailablePromQlQueryParams, qOpt) (0 until numQueries).foreach { i => probe.send(coordinatorActor, q2) } (0 until numQueries).foreach { _ => @@ -272,7 +274,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val q2 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Avg, PeriodicSeries( - RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 140000L)), queryOpt) + RawSeries(AllChunksSelector, multiFilter, Seq("min")), 120000L, 10000L, 140000L)), + UnavailablePromQlQueryParams, queryOpt) probe.send(coordinatorActor, q2) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => @@ -296,7 +299,8 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val queryOpt = QueryOptions(shardOverrides = Some(Seq(0, 1))) val series2 = (2 to 4).map(n => s"Series $n") val multiFilter = Seq(ColumnFilter("series", Filter.In(series2.toSet.asInstanceOf[Set[Any]]))) - val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, multiFilter, Seq("min")), queryOpt) + val q2 = LogicalPlan2Query(ref, RawSeries(AllChunksSelector, multiFilter, Seq("min")), + UnavailablePromQlQueryParams, queryOpt) probe.send(coordinatorActor, q2) val info1 = probe.expectMsgPF(3.seconds.dilated) { case QueryResult(_, schema, srvs) => @@ -368,7 +372,7 @@ class NodeCoordinatorActorSpec extends ActorTest(NodeCoordinatorActorSpec.getNew val q2 = LogicalPlan2Query(ref, Aggregate(AggregationOperator.Sum, PeriodicSeries( // No filters, operate on all rows. Yes this is not a possible PromQL query. So what - RawSeries(AllChunksSelector, Nil, Seq("AvgTone")), 0, 10, 99)), qOpt) + RawSeries(AllChunksSelector, Nil, Seq("AvgTone")), 0, 10, 99)), UnavailablePromQlQueryParams , qOpt) probe.send(coordinatorActor, q2) probe.expectMsgPF() { case QueryResult(_, schema, vectors) => diff --git a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala index cf71155ae7..11e687b068 100644 --- a/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/client/SerializationSpec.scala @@ -6,8 +6,8 @@ import akka.testkit.TestProbe import org.scalatest.concurrent.ScalaFutures import filodb.coordinator.{ActorSpecConfig, ActorTest, ShardMapper} -import filodb.coordinator.queryengine2.QueryEngine -import filodb.core.{query, MachineMetricsData, MetricsTestData, NamesTestData} +import filodb.coordinator.queryengine2.{EmptyFailureProvider, QueryEngine, UnavailablePromQlQueryParams} +import filodb.core.{query, MachineMetricsData, MetricsTestData, NamesTestData, SpreadChange} import filodb.core.binaryrecord2.BinaryRecordRowReader import filodb.core.metadata.Column.ColumnType import filodb.core.store.IngestionConfig @@ -41,7 +41,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) serializer.fromBinary(serializer.toBinary(thing)) } - it("should be able to serialize IngestionConfig, SetupDataset, DatasetResourceSpec, IngestionSource") { + it("should be able to serialize IngestionConfig,a SetupDataset, DatasetResourceSpec, IngestionSource") { val source1 = s""" |dataset = ${dataset.name} |num-shards = 128 @@ -171,7 +171,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) mapper.registerNode(Seq(0), node0) def mapperRef: ShardMapper = mapper val dataset = MetricsTestData.timeseriesDataset - val engine = new QueryEngine(dataset, mapperRef) + val engine = new QueryEngine(dataset, mapperRef, EmptyFailureProvider) val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_bucket")), ColumnFilter("job", Filter.Equals("myService")), ColumnFilter("le", Filter.Equals("0.3"))) @@ -192,7 +192,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) val execPlan = engine.materialize(logicalPlan, QueryOptions(Some(StaticSpreadProvider(SpreadChange(0, 0))), - 100), new StaticSpreadProvider(SpreadChange(0, 0))) + 100), UnavailablePromQlQueryParams) roundTrip(execPlan) shouldEqual execPlan } @@ -205,13 +205,13 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val from = to - 50 val qParams = TimeStepParams(from, 10, to) val dataset = MetricsTestData.timeseriesDataset - val engine = new QueryEngine(dataset, mapperRef) + val engine = new QueryEngine(dataset, mapperRef, EmptyFailureProvider) val logicalPlan1 = Parser.queryRangeToLogicalPlan( "sum(rate(http_request_duration_seconds_bucket{job=\"prometheus\"}[20s])) by (handler)", qParams) val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), - 100), new StaticSpreadProvider(SpreadChange(0, 0))) + 100), UnavailablePromQlQueryParams) roundTrip(execPlan1) shouldEqual execPlan1 // scalastyle:off @@ -219,7 +219,8 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) "sum(rate(http_request_duration_microseconds_sum{job=\"prometheus\"}[5m])) by (handler) / sum(rate(http_request_duration_microseconds_count{job=\"prometheus\"}[5m])) by (handler)", qParams) // scalastyle:on - val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100), new StaticSpreadProvider(SpreadChange(0, 0))) + val execPlan2 = engine.materialize(logicalPlan2, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100), + UnavailablePromQlQueryParams) roundTrip(execPlan2) shouldEqual execPlan2 } @@ -235,14 +236,14 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) val from = to - 50 val qParams = TimeStepParams(from, 10, to) val dataset = MetricsTestData.timeseriesDataset - val engine = new QueryEngine(dataset, mapperRef) + val engine = new QueryEngine(dataset, mapperRef, EmptyFailureProvider) // with column filters having shardcolumns val logicalPlan1 = Parser.metadataQueryToLogicalPlan( "http_request_duration_seconds_bucket{job=\"prometheus\"}", qParams) val execPlan1 = engine.materialize(logicalPlan1, QueryOptions(Some( - new StaticSpreadProvider(SpreadChange(0, 0))), 100), new StaticSpreadProvider(SpreadChange(0, 0))) + new StaticSpreadProvider(SpreadChange(0, 0))), 100), UnavailablePromQlQueryParams) val partKeysExec = execPlan1.asInstanceOf[PartKeysExec] // will be dispatched to single shard roundTrip(partKeysExec) shouldEqual partKeysExec @@ -251,7 +252,7 @@ class SerializationSpec extends ActorTest(SerializationSpecConfig.getNewSystem) "http_request_duration_seconds_bucket", qParams) val execPlan2 = engine.materialize(logicalPlan2, QueryOptions( - Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100)) + Some(new StaticSpreadProvider(SpreadChange(0, 0))), 100), UnavailablePromQlQueryParams) val partKeysDistConcatExec = execPlan2.asInstanceOf[PartKeysDistConcatExec] // will be dispatched to all active shards since no shard column filters in the query diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/FailureProviderSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/FailureProviderSpec.scala new file mode 100644 index 0000000000..4e0ec32dfe --- /dev/null +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/FailureProviderSpec.scala @@ -0,0 +1,203 @@ +package filodb.coordinator.queryengine2 + +import monix.eval.Task +import monix.execution.Scheduler +import org.scalatest.{FunSpec, Matchers} +import scala.concurrent.duration.FiniteDuration + +import filodb.core.DatasetRef +import filodb.core.query.{ColumnFilter, Filter} +import filodb.query._ +import filodb.query.exec.{ExecPlan, PlanDispatcher} + +class FailureProviderSpec extends FunSpec with Matchers { + val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request")), + ColumnFilter("job", Filter.Equals("myService")), + ColumnFilter("le", Filter.Equals("0.3"))) + + val to = 150000 + val from = to - 50000 + + val intervalSelector = IntervalSelector(from, to) + + val raw1 = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed1 = PeriodicSeriesWithWindowing(raw1, from, 1000, to, 5000, RangeFunctionId.Rate) + val summed1 = Aggregate(AggregationOperator.Sum, windowed1, Nil, Seq("job")) + + val f2 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_count")), + ColumnFilter("job", Filter.Equals("myService"))) + val raw2 = RawSeries(rangeSelector = intervalSelector, filters = f2, columns = Seq("value")) + val windowed2 = PeriodicSeriesWithWindowing(raw2, from + 1000, 1000, to, 5000, RangeFunctionId.Rate) + val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) + + val binaryJoinLogicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) + val dummyDispatcher = new PlanDispatcher { + override def dispatch(plan: ExecPlan) + (implicit sched: Scheduler, + timeout: FiniteDuration): Task[QueryResponse] = ??? + } + + val datasetRef = DatasetRef("dataset", Some("cassandra")) + + it("should check for PeriodicSeries plan") { + QueryRoutingPlanner.isPeriodicSeriesPlan(summed1) shouldEqual (true) + QueryRoutingPlanner.isPeriodicSeriesPlan(raw2) shouldEqual (false) + } + + it("should extract time from logical plan") { + QueryRoutingPlanner.hasSingleTimeRange(summed1) shouldEqual (true) + QueryRoutingPlanner.hasSingleTimeRange(binaryJoinLogicalPlan) shouldEqual (false) + + val timeRange = QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(summed1) + + timeRange.startInMillis shouldEqual (100000) + timeRange.endInMillis shouldEqual (150000) + } + + it("should update time in logical plan") { + + val expectedRaw = RawSeries(rangeSelector = IntervalSelector(20000, 30000), filters = f1, columns = Seq("value")) + val updatedTimeLogicalPlan = QueryRoutingPlanner.copyWithUpdatedTimeRange(summed1, TimeRange(20000, 30000), 0) + + QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).startInMillis shouldEqual (20000) + QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).endInMillis shouldEqual (30000) + + updatedTimeLogicalPlan.isInstanceOf[Aggregate] shouldEqual (true) + val aggregate = updatedTimeLogicalPlan.asInstanceOf[Aggregate] + aggregate.vectors.isInstanceOf[PeriodicSeriesWithWindowing] shouldEqual (true) + aggregate.asInstanceOf[Aggregate].vectors.asInstanceOf[PeriodicSeriesWithWindowing].rawSeries.toString shouldEqual + (expectedRaw.toString) + + } + + it("should sort and remove smaller overlapping failures and generate local and remote routes correctly") { + val failureTimeRanges = Seq(FailureTimeRange("local", datasetRef, + TimeRange(1500, 5000), false), FailureTimeRange("remote", datasetRef, + TimeRange(100, 200), true), FailureTimeRange("local", datasetRef, + TimeRange(1000, 2000), false), FailureTimeRange("remote", datasetRef, + TimeRange(100, 700), true)) + + val expectedResult = Seq(LocalRoute(Some(TimeRange(50, 1499))), + RemoteRoute(Some(TimeRange(1500, 3000)))) + val routes = QueryRoutingPlanner.plan(failureTimeRanges, TimeRange(50, 3000), 0 , 1) + + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should split failures to remote followed by local") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 200), false), FailureTimeRange("remote", datasetRef, + TimeRange(1000, 3000), true)) + + val expectedResult = Seq(RemoteRoute(Some(TimeRange(50, 999))), + LocalRoute(Some(TimeRange(1000, 5000)))) + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(50, 5000), 0 , 1) + + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should generate remote route when there is only one failure which is in local") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 200), false)) + + val expectedResult = Seq(RemoteRoute(Some(TimeRange(50, 5000)))) + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(50, 5000), 0, 1) + + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should generate correct routes for local-remote-local failures ") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 200), false), FailureTimeRange("remote", datasetRef, + TimeRange(1000, 3000), true), FailureTimeRange("local", datasetRef, + TimeRange(4000, 4500), false)) + + val expectedResult = Seq(RemoteRoute(Some(TimeRange(50, 999))), + LocalRoute(Some(TimeRange(1000, 3999))), RemoteRoute(Some(TimeRange(4000, 5000)))) + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(50, 5000), 0 , 1) + + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + + } + + it("should generate correct routes for remote-local-remote failures ") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("remote", datasetRef, + TimeRange(100, 200), true), FailureTimeRange("local", datasetRef, + TimeRange(1000, 3000), false), FailureTimeRange("remote", datasetRef, + TimeRange(4000, 4500), true)) + + val expectedResult = Seq(LocalRoute(Some(TimeRange(50, 999))), + RemoteRoute(Some(TimeRange(1000, 3999))), LocalRoute(Some(TimeRange(4000, 5000)))) + + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(50, 5000), 0 , 1) + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should generate correct routes for remote-local failures with lookback and step") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("remote", datasetRef, + TimeRange(55, 500), true), FailureTimeRange("local", datasetRef, + TimeRange(1000, 3000), false)) + + val expectedResult = Seq(LocalRoute(Some(TimeRange(100, 980))), + RemoteRoute(Some(TimeRange(1000, 5000)))) + + //Query time is 100 to 5000 + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(100, 5000), 50 , 20) + println("routes:" + routes) + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should update time in logical plan when lookBack is present") { + val expectedRaw = RawSeries(rangeSelector = IntervalSelector(19900, 30000), filters = f1, columns = Seq("value")) + val updatedTimeLogicalPlan = QueryRoutingPlanner.copyWithUpdatedTimeRange(summed1, TimeRange(20000, 30000), 100) + + QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).startInMillis shouldEqual (20000) + QueryRoutingPlanner.getPeriodicSeriesTimeFromLogicalPlan(updatedTimeLogicalPlan).endInMillis shouldEqual (30000) + + updatedTimeLogicalPlan.isInstanceOf[Aggregate] shouldEqual (true) + val aggregate = updatedTimeLogicalPlan.asInstanceOf[Aggregate] + aggregate.vectors.isInstanceOf[PeriodicSeriesWithWindowing] shouldEqual (true) + aggregate.asInstanceOf[Aggregate].vectors.asInstanceOf[PeriodicSeriesWithWindowing].rawSeries.toString shouldEqual + (expectedRaw.toString) + + } + + it("should generate correct routes for local-remote failures with lookback and step") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("local", datasetRef, + TimeRange(55, 500), false), FailureTimeRange("remote", datasetRef, + TimeRange(1000, 3000), true)) + + val expectedResult = Seq(RemoteRoute(Some(TimeRange(100, 980))), + LocalRoute(Some(TimeRange(1000, 5000)))) + + //Query time is 100 to 5000 + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(100, 5000), 50 , 20) + routes(0).equals(expectedResult(0)) shouldEqual true + routes(1).equals(expectedResult(1)) shouldEqual true + routes.sameElements(expectedResult) shouldEqual (true) + } + + it("should generate correct routes when failure start time and end time is same ") { + val failureTimeRangeNonOverlapping = Seq(FailureTimeRange("remote", datasetRef, + TimeRange(100, 100), true), FailureTimeRange("local", datasetRef, + TimeRange(1000, 3000), false), FailureTimeRange("remote", datasetRef, + TimeRange(4000, 4000), true)) + + val expectedResult = Seq(LocalRoute(Some(TimeRange(50, 999))), + RemoteRoute(Some(TimeRange(1000, 3999))), LocalRoute(Some(TimeRange(4000, 5000)))) + + val routes = QueryRoutingPlanner.plan(failureTimeRangeNonOverlapping, TimeRange(50, 5000), 0 , 1) + routes.sameElements(expectedResult) shouldEqual (true) + } + +} diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index 0bf0abdf06..3777155435 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -2,15 +2,21 @@ package filodb.coordinator.queryengine2 import akka.actor.ActorSystem import akka.testkit.TestProbe +import monix.eval.Task +import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} +import scala.concurrent.duration.FiniteDuration + import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands._ -import filodb.core.MetricsTestData +import filodb.core.{DatasetRef, MetricsTestData, SpreadChange} import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query._ +import filodb.query +import filodb.query.{_} import filodb.query.exec._ class QueryEngineSpec extends FunSpec with Matchers { @@ -25,7 +31,12 @@ class QueryEngineSpec extends FunSpec with Matchers { val dataset = MetricsTestData.timeseriesDataset - val engine = new QueryEngine(dataset, mapperRef) + val emptyDispatcher = new PlanDispatcher { + override def dispatch(plan: ExecPlan)(implicit sched: Scheduler, + timeout: FiniteDuration): Task[query.QueryResponse] = ??? + } + + val engine = new QueryEngine(dataset, mapperRef, EmptyFailureProvider) /* This is the PromQL @@ -36,8 +47,8 @@ class QueryEngineSpec extends FunSpec with Matchers { */ val f1 = Seq(ColumnFilter("__name__", Filter.Equals("http_request_duration_seconds_bucket")), - ColumnFilter("job", Filter.Equals("myService")), - ColumnFilter("le", Filter.Equals("0.3"))) + ColumnFilter("job", Filter.Equals("myService")), + ColumnFilter("le", Filter.Equals("0.3"))) val to = System.currentTimeMillis() val from = to - 50000 @@ -53,13 +64,14 @@ class QueryEngineSpec extends FunSpec with Matchers { val raw2 = RawSeries(rangeSelector = intervalSelector, filters= f2, columns = Seq("value")) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 0, 1, 0, None) it ("should generate ExecPlan for LogicalPlan") { // final logical plan val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val execPlan = engine.materialize(logicalPlan, QueryOptions()) + val execPlan = engine.materialize(logicalPlan, QueryOptions(), promQlQueryParams) /* Following ExecPlan should be generated: @@ -101,7 +113,7 @@ class QueryEngineSpec extends FunSpec with Matchers { // materialized exec plan val execPlan = engine.materialize(logicalPlan, - QueryOptions(), spreadProvider = StaticSpreadProvider(SpreadChange(0, 4))) + QueryOptions(Some(StaticSpreadProvider(SpreadChange(0, 4))), 1000000), promQlQueryParams) execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true // Now there should be multiple levels of reduce because we have 16 shards @@ -122,18 +134,18 @@ class QueryEngineSpec extends FunSpec with Matchers { it("should throw BadQuery if illegal column name in LogicalPlan") { val raw3 = raw2.copy(columns = Seq("foo")) intercept[BadQueryException] { - engine.materialize(raw3, QueryOptions()) + engine.materialize(raw3, QueryOptions(), promQlQueryParams) } } it("should rename Prom __name__ filters if dataset has different metric column") { // Custom QueryEngine with different dataset with different metric name val dataset2 = dataset.copy(options = dataset.options.copy( - metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job"))) - val engine2 = new QueryEngine(dataset2, mapperRef) + metricColumn = "kpi", shardKeyColumns = Seq("kpi", "job"))) + val engine2 = new QueryEngine(dataset2, mapperRef, EmptyFailureProvider) // materialized exec plan - val execPlan = engine2.materialize(raw2, QueryOptions()) + val execPlan = engine2.materialize(raw2, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[DistConcatExec] shouldEqual true execPlan.children.foreach { l1 => l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true @@ -152,7 +164,7 @@ class QueryEngineSpec extends FunSpec with Matchers { val logicalPlan = BinaryJoin(summed1, BinaryOperator.DIV, Cardinality.OneToOne, summed2) // materialized exec plan - val execPlan = engine.materialize(logicalPlan, QueryOptions(), FunctionalSpreadProvider(spreadFunc)) + val execPlan = engine.materialize(logicalPlan, QueryOptions(Some(FunctionalSpreadProvider(spreadFunc)), 1000000), promQlQueryParams) execPlan.printTree() execPlan.isInstanceOf[BinaryJoinExec] shouldEqual true @@ -168,7 +180,8 @@ class QueryEngineSpec extends FunSpec with Matchers { def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) // spread change time is in ms } - val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread)) + val execPlan = engine.materialize(lp, QueryOptions(Some(FunctionalSpreadProvider(spread)), 1000000), + promQlQueryParams) execPlan.rangeVectorTransformers.head.isInstanceOf[StitchRvsMapper] shouldEqual true } @@ -177,17 +190,19 @@ class QueryEngineSpec extends FunSpec with Matchers { def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread)) + val execPlan = engine.materialize(lp, QueryOptions(Some(FunctionalSpreadProvider(spread)), 1000000), + promQlQueryParams) execPlan.rangeVectorTransformers.isEmpty shouldEqual true } it("should stitch results before binary join when spread changed in query range") { val lp = Parser.queryRangeToLogicalPlan("""count(foo{job="bar"} + baz{job="bar"})""", - TimeStepParams(20000, 100, 30000)) + TimeStepParams(20000, 100, 30000)) def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(25000000, 2)) } - val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread)) + val execPlan = engine.materialize(lp, QueryOptions(Some(FunctionalSpreadProvider(spread)), 1000000), + promQlQueryParams) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.size shouldEqual 2 @@ -200,7 +215,8 @@ class QueryEngineSpec extends FunSpec with Matchers { def spread(filter: Seq[ColumnFilter]): Seq[SpreadChange] = { Seq(SpreadChange(0, 1), SpreadChange(35000000, 2)) } - val execPlan = engine.materialize(lp, QueryOptions(), FunctionalSpreadProvider(spread)) + val execPlan = engine.materialize(lp, QueryOptions(Some(FunctionalSpreadProvider(spread)), 1000000), + promQlQueryParams) val binaryJoinNode = execPlan.children(0) binaryJoinNode.isInstanceOf[BinaryJoinExec] shouldEqual true binaryJoinNode.children.foreach(_.isInstanceOf[StitchRvsExec] should not equal true) @@ -211,7 +227,7 @@ class QueryEngineSpec extends FunSpec with Matchers { val logicalPlan = BinaryJoin(summed1, BinaryOperator.LAND, Cardinality.ManyToMany, summed2) // materialized exec plan - val execPlan = engine.materialize(logicalPlan, QueryOptions()) + val execPlan = engine.materialize(logicalPlan, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[SetOperatorExec] shouldEqual true execPlan.children.foreach { l1 => @@ -225,4 +241,268 @@ class QueryEngineSpec extends FunSpec with Matchers { } } } + + it("should not generate PromQlExec plan when local overlapping failure is smaller") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("remote", datasetRef, + TimeRange(1500, 4000), true), + FailureTimeRange("local", datasetRef, //Removed + TimeRange(2000, 3000), false), FailureTimeRange("remote", datasetRef, + TimeRange(5000, 6000), true)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[ReduceAggregateExec] shouldEqual (true) + + // Should ignore smaller local failure which is from 1500 - 4000 and generate local exec plan + val reduceAggregateExec = execPlan.asInstanceOf[ReduceAggregateExec] + + reduceAggregateExec.children.length shouldEqual (2) //default spread is 1 so 2 shards + + reduceAggregateExec.children.foreach { l1 => + l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true + l1.rangeVectorTransformers.size shouldEqual 2 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].start shouldEqual (100) + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual (10000) + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + } + + it("should generate only PromQlExec when failure is present only in local") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(1000, 6000), false)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlExec].params.start shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlExec].params.end shouldEqual(to/1000) + } + + it("should generate RemotExecPlan with RawSeries time according to lookBack") { + val to = 2000000 + val from = 1000000 + val intervalSelector = IntervalSelector(from - 50000 , to) // Lookback of 50000 + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("", from/1000, 1, to/1000, None) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(910000, 1030000), false), FailureTimeRange("remote", datasetRef, + TimeRange(1060000, 1090000), true)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[StitchRvsExec] shouldEqual (true) + + val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] + stitchRvsExec.children.size shouldEqual (2) + stitchRvsExec.children(0).isInstanceOf[ReduceAggregateExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlExec] shouldEqual (true) + + val child1 = stitchRvsExec.children(0).asInstanceOf[ReduceAggregateExec] + val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlExec] + + child1.children.length shouldEqual (2) //default spread is 1 so 2 shards + + child1.children.foreach { l1 => + l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true + l1.asInstanceOf[SelectRawPartitionsExec].chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual + 1010000 + l1.asInstanceOf[SelectRawPartitionsExec].chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual 2000000 + l1.rangeVectorTransformers.size shouldEqual 2 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].start shouldEqual (1060000) + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual (2000000) + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + + child2.params.start shouldEqual from/1000 + child2.params.end shouldEqual (1060000-1)/1000 + child2.params.processFailure shouldEqual(false) + } + + it("should generate only PromQlExec when local failure starts before query time") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 10000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(50, 200), false)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlExec].params.start shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlExec].params.end shouldEqual(to/1000) + } + + it("should generate only PromQlExec when local failure timerange coincide with query time range") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 10000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(100, 10000), false)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlExec].params.start shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlExec].params.end shouldEqual(to/1000) + } + + it("should generate only PromQlExec when local failure starts before query end time and ends after query end time") { + val to = 10000 + val from = 100 + val intervalSelector = IntervalSelector(from, to) + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from, 100, to, 10000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(5000, 20000), false)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + execPlan.asInstanceOf[PromQlExec].params.start shouldEqual(from/1000) + execPlan.asInstanceOf[PromQlExec].params.end shouldEqual(to/1000) + } + + it("should generate PromQlExecPlan and LocalPlan with RawSeries time according to lookBack and step") { + val to = 2000 + val from = 900 + val lookBack = 300000 + val step = 60 + val intervalSelector = IntervalSelector(from * 1000 - lookBack , to * 1000) // Lookback of 300 + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(910000, 1030000), false), FailureTimeRange("remote", datasetRef, + TimeRange(1060000, 1090000), true)) + } + } + //900K to 1020K and 1020+60 k to 2000K + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[StitchRvsExec] shouldEqual (true) + + val stitchRvsExec = execPlan.asInstanceOf[StitchRvsExec] + stitchRvsExec.children.size shouldEqual (2) + stitchRvsExec.children(0).isInstanceOf[ReduceAggregateExec] shouldEqual (true) + stitchRvsExec.children(1).isInstanceOf[PromQlExec] shouldEqual (true) + + val child1 = stitchRvsExec.children(0).asInstanceOf[ReduceAggregateExec] + val child2 = stitchRvsExec.children(1).asInstanceOf[PromQlExec] + + child1.children.length shouldEqual (2) //default spread is 1 so 2 shards + + child1.children.foreach { l1 => + l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true + l1.asInstanceOf[SelectRawPartitionsExec].chunkMethod.asInstanceOf[TimeRangeChunkScan].startTime shouldEqual + (1080000-lookBack) + l1.asInstanceOf[SelectRawPartitionsExec].chunkMethod.asInstanceOf[TimeRangeChunkScan].endTime shouldEqual 2000000 + l1.rangeVectorTransformers.size shouldEqual 2 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].start shouldEqual (1080000) + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual (2000000) + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + + child2.params.start shouldEqual 900 + child2.params.end shouldEqual 1020 + child2.params.step shouldEqual 60 + child2.params.processFailure shouldEqual(false) + } + + it("should generate only PromQlExecPlan when second remote ends after query end time") { + val to = 2000 + val from = 900 + val lookBack = 300000 + val step = 60 + val intervalSelector = IntervalSelector(from * 1000 - lookBack , to * 1000) // Lookback of 300 + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(910000, 1030000), false), FailureTimeRange("remote", datasetRef, + TimeRange(2000000, 2500000), true)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + + val child = execPlan.asInstanceOf[PromQlExec] + + child.params.start shouldEqual 900 + child.params.end shouldEqual 1980 + child.params.step shouldEqual 60 + child.params.processFailure shouldEqual(false) + } } diff --git a/core/src/main/scala/filodb.core/SpreadProvider.scala b/core/src/main/scala/filodb.core/SpreadProvider.scala new file mode 100644 index 0000000000..9d5397ef2d --- /dev/null +++ b/core/src/main/scala/filodb.core/SpreadProvider.scala @@ -0,0 +1,9 @@ +package filodb.core + +import filodb.core.query.ColumnFilter + +trait SpreadProvider { + def spreadFunc(filter: Seq[ColumnFilter]): Seq[SpreadChange] +} + +final case class SpreadChange(time: Long = 0L, spread: Int = 1) \ No newline at end of file diff --git a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala index 3f6172dc67..bf11ffa1e5 100644 --- a/http/src/main/scala/filodb/http/PrometheusApiRoute.scala +++ b/http/src/main/scala/filodb/http/PrometheusApiRoute.scala @@ -9,17 +9,16 @@ import akka.stream.ActorMaterializer import akka.util.ByteString import com.typesafe.scalalogging.StrictLogging import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport -import io.circe.{Decoder, Encoder, HCursor, Json} import org.xerial.snappy.Snappy import remote.RemoteStorage.ReadRequest import filodb.coordinator.client.IngestionCommands.UnknownDataset import filodb.coordinator.client.QueryCommands._ -import filodb.core.DatasetRef +import filodb.coordinator.queryengine2.{PromQlQueryParams, TsdbQueryParams, UnavailablePromQlQueryParams} +import filodb.core.{DatasetRef, SpreadChange, SpreadProvider} import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.prometheus.query.PrometheusModel.Sampl -import filodb.query.{LogicalPlan, QueryError, QueryResult} +import filodb.query._ import filodb.query.exec.ExecPlan class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit am: ActorMaterializer) @@ -44,8 +43,9 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a 'step.as[Int], 'explainOnly.as[Boolean].?, 'verbose.as[Boolean].?, 'spread.as[Int].?) { (query, start, end, step, explainOnly, verbose, spread) => val logicalPlan = Parser.queryRangeToLogicalPlan(query, TimeStepParams(start.toLong, step, end.toLong)) + askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), verbose.getOrElse(false), - spread) + spread, PromQlQueryParams(query, start.toLong, step.toLong, end.toLong, spread)) } } } ~ @@ -60,7 +60,7 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a { (query, time, explainOnly, verbose, spread) => val logicalPlan = Parser.queryToLogicalPlan(query, time.toLong) askQueryAndRespond(dataset, logicalPlan, explainOnly.getOrElse(false), - verbose.getOrElse(false), spread) + verbose.getOrElse(false), spread, PromQlQueryParams(query, time.toLong, 1000, time.toLong, spread)) } } } ~ @@ -82,8 +82,8 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a // but Akka doesnt support snappy out of the box. Elegant solution is a TODO for later. val readReq = ReadRequest.parseFrom(Snappy.uncompress(bytes.toArray)) val asks = toFiloDBLogicalPlans(readReq).map { logicalPlan => - asyncAsk(nodeCoord, LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan), - settings.queryAskTimeout) + asyncAsk(nodeCoord, LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, + UnavailablePromQlQueryParams), settings.queryAskTimeout) } Future.sequence(asks) } @@ -108,13 +108,13 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a } private def askQueryAndRespond(dataset: String, logicalPlan: LogicalPlan, explainOnly: Boolean, verbose: Boolean, - spread: Option[Int]) = { + spread: Option[Int], tsdbQueryParams: TsdbQueryParams) = { val spreadProvider: Option[SpreadProvider] = spread.map(s => StaticSpreadProvider(SpreadChange(0, s))) val command = if (explainOnly) { - ExplainPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, QueryOptions(spreadProvider)) + ExplainPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, tsdbQueryParams, QueryOptions(spreadProvider)) } else { - LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, QueryOptions(spreadProvider)) + LogicalPlan2Query(DatasetRef.fromDotString(dataset), logicalPlan, tsdbQueryParams, QueryOptions(spreadProvider)) } onSuccess(asyncAsk(nodeCoord, command, settings.queryAskTimeout)) { case qr: QueryResult => complete(toPromSuccessResponse(qr, verbose)) @@ -125,21 +125,3 @@ class PrometheusApiRoute(nodeCoord: ActorRef, settings: HttpSettings)(implicit a } } } - -// TODO extend and make more generic -object PromCirceSupport { - // necessary to encode sample in promql response as an array with long and double value as string - implicit val encodeSampl: Encoder[Sampl] = new Encoder[Sampl] { - final def apply(a: Sampl): Json = Json.arr(Json.fromLong(a.timestamp), Json.fromString(a.value.toString)) - } - - implicit val decodeFoo: Decoder[Sampl] = new Decoder[Sampl] { - final def apply(c: HCursor): Decoder.Result[Sampl] = { - for {timestamp <- c.downArray.as[Long].right - value <- c.downArray.right.as[String].right - } yield { - Sampl(timestamp, value.toDouble) - } - } - } -} \ No newline at end of file diff --git a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala index 0ec5d38a42..a28bb14669 100644 --- a/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala +++ b/http/src/test/scala/filodb/http/PrometheusApiRouteSpec.scala @@ -6,11 +6,12 @@ import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest} import akka.testkit.TestProbe import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport import org.scalatest.FunSpec -import scala.concurrent.duration._ +import scala.concurrent.duration._ import filodb.coordinator._ import filodb.core.{AsyncTest, DatasetRef, TestData} -import filodb.prometheus.query.PrometheusModel.ExplainPlanResponse +import filodb.query.ExplainPlanResponse + object PrometheusApiRouteSpec extends ActorSpecConfig { override lazy val configString = s""" diff --git a/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala b/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala index 1397dbc015..1b0bde78bf 100644 --- a/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala +++ b/http/src/test/scala/filodb/prom/downsample/GaugeDownsampleValidator.scala @@ -14,8 +14,8 @@ import com.typesafe.config.ConfigFactory import com.typesafe.scalalogging.StrictLogging import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport -import filodb.http.PromCirceSupport -import filodb.prometheus.query.PrometheusModel.SuccessResponse +import filodb.query.PromCirceSupport +import filodb.query.SuccessResponse /** * Use this tool to validate raw data against downsampled data for gauges. diff --git a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala index e069c5dcc1..5ee71c589a 100644 --- a/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/HistogramQueryBenchmark.scala @@ -14,15 +14,15 @@ import org.openjdk.jmh.annotations._ import filodb.coordinator.{FilodbCluster, IngestionStarted, ShardMapper} import filodb.coordinator.client.QueryCommands._ -import filodb.coordinator.queryengine2.QueryEngine -import filodb.core.{MachineMetricsData, MetricsTestData, TestData} +import filodb.coordinator.queryengine2.{EmptyFailureProvider, QueryEngine, UnavailablePromQlQueryParams} +import filodb.core.{MachineMetricsData, MetricsTestData, SpreadChange, TestData} import filodb.core.binaryrecord2.RecordBuilder import filodb.core.memstore._ import filodb.core.store._ -import filodb.memory.format.SeqRowReader import filodb.memory.MemFactory +import filodb.memory.format.SeqRowReader import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig +import filodb.query.{QueryConfig, QueryOptions} //scalastyle:off regex /** @@ -76,8 +76,8 @@ class HistogramQueryBenchmark { shardMapper.updateFromEvent(IngestionStarted(histDataset.ref, 0, coordinator)) // Query configuration - val hEngine = new QueryEngine(histDataset, shardMapper) - val pEngine = new QueryEngine(promDataset, shardMapper) + val hEngine = new QueryEngine(histDataset, shardMapper, EmptyFailureProvider) + val pEngine = new QueryEngine(promDataset, shardMapper, EmptyFailureProvider) val startTime = 100000L + 100*1000 // 100 samples in. Look back 30 samples, which normally would be 5min val histQuery = """histogram_quantile(0.9, sum_over_time(http_requests_total{job="prometheus",__col__="h"}[30s]))""" @@ -88,12 +88,12 @@ class HistogramQueryBenchmark { val qOptions = QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 100). copy(shardOverrides = Some(Seq(0))) val hLogicalPlan = Parser.queryToLogicalPlan(histQuery, startTime/1000) - val hExecPlan = hEngine.materialize(hLogicalPlan, qOptions) + val hExecPlan = hEngine.materialize(hLogicalPlan, qOptions, UnavailablePromQlQueryParams) val querySched = Scheduler.singleThread(s"benchmark-query") val queryConfig = new QueryConfig(config.getConfig("query")) val pLogicalPlan = Parser.queryToLogicalPlan(promQuery, startTime/1000) - val pExecPlan = pEngine.materialize(pLogicalPlan, qOptions) + val pExecPlan = pEngine.materialize(pLogicalPlan, qOptions, UnavailablePromQlQueryParams) @TearDown def shutdownFiloActors(): Unit = { diff --git a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala index 6e1a8c98a0..2f7589e5d7 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryAndIngestBenchmark.scala @@ -13,7 +13,9 @@ import monix.eval.Task import monix.reactive.Observable import org.openjdk.jmh.annotations.{Level => JMHLevel, _} +import filodb.coordinator.queryengine2.UnavailablePromQlQueryParams import filodb.core.GlobalConfig +import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.store.StoreConfig @@ -21,7 +23,7 @@ import filodb.gateway.GatewayServer import filodb.gateway.conversion.PrometheusInputRecord import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryOptions, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -129,7 +131,8 @@ class QueryAndIngestBenchmark extends StrictLogging { 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(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 1000000)) + LogicalPlan2Query(dataset.ref, plan, UnavailablePromQlQueryParams, QueryOptions(Some(new StaticSpreadProvider + (SpreadChange(0, 1))), 1000000)) } private var testProducingFut: Option[Future[Unit]] = None diff --git a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala index 092916fea7..e15e735b40 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryHiCardInMemoryBenchmark.scala @@ -13,12 +13,14 @@ import monix.eval.Task import monix.reactive.Observable import org.openjdk.jmh.annotations._ +import filodb.coordinator.queryengine2.{EmptyFailureProvider, UnavailablePromQlQueryParams} +import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.QueryConfig +import filodb.query.{QueryConfig, QueryOptions} import filodb.query.exec.ExecPlan import filodb.timeseries.TestTimeseriesProducer @@ -100,7 +102,7 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { // Stuff for directly executing queries ourselves import filodb.coordinator.queryengine2.QueryEngine - val engine = new QueryEngine(dataset, shardMapper) + val engine = new QueryEngine(dataset, shardMapper, EmptyFailureProvider) val numQueries = 100 // Please make sure this number matches the OperationsPerInvocation below val queryIntervalSec = samplesDuration.toSeconds // # minutes between start and stop @@ -111,7 +113,7 @@ class QueryHiCardInMemoryBenchmark extends StrictLogging { val queryStartTime = ingestionStartTime + 7.minutes.toMillis // 7 minutes from start until 60 minutes from start val qParams = TimeStepParams(queryStartTime/1000, queryStep, queryStartTime/1000 + queryIntervalSec) val execPlan = engine.materialize(Parser.queryRangeToLogicalPlan(query, qParams), - QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 20000)) + QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 0))), 20000), UnavailablePromQlQueryParams) var child = execPlan while (child.children.size > 0) child = child.children(0) child diff --git a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala index 2f2ea2f985..88a80eed2f 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryInMemoryBenchmark.scala @@ -14,13 +14,14 @@ import monix.execution.Scheduler import monix.reactive.Observable import org.openjdk.jmh.annotations._ -import filodb.coordinator.{IngestionStarted, ShardMapper} +import filodb.coordinator.queryengine2.{EmptyFailureProvider, UnavailablePromQlQueryParams} +import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{SomeData, TimeSeriesMemStore} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryConfig, QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryConfig, QueryError => QError, QueryOptions, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -102,7 +103,8 @@ class QueryInMemoryBenchmark extends StrictLogging { // Stuff for directly executing queries ourselves import filodb.coordinator.queryengine2.QueryEngine - val engine = new QueryEngine(dataset, shardMapper) + + val engine = new QueryEngine(dataset, shardMapper, EmptyFailureProvider) /** * ## ======== Queries =========== @@ -119,7 +121,8 @@ class QueryInMemoryBenchmark extends StrictLogging { 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(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000)) + LogicalPlan2Query(dataset.ref, plan, UnavailablePromQlQueryParams, QueryOptions(Some(new StaticSpreadProvider + (SpreadChange(0, 1))), 20000)) } @TearDown @@ -148,7 +151,8 @@ class QueryInMemoryBenchmark extends StrictLogging { val qParams2 = TimeStepParams(queryTime/1000, noOverlapStep, (queryTime/1000) + queryIntervalMin*60) val logicalPlans2 = queries.map { q => Parser.queryRangeToLogicalPlan(q, qParams2) } val queryCommands2 = logicalPlans2.map { plan => - LogicalPlan2Query(dataset.ref, plan, QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 10000)) + LogicalPlan2Query(dataset.ref, plan, UnavailablePromQlQueryParams, QueryOptions(Some(new StaticSpreadProvider + (SpreadChange(0, 1))), 10000)) } @Benchmark @@ -171,7 +175,7 @@ class QueryInMemoryBenchmark extends StrictLogging { val qOptions = QueryOptions(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 10000) val logicalPlan = Parser.queryRangeToLogicalPlan(rawQuery, qParams) // Pick the children nodes, not the DistConcatExec. Thus we can run in a single thread this way - val execPlan = engine.materialize(logicalPlan, qOptions).children.head + val execPlan = engine.materialize(logicalPlan, qOptions, UnavailablePromQlQueryParams).children.head val querySched = Scheduler.singleThread(s"benchmark-query") val queryConfig = new QueryConfig(cluster.settings.allConfig.getConfig("filodb.query")) @@ -191,7 +195,7 @@ class QueryInMemoryBenchmark extends StrictLogging { val minQuery = """min_over_time(heap_usage{_ns="App-2"}[5m])""" val minLP = Parser.queryRangeToLogicalPlan(minQuery, qParams) - val minEP = engine.materialize(minLP, qOptions).children.head + val minEP = engine.materialize(minLP, qOptions, UnavailablePromQlQueryParams).children.head @Benchmark @BenchmarkMode(Array(Mode.Throughput)) @@ -207,7 +211,7 @@ class QueryInMemoryBenchmark extends StrictLogging { // sum(rate) query and rate involves counter correction, very important case val sumRateLP = Parser.queryRangeToLogicalPlan(sumRateQuery, qParams) - val sumRateEP = engine.materialize(sumRateLP, qOptions).children.head + val sumRateEP = engine.materialize(sumRateLP, qOptions, UnavailablePromQlQueryParams).children.head @Benchmark @BenchmarkMode(Array(Mode.Throughput)) diff --git a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala index 5a8c56e75c..1077da81ed 100644 --- a/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala +++ b/jmh/src/main/scala/filodb.jmh/QueryOnDemandBenchmark.scala @@ -13,13 +13,14 @@ import monix.eval.Task import monix.reactive.Observable import org.openjdk.jmh.annotations.{Level => JmhLevel, _} -import filodb.coordinator.ShardMapper +import filodb.coordinator.queryengine2.UnavailablePromQlQueryParams +import filodb.core.SpreadChange import filodb.core.binaryrecord2.RecordContainer import filodb.core.memstore.{DataOrCommand, FlushStream, SomeData, TimeSeriesMemStore} import filodb.core.store.StoreConfig import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.query.{QueryError => QError, QueryResult => QueryResult2} +import filodb.query.{QueryError => QError, QueryOptions, QueryResult => QueryResult2} import filodb.timeseries.TestTimeseriesProducer //scalastyle:off regex @@ -136,7 +137,8 @@ class QueryOnDemandBenchmark extends StrictLogging { 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(Some(new StaticSpreadProvider(SpreadChange(0, 1))), 20000)) + LogicalPlan2Query(dataset.ref, plan, UnavailablePromQlQueryParams, QueryOptions(Some(new StaticSpreadProvider + (SpreadChange(0, 1))), 20000)) } @TearDown diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 38b56da908..5b7cf23363 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -236,6 +236,10 @@ object FiloBuild extends Build { lazy val queryDeps = commonDeps ++ Seq( "com.tdunning" % "t-digest" % "3.1", + "com.softwaremill.sttp" %% "circe" % sttpVersion , + "com.softwaremill.sttp" %% "akka-http-backend" % sttpVersion, + "com.softwaremill.sttp" %% "core" % sttpVersion, + circeGeneric, scalaxyDep ) diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index f70b767f65..0c75c0f4f7 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -3,32 +3,12 @@ package filodb.prometheus.query import remote.RemoteStorage._ import filodb.core.query.{ColumnFilter, Filter, RangeVector} -import filodb.query.{IntervalSelector, LogicalPlan, QueryResultType, RawSeries} +import filodb.query.{Data, ErrorResponse, ExplainPlanResponse, IntervalSelector, LogicalPlan, QueryResultType, + RawSeries, Result, Sampl, SuccessResponse} import filodb.query.exec.ExecPlan object PrometheusModel { - sealed trait PromQueryResponse { - def status: String - } - - final case class ErrorResponse(errorType: String, error: String, status: String = "error") extends PromQueryResponse - - final case class SuccessResponse(data: Data, status: String = "success") extends PromQueryResponse - - final case class ExplainPlanResponse(debugInfo: Seq[String], status: String = "success") extends PromQueryResponse - - final case class Data(resultType: String, result: Seq[Result]) - - final case class Result(metric: Map[String, String], values: Seq[Sampl]) - - /** - * Metric value for a given timestamp - * @param timestamp in seconds since epoch - * @param value value of metric - */ - final case class Sampl(timestamp: Long, value: Double) - /** * Converts a prometheus read request to a Seq[LogicalPlan] */ diff --git a/query/src/main/scala/filodb/query/PromCirceSupport.scala b/query/src/main/scala/filodb/query/PromCirceSupport.scala new file mode 100644 index 0000000000..bb9499f39c --- /dev/null +++ b/query/src/main/scala/filodb/query/PromCirceSupport.scala @@ -0,0 +1,20 @@ +package filodb.query + +import io.circe.{Decoder, Encoder, HCursor, Json} + +object PromCirceSupport { + // necessary to encode sample in promql response as an array with long and double value as string + implicit val encodeSampl: Encoder[Sampl] = new Encoder[Sampl] { + final def apply(a: Sampl): Json = Json.arr(Json.fromLong(a.timestamp), Json.fromString(a.value.toString)) + } + + implicit val decodeFoo: Decoder[Sampl] = new Decoder[Sampl] { + final def apply(c: HCursor): Decoder.Result[Sampl] = { + for {timestamp <- c.downArray.as[Long].right + value <- c.downArray.right.as[String].right + } yield { + Sampl(timestamp, value.toDouble) + } + } + } +} \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/PromQlInvocationParams.scala b/query/src/main/scala/filodb/query/PromQlInvocationParams.scala new file mode 100644 index 0000000000..ffa5c5f43b --- /dev/null +++ b/query/src/main/scala/filodb/query/PromQlInvocationParams.scala @@ -0,0 +1,5 @@ +package filodb.query + +case class PromQlInvocationParams(endpoint: String, promQl: String, start: Long, step: Long, end: Long, + spread: Option[Int] = None, processFailure: Boolean = true) + diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala new file mode 100644 index 0000000000..d36d3c8094 --- /dev/null +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -0,0 +1,22 @@ +package filodb.query + +sealed trait PromQueryResponse { + def status: String +} + +final case class ErrorResponse(errorType: String, error: String, status: String = "error") extends PromQueryResponse + +final case class SuccessResponse(data: Data, status: String = "success") extends PromQueryResponse + +final case class ExplainPlanResponse(debugInfo: Seq[String], status: String = "success") extends PromQueryResponse + +final case class Data(resultType: String, result: Seq[Result]) + +final case class Result(metric: Map[String, String], values: Seq[Sampl]) + +/** + * Metric value for a given timestamp + * @param timestamp in seconds since epoch + * @param value value of metric + */ +final case class Sampl(timestamp: Long, value: Double) \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/QueryConfig.scala b/query/src/main/scala/filodb/query/QueryConfig.scala index b38cda3d31..accf92f5da 100644 --- a/query/src/main/scala/filodb/query/QueryConfig.scala +++ b/query/src/main/scala/filodb/query/QueryConfig.scala @@ -2,7 +2,7 @@ package filodb.query import scala.concurrent.duration.FiniteDuration -import com.typesafe.config.Config +import com.typesafe.config.{Config, ConfigFactory} import net.ceedubs.ficus.Ficus._ object QueryConfig { @@ -20,3 +20,5 @@ class QueryConfig(queryConfig: Config) { */ def has(feature: String): Boolean = queryConfig.as[Option[Boolean]](feature).getOrElse(false) } + +object EmptyQueryConfig extends QueryConfig(queryConfig = ConfigFactory.empty()) diff --git a/query/src/main/scala/filodb/query/QueryOptions.scala b/query/src/main/scala/filodb/query/QueryOptions.scala new file mode 100644 index 0000000000..34e8d5000e --- /dev/null +++ b/query/src/main/scala/filodb/query/QueryOptions.scala @@ -0,0 +1,47 @@ +package filodb.query + +import filodb.core.{SpreadChange, SpreadProvider} +import filodb.core.query.{ColumnFilter, Filter} + +/** + * This class provides general query processing parameters + * + * @param spreadFunc a function that returns chronologically ordered spread changes for the filter + */ +final case class QueryOptions(spreadProvider: Option[SpreadProvider] = None, + parallelism: Int = 16, + queryTimeoutSecs: Int = 30, + sampleLimit: Int = 1000000, + shardOverrides: Option[Seq[Int]] = None) + +object QueryOptions { + def apply(constSpread: Option[SpreadProvider], sampleLimit: Int): QueryOptions = + QueryOptions(spreadProvider = constSpread, sampleLimit = sampleLimit) + + /** + * Creates a spreadFunc that looks for a particular filter with keyName Equals a value, and then maps values + * present in the spreadMap to specific spread values, with a default if the filter/value not present in the map + */ + def simpleMapSpreadFunc(keyName: String, + spreadMap: collection.mutable.Map[collection.Map[String, String], Int], + defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = { + filters: Seq[ColumnFilter] => + filters.collectFirst { + case ColumnFilter(key, Filter.Equals(filtVal: String)) if key == keyName => filtVal + }.map { tagValue => + Seq(SpreadChange(spread = spreadMap.getOrElse(collection.mutable.Map(keyName->tagValue), defaultSpread))) + }.getOrElse(Seq(SpreadChange(defaultSpread))) + } + + import collection.JavaConverters._ + + def simpleMapSpreadFunc(keyName: String, + spreadMap: java.util.Map[java.util.Map[String, String], Integer], + defaultSpread: Int): Seq[ColumnFilter] => Seq[SpreadChange] = { + val spreadAssignment: collection.mutable.Map[collection.Map[String, String], Int]= spreadMap.asScala.map { + case (d, v) => d.asScala -> v.toInt + } + + simpleMapSpreadFunc(keyName, spreadAssignment, defaultSpread) + } +} diff --git a/query/src/main/scala/filodb/query/ResultTypes.scala b/query/src/main/scala/filodb/query/ResultTypes.scala index 8e78cb39fc..6a31ff2cf6 100644 --- a/query/src/main/scala/filodb/query/ResultTypes.scala +++ b/query/src/main/scala/filodb/query/ResultTypes.scala @@ -2,7 +2,7 @@ package filodb.query import enumeratum.{Enum, EnumEntry} -import filodb.core.{DatasetRef, ErrorResponse, NodeCommand, NodeResponse} +import filodb.core.{DatasetRef, NodeCommand, NodeResponse} import filodb.core.query.{RangeVector, ResultSchema} trait QueryCommand extends NodeCommand with java.io.Serializable { @@ -14,7 +14,7 @@ trait QueryResponse extends NodeResponse with java.io.Serializable { def id: String } -final case class QueryError(id: String, t: Throwable) extends QueryResponse with ErrorResponse { +final case class QueryError(id: String, t: Throwable) extends QueryResponse with filodb.core.ErrorResponse { override def toString: String = s"QueryError id=$id ${t.getClass.getName} ${t.getMessage}\n" + t.getStackTrace.map(_.toString).mkString("\n") } diff --git a/query/src/main/scala/filodb/query/exec/ExecPlan.scala b/query/src/main/scala/filodb/query/exec/ExecPlan.scala index 1e1dd7ad77..cfa8fed6e9 100644 --- a/query/src/main/scala/filodb/query/exec/ExecPlan.scala +++ b/query/src/main/scala/filodb/query/exec/ExecPlan.scala @@ -101,7 +101,7 @@ trait ExecPlan extends QueryCommand { * */ // scalastyle:off method.length - final def execute(source: ChunkSource, + def execute(source: ChunkSource, dataset: Dataset, queryConfig: QueryConfig) (implicit sched: Scheduler, diff --git a/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala new file mode 100644 index 0000000000..00b8d76d19 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/InProcessPlanDispatcher.scala @@ -0,0 +1,59 @@ +package filodb.query.exec + +import scala.concurrent.duration.FiniteDuration + +import monix.eval.Task +import monix.execution.Scheduler +import monix.reactive.Observable + +import filodb.core.DatasetRef +import filodb.core.Types.ColumnId +import filodb.core.metadata.Dataset +import filodb.core.store.{ChunkScanMethod, ChunkSource, ChunkSourceStats, + PartitionScanMethod, RawPartData, ReadablePartition, ScanSplit} +import filodb.query.{EmptyQueryConfig, QueryConfig, QueryResponse} + +/** + * Dispatcher which will make a No-Op style call to ExecPlan#excecute(). + * Goal is that Non-Leaf plans can be executed locally in JVM and make network + * calls only for children. + * @param dataset to be used by ExecPlan#execute + */ +case class InProcessPlanDispatcher(dataset: Dataset) extends PlanDispatcher { + + // Empty query config, since its does not apply in case of non-leaf plans + val queryConfig: QueryConfig = EmptyQueryConfig + + override def dispatch(plan: ExecPlan)(implicit sched: Scheduler, + timeout: FiniteDuration): Task[QueryResponse] = { + // unsupported source since its does not apply in case of non-leaf plans + val source = UnsupportedChunkSource() + // translate implicit ExecutionContext to monix.Scheduler + plan.execute(source, dataset, queryConfig) + } + +} + +/** + * No-op chunk source which does nothing and throws exception for all functions. + */ +case class UnsupportedChunkSource() extends ChunkSource { + + override def scanPartitions(dataset: Dataset, columnIDs: Seq[ColumnId], partMethod: PartitionScanMethod, + chunkMethod: ChunkScanMethod): Observable[ReadablePartition] = + throw new UnsupportedOperationException("This operation is not supported") + + override def groupsInDataset(dataset: Dataset): Int = + throw new UnsupportedOperationException("This operation is not supported") + + override def stats: ChunkSourceStats = + throw new UnsupportedOperationException("This operation is not supported") + + override def getScanSplits(dataset: DatasetRef, splitsPerNode: Int): Seq[ScanSplit] = + throw new UnsupportedOperationException("This operation is not supported") + + override def readRawPartitions(dataset: Dataset, columnIDs: Seq[ColumnId], partMethod: PartitionScanMethod, + chunkMethod: ChunkScanMethod): Observable[RawPartData] = + throw new UnsupportedOperationException("This operation is not supported") +} + diff --git a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala index fe4b273f76..8ca1f87e08 100644 --- a/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala +++ b/query/src/main/scala/filodb/query/exec/PlanDispatcher.scala @@ -1,12 +1,12 @@ package filodb.query.exec -import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration import akka.actor.ActorRef import akka.pattern.ask import akka.util.Timeout import monix.eval.Task +import monix.execution.Scheduler import filodb.query.QueryResponse @@ -16,7 +16,7 @@ import filodb.query.QueryResponse */ trait PlanDispatcher extends java.io.Serializable { def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] } @@ -27,7 +27,7 @@ trait PlanDispatcher extends java.io.Serializable { case class ActorPlanDispatcher(target: ActorRef) extends PlanDispatcher { def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = { implicit val _ = Timeout(timeout) val fut = (target ? plan).map { diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala new file mode 100644 index 0000000000..116aa851b6 --- /dev/null +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -0,0 +1,131 @@ +package filodb.query.exec + +import com.softwaremill.sttp.akkahttp.AkkaHttpBackend +import com.softwaremill.sttp.circe._ +import com.typesafe.scalalogging.StrictLogging +import monix.eval.Task +import monix.execution.Scheduler +import monix.reactive.Observable +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration +import scala.sys.ShutdownHookThread + +import filodb.core.DatasetRef +import filodb.core.metadata.Column.ColumnType +import filodb.core.metadata.Dataset +import filodb.core.query._ +import filodb.core.store.ChunkSource +import filodb.memory.format.RowReader +import filodb.memory.format.ZeroCopyUTF8String._ +import filodb.query._ + +case class PromQlExec(id: String, + dispatcher: PlanDispatcher, + dataset: DatasetRef, params: PromQlInvocationParams, + submitTime: Long = System.currentTimeMillis()) + extends LeafExecPlan { + + protected def args: String = params.toString + import PromQlExec._ + + /** + * Limit on number of samples returned by this ExecPlan + */ + override def limit: Int = ??? + + /** + * Sub classes should override this method to provide a concrete + * implementation of the operation represented by this exec plan + * node + */ + override protected def doExecute(source: ChunkSource, dataset: Dataset, queryConfig: QueryConfig) + (implicit sched: Scheduler, timeout: FiniteDuration): Observable[RangeVector] = ??? + + /** + * Sub classes should implement this with schema of RangeVectors returned + * from doExecute() abstract method. + */ + override protected def schemaOfDoExecute(dataset: Dataset): ResultSchema = ??? + + override def execute(source: ChunkSource, + dataset: Dataset, + queryConfig: QueryConfig) + (implicit sched: Scheduler, + timeout: FiniteDuration): Task[QueryResponse] = { + + val queryResponse = PromQlExec.httpGet(params).map { response => + + response.unsafeBody match { + case Left(error) => QueryError(id, error.error) + case Right(response) => toQueryResponse(response.data, id) + } + + } + Task.fromFuture(queryResponse) + } + + def toQueryResponse(data: Data, id: String): QueryResponse = { + + val rangeVectors = data.result.map { r => + + val rv = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) + + override def rows: Iterator[RowReader] = { + r.values.iterator.map { v => + row.setLong(0, (v.timestamp * 1000)) + row.setDouble(1, v.value) + row + } + } + + override def numRows: Option[Int] = Some(r.values.size) + + } + SerializableRangeVector(rv, builder, recSchema, printTree(false)) + } + QueryResult(id, resultSchema, rangeVectors) + } + +} + +object PromQlExec extends StrictLogging{ + + import com.softwaremill.sttp._ + import io.circe.generic.auto._ + + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val recSchema = SerializableRangeVector.toSchema(columns) + val builder = SerializableRangeVector.toBuilder(recSchema) + val resultSchema = ResultSchema(columns, 1) + + // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . + // Needed to override Sampl case class Encoder. + import PromCirceSupport._ + implicit val backend = AkkaHttpBackend() + + ShutdownHookThread(shutdown()) + + def httpGet(params: PromQlInvocationParams)(implicit scheduler: Scheduler): + Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { + var urlParams = Map("query" -> params.promQl, "start" -> params.start, "end" -> params.end, "step" -> params.step, + "processFailure" -> params.processFailure) + if (params.spread.isDefined) + urlParams = urlParams + ("spread" -> params.spread.get) + + val endpoint = params.endpoint + val url = uri"$endpoint?$urlParams" + logger.debug("promqlexec url is {}", url) + sttp.get(url).response(asJson[SuccessResponse]).send() + } + + def shutdown(): Unit = + { + logger.info("Shutting PromQlExec http") + backend.close() + } +} + diff --git a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala index bc5c97b214..0d8c6568d1 100644 --- a/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala +++ b/query/src/main/scala/filodb/query/exec/StitchRvsExec.scala @@ -73,7 +73,7 @@ final case class StitchRvsExec(id: String, case (QueryResult(_, _, result), _) => result case (QueryError(_, ex), _) => throw ex }.toListL.map(_.flatten).map { srvs => - val groups = srvs.groupBy(_.key) + val groups = srvs.groupBy(_.key.labelValues) groups.mapValues { toMerge => val rows = StitchRvsExec.merge(toMerge.map(_.rows)) val key = toMerge.head.key diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala index e106d75c07..227e53c642 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinExecSpec.scala @@ -1,11 +1,11 @@ package filodb.query.exec -import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration import scala.util.Random import com.typesafe.config.ConfigFactory import monix.eval.Task +import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} @@ -27,7 +27,7 @@ class BinaryJoinExecSpec extends FunSpec with Matchers with ScalaFutures { val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = ??? } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala index ec1a39fd80..d82f68f219 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinGroupingSpec.scala @@ -1,11 +1,10 @@ package filodb.query.exec -import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration import scala.util.Random - import com.typesafe.config.ConfigFactory import monix.eval.Task +import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} @@ -33,7 +32,7 @@ class BinaryJoinGroupingSpec extends FunSpec with Matchers with ScalaFutures { val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = ??? } diff --git a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala index e92bf14d29..145d4ff16b 100644 --- a/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala +++ b/query/src/test/scala/filodb/query/exec/BinaryJoinSetOperatorSpec.scala @@ -1,22 +1,20 @@ package filodb.query.exec -import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration import scala.util.Random - import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler.Implicits.global import monix.reactive.Observable import org.scalatest.{FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures - import filodb.core.MetricsTestData import filodb.core.metadata.Column.ColumnType import filodb.core.query._ import filodb.memory.format.{RowReader, ZeroCopyUTF8String} import filodb.memory.format.ZeroCopyUTF8String._ import filodb.query._ +import monix.execution.Scheduler class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures { @@ -36,7 +34,7 @@ class BinaryJoinSetOperatorSpec extends FunSpec with Matchers with ScalaFutures val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = ??? } val resultSchema = ResultSchema(MetricsTestData.timeseriesDataset.infosFromIDs(0 to 1), 1) diff --git a/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala new file mode 100644 index 0000000000..88e35c77d5 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/InProcessPlanDispatcherSpec.scala @@ -0,0 +1,122 @@ +package filodb.query.exec + +import java.util.concurrent.{Executors, TimeUnit} + +import scala.collection.immutable +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.FiniteDuration + +import com.typesafe.config.{Config, ConfigFactory} +import monix.eval.Task +import monix.execution.Scheduler +import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.time.{Millis, Seconds, Span} + +import filodb.core.MetricsTestData.{builder, timeseriesDataset} +import filodb.core.TestData +import filodb.core.binaryrecord2.{RecordBuilder, RecordContainer} +import filodb.core.memstore.{FixedMaxPartitionsEvictionPolicy, SomeData, TimeSeriesMemStore} +import filodb.core.metadata.Dataset +import filodb.core.query.{ColumnFilter, Filter} +import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore} +import filodb.memory.MemFactory +import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} +import filodb.query._ + +class InProcessPlanDispatcherSpec extends FunSpec with Matchers with ScalaFutures with BeforeAndAfterAll { + + import ZeroCopyUTF8String._ + import filodb.core.{MachineMetricsData => MMD} + + override def beforeAll(): Unit = { + memStore.setup(timeseriesDataset, 0, TestData.storeConf) + memStore.ingest(timeseriesDataset.ref, 0, SomeData(container, 0)) + memStore.setup(MMD.dataset1, 0, TestData.storeConf) + memStore.ingest(MMD.dataset1.ref, 0, mmdSomeData) + memStore.setup(MMD.histDataset, 0, TestData.storeConf) + memStore.ingest(MMD.histDataset.ref, 0, MMD.records(MMD.histDataset, histData)) + memStore.setup(MMD.histMaxDS, 0, TestData.storeConf) + memStore.ingest(MMD.histMaxDS.ref, 0, MMD.records(MMD.histMaxDS, histMaxData)) + memStore.refreshIndexForTesting(timeseriesDataset.ref) + memStore.refreshIndexForTesting(MMD.dataset1.ref) + memStore.refreshIndexForTesting(MMD.histDataset.ref) + memStore.refreshIndexForTesting(MMD.histMaxDS.ref) + } + + override def afterAll(): Unit = { + memStore.shutdown() + } + + val queryId: String = "InProcessPlanDispatcherSpec" + + implicit val executor: ExecutionContext = ExecutionContext.fromExecutor(Executors.newFixedThreadPool(2)) + implicit val scheduler: Scheduler = Scheduler(executor) + implicit val timeout: FiniteDuration = FiniteDuration(5, TimeUnit.SECONDS) + implicit val defaultPatience: PatienceConfig = PatienceConfig(timeout = Span(30, Seconds), + interval = Span(250, Millis)) + + val dataset: Dataset = timeseriesDataset + + val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val queryConfig = new QueryConfig(config.getConfig("query")) + val policy = new FixedMaxPartitionsEvictionPolicy(20) + val memStore = new TimeSeriesMemStore(config, new NullColumnStore, new InMemoryMetaStore(), Some(policy)) + + val partKeyLabelValues: Map[String, String] = + Map("__name__"->"http_req_total", "job"->"myCoolService", "instance"->"someHost:8787") + val partTagsUTF8: Map[ZeroCopyUTF8String, ZeroCopyUTF8String] = + partKeyLabelValues.map { case (k, v) => (k.utf8, v.utf8) } + val now: Long = System.currentTimeMillis() + val numRawSamples = 1000 + val reportingInterval = 10000 + val tuples: immutable.IndexedSeq[(Long, Double)] = (numRawSamples until 0).by(-1).map { n => + (now - n * reportingInterval, n.toDouble) + } + + // NOTE: due to max-chunk-size in storeConf = 100, this will make (numRawSamples / 100) chunks + // Be sure to reset the builder; it is in an Object so static and shared amongst tests + builder.reset() + tuples.map { t => SeqRowReader(Seq(t._1, t._2, partTagsUTF8)) }.foreach(builder.addFromReader) + val container: RecordContainer = builder.allContainers.head + + val mmdBuilder = new RecordBuilder(MemFactory.onHeapFactory, MMD.dataset1.ingestionSchema) + val mmdTuples: Stream[Seq[Any]] = MMD.linearMultiSeries().take(100) + val mmdSomeData: SomeData = MMD.records(MMD.dataset1, mmdTuples) + val histData: Stream[Seq[Any]] = MMD.linearHistSeries().take(100) + val histMaxData: Stream[Seq[Any]] = MMD.histMax(histData) + + it ("inprocess dispatcher should execute and return monix task which in turn should return QueryResult") { + + val filters = Seq (ColumnFilter("__name__", Filter.Equals("http_req_total".utf8)), + ColumnFilter("job", Filter.Equals("myCoolService".utf8))) + + val dispatcher: PlanDispatcher = InProcessPlanDispatcher(dataset) + + val dummyDispatcher = DummyDispatcher(memStore, queryConfig) + + val execPlan1 = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + timeseriesDataset.ref, 0, filters, AllChunkScan, Nil) + val execPlan2 = SelectRawPartitionsExec("someQueryId", now, numRawSamples, dummyDispatcher, + timeseriesDataset.ref, 0, filters, AllChunkScan, Nil) + + val sep = StitchRvsExec(queryId, dispatcher, Seq(execPlan1, execPlan2)) + val result = dispatcher.dispatch(sep).runAsync.futureValue + + result match { + case e: QueryError => throw e.t + case r: QueryResult => + r.result.size shouldEqual 1 + r.result.head.numRows shouldEqual Some(numRawSamples) + } + } +} + +case class DummyDispatcher(memStore: TimeSeriesMemStore, queryConfig: QueryConfig) extends PlanDispatcher { + // run locally withing any check. + override def dispatch(plan: ExecPlan) + (implicit sched: Scheduler, + timeout: FiniteDuration): Task[QueryResponse] = { + plan.execute(memStore, timeseriesDataset, queryConfig) + } +} \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala index ee453d4bc9..a884a390ef 100644 --- a/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/MetadataExecSpec.scala @@ -1,11 +1,11 @@ package filodb.query.exec import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import monix.eval.Task +import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures @@ -67,7 +67,7 @@ class MetadataExecSpec extends FunSpec with Matchers with ScalaFutures with Befo val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = ??? } diff --git a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala new file mode 100644 index 0000000000..6c30c446b8 --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala @@ -0,0 +1,41 @@ +package filodb.query.exec + +import scala.concurrent.duration.FiniteDuration + +import monix.eval.Task +import monix.execution.Scheduler +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.metadata.{Dataset, DatasetOptions} +import filodb.query +import filodb.query.{Data, PromQlInvocationParams, QueryResponse, QueryResult, Sampl} + +class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { + + val timeseriesDataset = Dataset.make("timeseries", + Seq("tags:map"), + Seq("timestamp:ts", "value:double:detectDrops=true"), + Seq("timestamp"), + Seq.empty, + DatasetOptions(Seq("__name__", "job"), "__name__", "value")).get + + val dummyDispatcher = new PlanDispatcher { + override def dispatch(plan: ExecPlan) + (implicit sched: Scheduler, + timeout: FiniteDuration): Task[QueryResponse] = ??? + } + + it ("should convert Data to QueryResponse ") { + val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) + val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams("", "", 0, 0 , 0)) + val result = query.Result (Map("instance" ->"inst1"), Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))) + val res = exec.toQueryResponse(Data("vector", Seq(result)), "id") + res.isInstanceOf[QueryResult] shouldEqual true + val queryResult = res.asInstanceOf[QueryResult] + queryResult.result(0).numRows.get shouldEqual(3) + val data = queryResult.result.flatMap(x=>x.rows.map{ r => (r.getLong(0) , r.getDouble(1)) }.toList) + data.shouldEqual(expectedResult) + + } +} diff --git a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala index 36d16808b2..a13e868890 100644 --- a/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/SelectRawPartitionsExecSpec.scala @@ -1,15 +1,12 @@ package filodb.query.exec -import scala.concurrent.ExecutionContext import scala.concurrent.duration._ - import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSpec, Matchers} import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} - import filodb.core.{TestData, Types} import filodb.core.MetricsTestData._ import filodb.core.binaryrecord2.RecordBuilder @@ -20,11 +17,12 @@ import filodb.core.store.{AllChunkScan, InMemoryMetaStore, NullColumnStore, Time import filodb.memory.MemFactory import filodb.memory.format.{SeqRowReader, ZeroCopyUTF8String} import filodb.query._ +import monix.execution.Scheduler object SelectRawPartitionsExecSpec { val dummyDispatcher = new PlanDispatcher { override def dispatch(plan: ExecPlan) - (implicit sched: ExecutionContext, + (implicit sched: Scheduler, timeout: FiniteDuration): Task[QueryResponse] = ??? } diff --git a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala index 90ec776618..eca76bd0fb 100644 --- a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala +++ b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala @@ -2,7 +2,6 @@ package filodb.standalone import scala.collection.JavaConverters._ import scala.concurrent.duration._ - import akka.actor.ActorRef import akka.remote.testkit.{MultiNodeConfig, MultiNodeSpec} import akka.testkit.ImplicitSender @@ -21,13 +20,14 @@ import remote.RemoteStorage.{LabelMatcher, Query, ReadRequest, ReadResponse} import filodb.coordinator._ import filodb.coordinator.NodeClusterActor.{DatasetResourceSpec, IngestionSource} import filodb.coordinator.client.LocalClient +import filodb.coordinator.queryengine2.UnavailablePromQlQueryParams import filodb.core.DatasetRef import filodb.core.store.StoreConfig -import filodb.http.PromCirceSupport import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser -import filodb.prometheus.query.PrometheusModel.SuccessResponse import filodb.query.{QueryError, QueryResult => QueryResult2} +import filodb.query.PromCirceSupport +import filodb.query.SuccessResponse /** * A trait used for MultiJVM tests based on starting the standalone FiloServer using timeseries-dev config @@ -148,7 +148,7 @@ abstract class StandaloneMultiJvmSpec(config: MultiNodeConfig) extends MultiNode val logicalPlan = Parser.queryToLogicalPlan(query, queryTimestamp/1000) val curTime = System.currentTimeMillis - val result = client.logicalPlan2Query(dataset, logicalPlan) match { + val result = client.logicalPlan2Query(dataset, logicalPlan, UnavailablePromQlQueryParams ) match { case r: QueryResult2 => val vals = r.result.flatMap(_.rows.map { r => (r.getLong(0) - curTime, r.getDouble(1)) }) // info(s"result values were $vals") @@ -166,7 +166,7 @@ abstract class StandaloneMultiJvmSpec(config: MultiNodeConfig) extends MultiNode val logicalPlan = Parser.queryRangeToLogicalPlan(query1, TimeStepParams(startTime/1000, 60, endTime/1000)) var totalSamples = 0 - client.logicalPlan2Query(dataset, logicalPlan) match { + client.logicalPlan2Query(dataset, logicalPlan, UnavailablePromQlQueryParams) match { case r: QueryResult2 => // Transform range query vectors val map = r.result.map { rv => @@ -190,7 +190,7 @@ abstract class StandaloneMultiJvmSpec(config: MultiNodeConfig) extends MultiNode def printChunkMeta(client: LocalClient): Unit = { val chunkMetaQuery = "_filodb_chunkmeta_all(heap_usage{dc=\"DC0\",_ns=\"App-2\"})" val logicalPlan = Parser.queryRangeToLogicalPlan(chunkMetaQuery, TimeStepParams(0, 60, Int.MaxValue)) - client.logicalPlan2Query(dataset, logicalPlan) match { + client.logicalPlan2Query(dataset, logicalPlan, UnavailablePromQlQueryParams) match { case QueryResult2(_, schema, result) => result.foreach(rv => println(rv.prettyPrint())) case e: QueryError => fail(e.t) } From 1a2637d5dd4fc27354898a7e6a5d6586d2ba6678 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 8 Aug 2019 09:37:16 -0700 Subject: [PATCH 10/28] bug(query): RecordBuilder should be part of PromQlExec (#456) --- query/src/main/scala/filodb/query/exec/PromQlExec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index 116aa851b6..8b02e3e05b 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -28,6 +28,8 @@ case class PromQlExec(id: String, protected def args: String = params.toString import PromQlExec._ + val builder = SerializableRangeVector.toBuilder(recSchema) + /** * Limit on number of samples returned by this ExecPlan */ @@ -99,7 +101,6 @@ object PromQlExec extends StrictLogging{ val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)) val recSchema = SerializableRangeVector.toSchema(columns) - val builder = SerializableRangeVector.toBuilder(recSchema) val resultSchema = ResultSchema(columns, 1) // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . From 3cd6feb373e210c26490663a0a27c05eaba4be79 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Fri, 9 Aug 2019 09:44:03 -0700 Subject: [PATCH 11/28] feat(coordinator) : routing for instant queries (#459) --- .../queryengine2/QueryEngine.scala | 13 +++- .../queryengine2/QueryEngineSpec.scala | 71 ++++++++++++++++++- 2 files changed, 81 insertions(+), 3 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index 567ade399b..afcb92b46b 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -134,8 +134,17 @@ class QueryEngine(dataset: Dataset, if (failures.isEmpty) { generateLocalExecPlan(rootLogicalPlan, queryId, submitTime, options, querySpreadProvider) } else { - val routes = QueryRoutingPlanner.plan(failures, periodicSeriesTime, lookBackTime, - tsdbQueryParams.asInstanceOf[PromQlQueryParams].step * 1000) + val promQlQueryParams = tsdbQueryParams.asInstanceOf[PromQlQueryParams] + val routes : Seq[Route] = if (promQlQueryParams.start == promQlQueryParams.end) { // Instant Query + if (failures.forall(_.isRemote.equals(false))) { + Seq(RemoteRoute(Some(TimeRange(periodicSeriesTime.startInMillis, periodicSeriesTime.endInMillis)))) + } else { + Seq(LocalRoute(None)) + } + } else { + QueryRoutingPlanner.plan(failures, periodicSeriesTime, lookBackTime, + promQlQueryParams.step * 1000) + } logger.debug("Routes:" + routes) routeExecPlanMapper(routes, rootLogicalPlan, queryId, submitTime, options, querySpreadProvider, lookBackTime, tsdbQueryParams) diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index 3777155435..ee1c1caacd 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -64,7 +64,7 @@ class QueryEngineSpec extends FunSpec with Matchers { val raw2 = RawSeries(rangeSelector = intervalSelector, filters= f2, columns = Seq("value")) val windowed2 = PeriodicSeriesWithWindowing(raw2, from, 1000, to, 5000, RangeFunctionId.Rate) val summed2 = Aggregate(AggregationOperator.Sum, windowed2, Nil, Seq("job")) - val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 0, 1, 0, None) + val promQlQueryParams = PromQlQueryParams("sum(heap_usage)", 100, 1, 1000, None) it ("should generate ExecPlan for LogicalPlan") { // final logical plan @@ -505,4 +505,73 @@ class QueryEngineSpec extends FunSpec with Matchers { child.params.step shouldEqual 60 child.params.processFailure shouldEqual(false) } + + it("should not do routing for InstantQueries when there are local and remote failures") { + val to = 900 + val from = 900 + val lookBack = 300000 + val step = 1000 + val intervalSelector = IntervalSelector(from * 1000 - lookBack , to * 1000) // Lookback of 300 + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(910000, 1030000), false), FailureTimeRange("remote", datasetRef, + TimeRange(2000000, 2500000), true)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[ReduceAggregateExec] shouldEqual (true) + + val reduceAggregateExec = execPlan.asInstanceOf[ReduceAggregateExec] + + reduceAggregateExec.children.length shouldEqual (2) //default spread is 1 so 2 shards + + reduceAggregateExec.children.foreach { l1 => + l1.isInstanceOf[SelectRawPartitionsExec] shouldEqual true + l1.rangeVectorTransformers.size shouldEqual 2 + l1.rangeVectorTransformers(0).isInstanceOf[PeriodicSamplesMapper] shouldEqual true + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].start shouldEqual from *1000 + l1.rangeVectorTransformers(0).asInstanceOf[PeriodicSamplesMapper].end shouldEqual to * 1000 + l1.rangeVectorTransformers(1).isInstanceOf[AggregateMapReduce] shouldEqual true + } + } + + it("should generate PromQlExec for InstantQueries when all failures are local") { + val to = 900 + val from = 900 + val lookBack = 300000 + val step = 1000 + val intervalSelector = IntervalSelector(from * 1000 - lookBack , to * 1000) // Lookback of 300 + val raw = RawSeries(rangeSelector = intervalSelector, filters = f1, columns = Seq("value")) + val windowed = PeriodicSeriesWithWindowing(raw, from * 1000, step * 1000, to * 1000, 5000, RangeFunctionId.Rate) + val summed = Aggregate(AggregationOperator.Sum, windowed, Nil, Seq("job")) + val promQlQueryParams = PromQlQueryParams("dummy query", from, step, to, None) + + val failureProvider = new FailureProvider { + override def getFailures(datasetRef: DatasetRef, queryTimeRange: TimeRange): Seq[FailureTimeRange] = { + Seq(FailureTimeRange("local", datasetRef, + TimeRange(910000, 1030000), false), FailureTimeRange("remote", datasetRef, + TimeRange(2000000, 2500000), false)) + } + } + + val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) + + execPlan.isInstanceOf[PromQlExec] shouldEqual (true) + + val child = execPlan.asInstanceOf[PromQlExec] + child.params.start shouldEqual from + child.params.end shouldEqual to + child.params.step shouldEqual step + child.params.processFailure shouldEqual(false) + } } From b16159ac9c10c75ca0931296f2a8be204334dab8 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 13 Aug 2019 09:56:21 -0700 Subject: [PATCH 12/28] bug[memory]: Guard BlockManager data structures with the lock. This should protect blocks from being reclaimed before they're fully switched to the "in use" state with a timestamp. (#460) --- .../main/scala/filodb.memory/BlockManager.scala | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 8cc17026b2..4e1d39d3ff 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -216,12 +216,22 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, def timeBuckets: Seq[Long] = usedBlocksTimeOrdered.keySet.asScala.toSeq def markBucketedBlocksReclaimable(upTo: Long): Unit = { - usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => - list.asScala.foreach(_.markReclaimable) + lock.lock() + try { + usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => + list.asScala.foreach(_.markReclaimable) + } + } finally { + lock.unlock() } } - def hasTimeBucket(bucket: Long): Boolean = usedBlocksTimeOrdered.containsKey(bucket) + def hasTimeBucket(bucket: Long): Boolean = { + lock.lock() + val result = usedBlocksTimeOrdered.containsKey(bucket) + lock.unlock() + result + } /** * Used during testing only to try and reclaim all existing blocks From 9ee7418e3e0b864adc1b08db13366d3e7581c886 Mon Sep 17 00:00:00 2001 From: mishra-arun <50001250+mishra-arun@users.noreply.github.com> Date: Tue, 13 Aug 2019 10:21:55 -0700 Subject: [PATCH 13/28] bug(query) Fix for handling different schemas for instant query responses (#461) --- .../prometheus/query/PrometheusModel.scala | 7 +++-- .../filodb/query/PromQueryResponse.scala | 2 +- .../scala/filodb/query/exec/PromQlExec.scala | 30 ++++++++++--------- .../filodb/query/exec/PromQlExecSpec.scala | 18 +++++++++-- .../standalone/StandaloneMultiJvmSpec.scala | 2 +- 5 files changed, 39 insertions(+), 20 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala index 0c75c0f4f7..02f6a5ae67 100644 --- a/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala +++ b/prometheus/src/main/scala/filodb/prometheus/query/PrometheusModel.scala @@ -89,9 +89,12 @@ object PrometheusModel { // remove NaN in HTTP results // Known Issue: Until we support NA in our vectors, we may not be able to return NaN as an end-of-time-series // in HTTP raw query results. - srv.rows.filter(!_.getDouble(1).isNaN).map { r => + Some( + srv.rows.filter(!_.getDouble(1).isNaN).map { r => Sampl(r.getLong(0) / 1000, r.getDouble(1)) - }.toSeq + }.toSeq + ), + None ) } diff --git a/query/src/main/scala/filodb/query/PromQueryResponse.scala b/query/src/main/scala/filodb/query/PromQueryResponse.scala index d36d3c8094..06bea88b3a 100644 --- a/query/src/main/scala/filodb/query/PromQueryResponse.scala +++ b/query/src/main/scala/filodb/query/PromQueryResponse.scala @@ -12,7 +12,7 @@ final case class ExplainPlanResponse(debugInfo: Seq[String], status: String = "s final case class Data(resultType: String, result: Seq[Result]) -final case class Result(metric: Map[String, String], values: Seq[Sampl]) +final case class Result(metric: Map[String, String], values: Option[Seq[Sampl]], value: Option[Sampl] = None) /** * Metric value for a given timestamp diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index 8b02e3e05b..72f931d85d 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -59,7 +59,7 @@ case class PromQlExec(id: String, response.unsafeBody match { case Left(error) => QueryError(id, error.error) - case Right(response) => toQueryResponse(response.data, id) + case Right(successResponse) => toQueryResponse(successResponse.data, id) } } @@ -70,23 +70,25 @@ case class PromQlExec(id: String, val rangeVectors = data.result.map { r => - val rv = new RangeVector { - val row = new TransientRow() + val samples = r.values.getOrElse(Seq(r.value.get)) - override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) + val rv = new RangeVector { + val row = new TransientRow() - override def rows: Iterator[RowReader] = { - r.values.iterator.map { v => - row.setLong(0, (v.timestamp * 1000)) - row.setDouble(1, v.value) - row - } - } - - override def numRows: Option[Int] = Some(r.values.size) + override def key: RangeVectorKey = CustomRangeVectorKey(r.metric.map (m => m._1.utf8 -> m._2.utf8)) + override def rows: Iterator[RowReader] = { + samples.iterator.map { v => + row.setLong(0, v.timestamp * 1000) + row.setDouble(1, v.value) + row + } } - SerializableRangeVector(rv, builder, recSchema, printTree(false)) + + override def numRows: Option[Int] = Option(samples.size) + + } + SerializableRangeVector(rv, builder, recSchema, printTree(useNewline = false)) } QueryResult(id, resultSchema, rangeVectors) } diff --git a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala index 6c30c446b8..1fb431cf03 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala @@ -26,10 +26,10 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { timeout: FiniteDuration): Task[QueryResponse] = ??? } - it ("should convert Data to QueryResponse ") { + it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams("", "", 0, 0 , 0)) - val result = query.Result (Map("instance" ->"inst1"), Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))) + val result = query.Result (Map("instance" ->"inst1"), Some(Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))), None) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id") res.isInstanceOf[QueryResult] shouldEqual true val queryResult = res.asInstanceOf[QueryResult] @@ -38,4 +38,18 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { data.shouldEqual(expectedResult) } + + it ("should convert vector Data to QueryResponse ") { + val expectedResult = List((1000000, 1.0)) + val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams("", "", 0, 0 , 0)) + val result = query.Result (Map("instance" ->"inst1"), None, Some(Sampl(1000, 1))) + val res = exec.toQueryResponse(Data("vector", Seq(result)), "id") + res.isInstanceOf[QueryResult] shouldEqual true + val queryResult = res.asInstanceOf[QueryResult] + queryResult.result(0).numRows.get shouldEqual(1) + val data = queryResult.result.flatMap(x=>x.rows.map{ r => (r.getLong(0) , r.getDouble(1)) }.toList) + data.shouldEqual(expectedResult) + + } + } diff --git a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala index eca76bd0fb..bee71f7d8c 100644 --- a/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala +++ b/standalone/src/multi-jvm/scala/filodb/standalone/StandaloneMultiJvmSpec.scala @@ -205,7 +205,7 @@ abstract class StandaloneMultiJvmSpec(config: MultiNodeConfig) extends MultiNode val url = uri"http://localhost:8080/promql/prometheus/api/v1/query?query=$query&time=${queryTimestamp/1000}" info(s"Querying: $url") val result1 = sttp.get(url).response(asJson[SuccessResponse]).send().futureValue.unsafeBody.right.get.data.result - val result = result1.flatMap(_.values.map { d => (d.timestamp, d.value) }) + val result = result1.flatMap(_.values.get.map { d => (d.timestamp, d.value) }) info(s"result values were $result") result.length should be > 0 val sum = result.map(_._2).sum From ff4f83cde5a7c22da14b374898e41e546a799ce2 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Wed, 14 Aug 2019 14:12:11 -0700 Subject: [PATCH 14/28] misc(memory, core): Log block reclaim as info; log corrupt vector address and skip. (#464) * misc(memory, core): Log block reclaim as info; log corrupt vector address and skip. * Always update the last end time which skipping vectors. --- .../scala/filodb.core/store/ChunkSetInfo.scala | 14 ++++++++++---- memory/src/main/scala/filodb.memory/Block.scala | 2 +- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index d0e14476bb..34e4de5358 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -396,10 +396,16 @@ extends Iterator[ChunkQueryInfo] { // Add if next chunkset is within window and not empty. Otherwise keep going if (curWindowStart <= next.endTime && next.numRows > 0) { val tsVector = next.vectorPtr(tsColID) - val tsReader = vectors.LongBinaryVector(tsVector) - val valueVector = next.vectorPtr(rv.valueColID) - val valueReader = rv.partition.chunkReader(rv.valueColID, valueVector) - windowInfos += ChunkQueryInfo(next.infoAddr, tsVector, tsReader, valueVector, valueReader) + try { + val tsReader = vectors.LongBinaryVector(tsVector) + val valueVector = next.vectorPtr(rv.valueColID) + val valueReader = rv.partition.chunkReader(rv.valueColID, valueVector) + windowInfos += ChunkQueryInfo(next.infoAddr, tsVector, tsReader, valueVector, valueReader) + } catch { + case e: Throwable => { + ChunkSetInfo.log.error(s"Corrupt vector at ${java.lang.Long.toHexString(tsVector)}", e) + } + } lastEndTime = Math.max(next.endTime, lastEndTime) } } diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index ca59337e5b..a4fabd6b8e 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -93,7 +93,7 @@ trait ReusableMemory extends StrictLogging { * Marks this memory as free and calls reclaimListener for every piece of metadata. */ protected def free() = { - logger.debug(s"Reclaiming block at ${jLong.toHexString(address)}...") + logger.info(s"Reclaiming block at ${jLong.toHexString(address)}...") reclaimWithMetadata() } From 700bc999ee27f33af06723a7347a2f7a2123d030 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 15 Aug 2019 10:59:18 -0700 Subject: [PATCH 15/28] Trusty build in travis --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 3266eedd79..e2e672db87 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,4 +1,5 @@ language: scala +dist: trusty env: global: _JAVA_OPTIONS="-Dakka.test.timefactor=3 -XX:MaxMetaspaceSize=256m" From f6b21d602cb10d86a669d669cd37c01dac78cfe2 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Thu, 15 Aug 2019 14:02:24 -0700 Subject: [PATCH 16/28] incorrect result type for QueryResult (#467) --- .../main/scala/filodb/query/ResultTypes.scala | 6 +- .../scala/filodb/query/ResultTypesSpec.scala | 77 +++++++++++++++++++ 2 files changed, 80 insertions(+), 3 deletions(-) create mode 100644 query/src/test/scala/filodb/query/ResultTypesSpec.scala diff --git a/query/src/main/scala/filodb/query/ResultTypes.scala b/query/src/main/scala/filodb/query/ResultTypes.scala index 6a31ff2cf6..d14019aaae 100644 --- a/query/src/main/scala/filodb/query/ResultTypes.scala +++ b/query/src/main/scala/filodb/query/ResultTypes.scala @@ -38,11 +38,11 @@ final case class QueryResult(id: String, result: Seq[RangeVector]) extends QueryResponse { def resultType: QueryResultType = { result match { - case Seq(one) => if (one.numRows.contains(1)) QueryResultType.Scalar else QueryResultType.RangeVectors + case Nil => QueryResultType.RangeVectors + case Seq(one) if one.key.labelValues.isEmpty && one.numRows.contains(1) => QueryResultType.Scalar case many: Seq[RangeVector] => if (many.forall(_.numRows.contains(1))) QueryResultType.InstantVector - else QueryResultType.RangeVectors + else QueryResultType.RangeVectors } } } - diff --git a/query/src/test/scala/filodb/query/ResultTypesSpec.scala b/query/src/test/scala/filodb/query/ResultTypesSpec.scala new file mode 100644 index 0000000000..ef4f92c3e1 --- /dev/null +++ b/query/src/test/scala/filodb/query/ResultTypesSpec.scala @@ -0,0 +1,77 @@ +package filodb.query + +import org.scalatest.{FunSpec, Matchers} +import org.scalatest.concurrent.ScalaFutures + +import filodb.core.metadata.Column.ColumnType +import filodb.core.query._ +import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.query.exec.TransientRow + +class ResultTypesSpec extends FunSpec with Matchers with ScalaFutures { + + val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), + ColumnInfo("value", ColumnType.DoubleColumn)) + val resultSchema = ResultSchema(columns, 1) + val ignoreKey = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) + it("should have result type RangeVectors when multiple rows exist ") { + val rv = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = ignoreKey + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d), + new TransientRow(2L, 5.1d)).iterator + + override def numRows: Option[Int] = Option(rows.size) + + } + val queryResult = QueryResult("id:1", resultSchema, Seq(rv)) + queryResult.resultType.toString shouldEqual ("RangeVectors") + } + + it("should have result type InstantVector when there is only one row per RangeVector") { + val rv1 = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = ignoreKey + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d)).toIterator + override def numRows: Option[Int] = Option(rows.size) + + } + + val rv2 = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = ignoreKey + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 9.4d)).toIterator + override def numRows: Option[Int] = Option(rows.size) + + } + val queryResult = QueryResult("id:1", resultSchema, Seq(rv1, rv2)) + + queryResult.resultType shouldEqual(QueryResultType.InstantVector) + } + + it("should have result type Scalar when there is no RangeVectorKey") { + val rv1 = new RangeVector { + val row = new TransientRow() + + override def key: RangeVectorKey = CustomRangeVectorKey(Map.empty) + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3.3d)).toIterator + override def numRows: Option[Int] = Option(rows.size) + + } + + val queryResult = QueryResult("id:1", resultSchema, Seq(rv1)) + queryResult.resultType shouldEqual(QueryResultType.Scalar) + } +} \ No newline at end of file From 03fc68857e46db358a3d5262e5908bdac2f9ce96 Mon Sep 17 00:00:00 2001 From: mishra-arun <50001250+mishra-arun@users.noreply.github.com> Date: Thu, 15 Aug 2019 16:47:17 -0700 Subject: [PATCH 17/28] bug(query) : Replace AkkaHttpBackend with AsyncHttpClientFutureBackend for PromQLExec (#468) --- .../queryengine2/QueryEngine.scala | 13 ++++------ .../queryengine2/QueryEngineSpec.scala | 26 +++++++++++-------- project/FiloBuild.scala | 8 +++--- .../filodb/query/PromQlInvocationParams.scala | 6 +++-- .../scala/filodb/query/exec/PromQlExec.scala | 16 ++++++++---- .../filodb/query/exec/PromQlExecSpec.scala | 5 ++-- 6 files changed, 42 insertions(+), 32 deletions(-) diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index afcb92b46b..fbdd351354 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -3,7 +3,7 @@ package filodb.coordinator.queryengine2 import java.util.UUID import java.util.concurrent.ThreadLocalRandom -import scala.concurrent.duration.FiniteDuration +import scala.concurrent.duration._ import akka.actor.ActorRef import com.typesafe.config.{Config, ConfigFactory} @@ -82,13 +82,10 @@ class QueryEngine(dataset: Dataset, case route: RemoteRoute => val timeRange = route.timeRange.get val queryParams = tsdbQueryParams.asInstanceOf[PromQlQueryParams] - val endpoint = queryEngineConfig.isEmpty() match { - case false => queryEngineConfig.getString("routing.buddy.http.endpoint") - case _ => "" - } - - val promQlInvocationParams = PromQlInvocationParams(endpoint, queryParams.promQl, (timeRange.startInMillis - /1000), queryParams.step, (timeRange.endInMillis / 1000), queryParams.spread, false) + val routingConfig = queryEngineConfig.getConfig("routing") + val promQlInvocationParams = PromQlInvocationParams(routingConfig, queryParams.promQl, + (timeRange.startInMillis / 1000), queryParams.step, (timeRange.endInMillis / 1000), queryParams.spread, + false) logger.debug("PromQlExec params:" + promQlInvocationParams) PromQlExec(queryId, InProcessPlanDispatcher(dataset), dataset.ref, promQlInvocationParams, submitTime) } diff --git a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala index ee1c1caacd..e6b8c28ee0 100644 --- a/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala +++ b/coordinator/src/test/scala/filodb.coordinator/queryengine2/QueryEngineSpec.scala @@ -5,9 +5,10 @@ import akka.testkit.TestProbe import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} - import scala.concurrent.duration.FiniteDuration +import com.typesafe.config.ConfigFactory + import filodb.coordinator.ShardMapper import filodb.coordinator.client.QueryCommands._ import filodb.core.{DatasetRef, MetricsTestData, SpreadChange} @@ -16,7 +17,7 @@ import filodb.core.store.TimeRangeChunkScan import filodb.prometheus.ast.TimeStepParams import filodb.prometheus.parse.Parser import filodb.query -import filodb.query.{_} +import filodb.query._ import filodb.query.exec._ class QueryEngineSpec extends FunSpec with Matchers { @@ -38,6 +39,9 @@ class QueryEngineSpec extends FunSpec with Matchers { val engine = new QueryEngine(dataset, mapperRef, EmptyFailureProvider) + val queryEngineConfigString = "routing {\n buddy {\n http {\n timeout = 10.seconds\n }\n }\n}" + + val queryEngineConfig = ConfigFactory.parseString(queryEngineConfigString) /* This is the PromQL @@ -295,7 +299,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) @@ -320,7 +324,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[StitchRvsExec] shouldEqual (true) @@ -367,7 +371,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) @@ -390,7 +394,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) @@ -413,7 +417,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) @@ -441,7 +445,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } //900K to 1020K and 1020+60 k to 2000K - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[StitchRvsExec] shouldEqual (true) @@ -493,7 +497,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) @@ -525,7 +529,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[ReduceAggregateExec] shouldEqual (true) @@ -563,7 +567,7 @@ class QueryEngineSpec extends FunSpec with Matchers { } } - val engine = new QueryEngine(dataset, mapperRef, failureProvider) + val engine = new QueryEngine(dataset, mapperRef, failureProvider, StaticSpreadProvider(), queryEngineConfig) val execPlan = engine.materialize(summed, QueryOptions(), promQlQueryParams) execPlan.isInstanceOf[PromQlExec] shouldEqual (true) diff --git a/project/FiloBuild.scala b/project/FiloBuild.scala index 5b7cf23363..9395c858e3 100644 --- a/project/FiloBuild.scala +++ b/project/FiloBuild.scala @@ -235,10 +235,10 @@ object FiloBuild extends Build { ) lazy val queryDeps = commonDeps ++ Seq( - "com.tdunning" % "t-digest" % "3.1", - "com.softwaremill.sttp" %% "circe" % sttpVersion , - "com.softwaremill.sttp" %% "akka-http-backend" % sttpVersion, - "com.softwaremill.sttp" %% "core" % sttpVersion, + "com.tdunning" % "t-digest" % "3.1", + "com.softwaremill.sttp" %% "circe" % sttpVersion , + "com.softwaremill.sttp" %% "async-http-client-backend-future" % sttpVersion, + "com.softwaremill.sttp" %% "core" % sttpVersion, circeGeneric, scalaxyDep ) diff --git a/query/src/main/scala/filodb/query/PromQlInvocationParams.scala b/query/src/main/scala/filodb/query/PromQlInvocationParams.scala index ffa5c5f43b..7d9a376aba 100644 --- a/query/src/main/scala/filodb/query/PromQlInvocationParams.scala +++ b/query/src/main/scala/filodb/query/PromQlInvocationParams.scala @@ -1,5 +1,7 @@ package filodb.query -case class PromQlInvocationParams(endpoint: String, promQl: String, start: Long, step: Long, end: Long, - spread: Option[Int] = None, processFailure: Boolean = true) +import com.typesafe.config.Config + +case class PromQlInvocationParams(config: Config, promQl: String, start: Long, step: Long, end: Long, + spread: Option[Int] = None, processFailure: Boolean = true) diff --git a/query/src/main/scala/filodb/query/exec/PromQlExec.scala b/query/src/main/scala/filodb/query/exec/PromQlExec.scala index 72f931d85d..2843d83e06 100644 --- a/query/src/main/scala/filodb/query/exec/PromQlExec.scala +++ b/query/src/main/scala/filodb/query/exec/PromQlExec.scala @@ -1,13 +1,13 @@ package filodb.query.exec -import com.softwaremill.sttp.akkahttp.AkkaHttpBackend +import com.softwaremill.sttp.asynchttpclient.future.AsyncHttpClientFutureBackend import com.softwaremill.sttp.circe._ import com.typesafe.scalalogging.StrictLogging import monix.eval.Task import monix.execution.Scheduler import monix.reactive.Observable import scala.concurrent.Future -import scala.concurrent.duration.FiniteDuration +import scala.concurrent.duration._ import scala.sys.ShutdownHookThread import filodb.core.DatasetRef @@ -99,6 +99,7 @@ object PromQlExec extends StrictLogging{ import com.softwaremill.sttp._ import io.circe.generic.auto._ + import net.ceedubs.ficus.Ficus._ val columns: Seq[ColumnInfo] = Seq(ColumnInfo("timestamp", ColumnType.LongColumn), ColumnInfo("value", ColumnType.DoubleColumn)) @@ -108,21 +109,26 @@ object PromQlExec extends StrictLogging{ // DO NOT REMOVE PromCirceSupport import below assuming it is unused - Intellij removes it in auto-imports :( . // Needed to override Sampl case class Encoder. import PromCirceSupport._ - implicit val backend = AkkaHttpBackend() + implicit val backend = AsyncHttpClientFutureBackend() ShutdownHookThread(shutdown()) def httpGet(params: PromQlInvocationParams)(implicit scheduler: Scheduler): Future[Response[scala.Either[DeserializationError[io.circe.Error], SuccessResponse]]] = { + val endpoint = params.config.as[Option[String]]("buddy.http.endpoint").get + val readTimeout = params.config.as[Option[FiniteDuration]]("buddy.http.timeout").getOrElse(60.seconds) var urlParams = Map("query" -> params.promQl, "start" -> params.start, "end" -> params.end, "step" -> params.step, "processFailure" -> params.processFailure) if (params.spread.isDefined) urlParams = urlParams + ("spread" -> params.spread.get) - val endpoint = params.endpoint val url = uri"$endpoint?$urlParams" logger.debug("promqlexec url is {}", url) - sttp.get(url).response(asJson[SuccessResponse]).send() + sttp + .get(url) + .readTimeout(readTimeout) + .response(asJson[SuccessResponse]) + .send() } def shutdown(): Unit = diff --git a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala index 1fb431cf03..00ad391132 100644 --- a/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala +++ b/query/src/test/scala/filodb/query/exec/PromQlExecSpec.scala @@ -2,6 +2,7 @@ package filodb.query.exec import scala.concurrent.duration.FiniteDuration +import com.typesafe.config.ConfigFactory import monix.eval.Task import monix.execution.Scheduler import org.scalatest.{FunSpec, Matchers} @@ -28,7 +29,7 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { it ("should convert matrix Data to QueryResponse ") { val expectedResult = List((1000000, 1.0), (2000000, 2.0), (3000000, 3.0)) - val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams("", "", 0, 0 , 0)) + val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams(ConfigFactory.empty(), "", 0, 0 , 0)) val result = query.Result (Map("instance" ->"inst1"), Some(Seq(Sampl(1000, 1), Sampl(2000, 2), Sampl(3000, 3))), None) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id") res.isInstanceOf[QueryResult] shouldEqual true @@ -41,7 +42,7 @@ class PromQlExecSpec extends FunSpec with Matchers with ScalaFutures { it ("should convert vector Data to QueryResponse ") { val expectedResult = List((1000000, 1.0)) - val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams("", "", 0, 0 , 0)) + val exec = PromQlExec("test", dummyDispatcher, timeseriesDataset.ref, PromQlInvocationParams(ConfigFactory.empty(), "", 0, 0 , 0)) val result = query.Result (Map("instance" ->"inst1"), None, Some(Sampl(1000, 1))) val res = exec.toQueryResponse(Data("vector", Seq(result)), "id") res.isInstanceOf[QueryResult] shouldEqual true From 3af01b7e0ef3bf43c23605cc4e27a17345ed90da Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 26 Aug 2019 11:02:47 -0700 Subject: [PATCH 18/28] debug(memory): Add logging around time-bucketed reclaims --- memory/src/main/scala/filodb.memory/BlockManager.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 4e1d39d3ff..4d1948aa95 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -184,6 +184,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, while ( reclaimed < num && timeOrderedListIt.hasNext ) { val entry = timeOrderedListIt.next + logger.info(s"timeBlockReclaim: Attempting to reclaim time ordered block list at t=${}") reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric) // If the block list is now empty, remove it from tree map if (entry.getValue.isEmpty) timeOrderedListIt.remove() @@ -218,6 +219,9 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, def markBucketedBlocksReclaimable(upTo: Long): Unit = { lock.lock() try { + logger.info(s"timeBlockReclaim: Marking ($upTo) - this is -${(System.currentTimeMillis - upTo)/3600000}hrs") + val keys = usedBlocksTimeOrdered.headMap(upTo).keySet.asScala + logger.info(s"timeBlockReclaim: Marking lists $keys as reclaimable") usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => list.asScala.foreach(_.markReclaimable) } From f1032406e3e4ef628a582e0d3c0c689d3b99be3f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Wed, 28 Aug 2019 21:04:25 -0700 Subject: [PATCH 19/28] feat(memory, core): BlockDetective: detailed block corruption ownership/reclaim analysis (#482) --- .../scala/filodb.coordinator/QueryActor.scala | 5 ++ .../memstore/DemandPagedChunkStore.scala | 4 ++ .../scala/filodb.core/memstore/MemStore.scala | 7 ++- .../memstore/TimeSeriesMemStore.scala | 3 + .../memstore/TimeSeriesShard.scala | 11 +++- .../filodb.core/store/ChunkSetInfo.scala | 13 +++- .../src/test/scala/filodb.core/TestData.scala | 3 +- .../downsample/ShardDownsamplerSpec.scala | 5 +- .../memstore/PartitionSetSpec.scala | 2 +- .../memstore/TimeSeriesPartitionSpec.scala | 14 ++--- .../src/main/scala/filodb.memory/Block.scala | 21 +++++-- .../scala/filodb.memory/BlockDetective.scala | 41 +++++++++++++ .../scala/filodb.memory/BlockManager.scala | 59 ++++++++++++++++--- .../filodb.memory/BlockMemFactoryPool.scala | 25 ++++++-- .../main/scala/filodb.memory/MemFactory.scala | 10 +++- .../PageAlignedBlockManagerSpec.scala | 36 +++++++++++ .../format/vectors/IntBinaryVectorTest.scala | 2 +- .../rangefn/AggrOverTimeFunctionsSpec.scala | 6 +- 18 files changed, 228 insertions(+), 39 deletions(-) create mode 100644 memory/src/main/scala/filodb.memory/BlockDetective.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala index d5cf1e848f..eeb5e0a831 100644 --- a/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala +++ b/coordinator/src/main/scala/filodb.coordinator/QueryActor.scala @@ -16,6 +16,7 @@ import filodb.core._ import filodb.core.memstore.{FiloSchedulers, MemStore, TermInfo} import filodb.core.metadata.Dataset import filodb.core.query.ColumnFilter +import filodb.core.store.CorruptVectorException import filodb.query._ import filodb.query.exec.ExecPlan @@ -104,6 +105,10 @@ final class QueryActor(memStore: MemStore, case e: QueryError => queryErrors.increment logger.debug(s"queryId ${q.id} Normal QueryError returned from query execution: $e") + e.t match { + case cve: CorruptVectorException => memStore.analyzeAndLogCorruptPtr(dataset.ref, cve) + case t: Throwable => + } } span.finish() }(queryScheduler).recover { case ex => diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index 66297c11bf..e98843f516 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -44,12 +44,16 @@ extends RawToPartitionMaker with StrictLogging { import TimeSeriesShard._ import collection.JavaConverters._ + private val baseContext = Map("dataset" -> tsShard.dataset.name, + "shard" -> tsShard.shardNum.toString) + private def getMemFactory(bucket: Long): BlockMemFactory = { val factory = memFactories.get(bucket) if (factory == UnsafeUtils.ZeroPointer) { val newFactory = new BlockMemFactory(blockManager, Some(bucket), tsShard.dataset.blockMetaSize, + baseContext ++ Map("bucket" -> bucket.toString), markFullBlocksAsReclaimable = true) memFactories.put(bucket, newFactory) newFactory diff --git a/core/src/main/scala/filodb.core/memstore/MemStore.scala b/core/src/main/scala/filodb.core/memstore/MemStore.scala index ac7e4e6da5..3000ab1e75 100644 --- a/core/src/main/scala/filodb.core/memstore/MemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/MemStore.scala @@ -13,7 +13,7 @@ import filodb.core.downsample.DownsampleConfig import filodb.core.metadata.{Column, Dataset} import filodb.core.metadata.Column.ColumnType._ import filodb.core.query.ColumnFilter -import filodb.core.store.{ChunkSource, ColumnStore, MetaStore, StoreConfig} +import filodb.core.store._ import filodb.memory.MemFactory import filodb.memory.format.{vectors => bv, _} @@ -195,6 +195,11 @@ trait MemStore extends ChunkSource { */ def refreshIndexForTesting(dataset: DatasetRef): Unit + /** + * Analyzes a corrupt pointer/vector for reclaim and ownership history, and logs details + */ + def analyzeAndLogCorruptPtr(ref: DatasetRef, cve: CorruptVectorException): Unit + /** * WARNING: truncates all the data in the memstore for the given dataset, and also the data * in any underlying ChunkSink too. diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala index b52b8b8877..423918e1e4 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesMemStore.scala @@ -240,6 +240,9 @@ extends MemStore with StrictLogging { def groupsInDataset(dataset: Dataset): Int = datasets.get(dataset.ref).map(_.values.asScala.head.storeConfig.groupsPerShard).getOrElse(1) + def analyzeAndLogCorruptPtr(ref: DatasetRef, cve: CorruptVectorException): Unit = + getShard(ref, cve.shard).get.analyzeAndLogCorruptPtr(cve) + def reset(): Unit = { datasets.clear() downsamplePublishers.valuesIterator.foreach { _.stop() } diff --git a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala index 24b1022cf2..8ed6a88a2a 100644 --- a/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala +++ b/core/src/main/scala/filodb.core/memstore/TimeSeriesShard.scala @@ -258,13 +258,15 @@ class TimeSeriesShard(val dataset: Dataset, private[memstore] final val partSetLock = new StampedLock // The off-heap block store used for encoded chunks + private val context = Map("dataset" -> dataset.ref.dataset, "shard" -> shardNum.toString) private val blockStore = new PageAlignedBlockManager(blockMemorySize, shardStats.memoryStats, reclaimListener, storeConfig.numPagesPerBlock) - private val blockFactoryPool = new BlockMemFactoryPool(blockStore, dataset.blockMetaSize) + private val blockFactoryPool = new BlockMemFactoryPool(blockStore, dataset.blockMetaSize, context) // Each shard has a single ingestion stream at a time. This BlockMemFactory is used for buffer overflow encoding // strictly during ingest() and switchBuffers(). - private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, None, dataset.blockMetaSize, true) + private[core] val overflowBlockFactory = new BlockMemFactory(blockStore, None, dataset.blockMetaSize, + context ++ Map("overflow" -> "true"), true) val partitionMaker = new DemandPagedChunkStore(this, blockStore, chunkRetentionHours) private val partKeyBuilder = new RecordBuilder(MemFactory.onHeapFactory, dataset.partKeySchema, @@ -771,7 +773,7 @@ class TimeSeriesShard(val dataset: Dataset, .withTag("shard", shardNum).start() // Only allocate the blockHolder when we actually have chunks/partitions to flush - val blockHolder = blockFactoryPool.checkout() + val blockHolder = blockFactoryPool.checkout(Map("flushGroup" -> flushGroup.groupNum.toString)) // This initializes the containers for the downsample records. Yes, we create new containers // and not reuse them at the moment and there is allocation for every call of this method @@ -1247,6 +1249,9 @@ class TimeSeriesShard(val dataset: Dataset, id } + def analyzeAndLogCorruptPtr(cve: CorruptVectorException): Unit = + logger.error(cve.getMessage + "\n" + BlockDetective.stringReport(cve.ptr, blockStore, blockFactoryPool)) + /** * Check and evict partitions to free up memory and heap space. NOTE: This must be called in the ingestion * stream so that there won't be concurrent other modifications. Ideally this is called when trying to add partitions diff --git a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala index 34e4de5358..3fd54a9569 100644 --- a/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala +++ b/core/src/main/scala/filodb.core/store/ChunkSetInfo.scala @@ -337,6 +337,13 @@ final case class ChunkQueryInfo(infoPtr: NativePointer, def info: ChunkSetInfo = ChunkSetInfo(infoPtr) } +final case class CorruptVectorException(ptr: NativePointer, + chunkStartTime: Long, + partKeyString: String, + shard: Int, + innerErr: Throwable) extends +Exception(f"CorruptVector at 0x$ptr%016x startTime=$chunkStartTime shard=$shard partition=$partKeyString", innerErr) + /** * A sliding window based iterator over the chunks needed to be read from for each window. * Assumes the ChunkInfos are in increasing time order. @@ -402,9 +409,9 @@ extends Iterator[ChunkQueryInfo] { val valueReader = rv.partition.chunkReader(rv.valueColID, valueVector) windowInfos += ChunkQueryInfo(next.infoAddr, tsVector, tsReader, valueVector, valueReader) } catch { - case e: Throwable => { - ChunkSetInfo.log.error(s"Corrupt vector at ${java.lang.Long.toHexString(tsVector)}", e) - } + case m: MatchError => + throw CorruptVectorException(tsVector, next.startTime, rv.partition.stringPartition, + rv.partition.shard, m) } lastEndTime = Math.max(next.endTime, lastEndTime) } diff --git a/core/src/test/scala/filodb.core/TestData.scala b/core/src/test/scala/filodb.core/TestData.scala index 28599f235c..8824de662e 100644 --- a/core/src/test/scala/filodb.core/TestData.scala +++ b/core/src/test/scala/filodb.core/TestData.scala @@ -217,6 +217,7 @@ object MachineMetricsData { import scala.util.Random.nextInt val columns = Seq("timestamp:long", "min:double", "avg:double", "max:double", "count:long") + val dummyContext = Map("test" -> "test") def singleSeriesData(initTs: Long = System.currentTimeMillis, incr: Long = 1000): Stream[Product] = { @@ -365,7 +366,7 @@ object MachineMetricsData { val histPartKey = histKeyBuilder.addFromObjects(extraTags) val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) - private val histIngestBH = new BlockMemFactory(blockStore, None, histDataset.blockMetaSize, true) + private val histIngestBH = new BlockMemFactory(blockStore, None, histDataset.blockMetaSize, dummyContext, true) private val histBufferPool = new WriteBufferPool(TestData.nativeMem, histDataset, TestData.storeConf) // Designed explicitly to work with linearHistSeries records and histDataset from MachineMetricsData diff --git a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala index 08f2746926..74dc975501 100644 --- a/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala +++ b/core/src/test/scala/filodb.core/downsample/ShardDownsamplerSpec.scala @@ -15,7 +15,7 @@ import filodb.memory._ import filodb.memory.format.{TupleRowReader, ZeroCopyUTF8String, vectors => bv} // scalastyle:off null -class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll { +class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll { val promDataset = Dataset.make("custom1", Seq("someStr:string", "tags:map"), @@ -32,7 +32,8 @@ class ShardDownsamplerSpec extends FunSpec with Matchers with BeforeAndAfterAll DatasetOptions(Seq("name", "namespace"), "name", "total", true)).get private val blockStore = MMD.blockStore - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, true) + protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, promDataset.blockMetaSize, + MMD.dummyContext, true) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, promDataset, storeConf) diff --git a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala index f458b393f6..c3d3e8e938 100644 --- a/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/PartitionSetSpec.scala @@ -32,7 +32,7 @@ class PartitionSetSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) protected val bufferPool = new WriteBufferPool(memFactory, dataset2, TestData.storeConf) - private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.blockMetaSize, true) + private val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset2.blockMetaSize, dummyContext, true) val builder = new RecordBuilder(memFactory, dataset2.ingestionSchema) val partSet = PartitionSet.empty() diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index b38ae6e070..d2e24dc113 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -69,7 +69,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), reclaimer, 1) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, true) + protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext, true) before { colStore.truncate(dataset1.ref).futureValue @@ -111,7 +111,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { val data1 = part.timeRangeRows(AllChunkScan, Array(1)).map(_.getDouble(0)).toBuffer data1 shouldEqual (minData take 10) - val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(r, ingestBlockHolder) } @@ -152,7 +152,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { part.switchBuffers(ingestBlockHolder) part.appendingChunkLen shouldEqual 0 - val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).zipWithIndex.foreach { case (r, i) => part.ingest(r, ingestBlockHolder) } @@ -209,7 +209,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // First 10 rows ingested. Now flush in a separate Future while ingesting 6 more rows part.switchBuffers(ingestBlockHolder) myBufferPool.poolSize shouldEqual origPoolSize // current chunks become null, no new allocation yet - val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut = Future(part.makeFlushChunks(blockHolder).toBuffer) data.drop(10).take(6).zipWithIndex.foreach { case (r, i) => part.ingest(r, ingestBlockHolder) } @@ -240,7 +240,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now, switch buffers and flush again, ingesting 5 more rows // There should now be 3 chunks total, the current write buffers plus the two flushed ones part.switchBuffers(ingestBlockHolder) - val holder2 = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val holder2 = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) // Task needs to fully iterate over the chunks, to release the shared lock. val flushFut2 = Future(part.makeFlushChunks(holder2).toBuffer) data.drop(16).zipWithIndex.foreach { case (r, i) => part.ingest(r, ingestBlockHolder) } @@ -269,7 +269,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder) chunkSets.isEmpty shouldEqual false part.numChunks shouldEqual 1 @@ -359,7 +359,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { // Now simulate a flush, verify that both chunksets flushed // Now, switch buffers and flush. Appenders will be empty. part.switchBuffers(ingestBlockHolder) - val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize) + val blockHolder = new BlockMemFactory(blockStore, None, dataset1.blockMetaSize, dummyContext) val chunkSets = part.makeFlushChunks(blockHolder).toSeq chunkSets should have length (2) part.numChunks shouldEqual 2 diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index a4fabd6b8e..b8752befe4 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -1,6 +1,5 @@ package filodb.memory -import java.lang.{Long => jLong} import java.nio.ByteBuffer import java.util.ConcurrentModificationException import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} @@ -93,7 +92,6 @@ trait ReusableMemory extends StrictLogging { * Marks this memory as free and calls reclaimListener for every piece of metadata. */ protected def free() = { - logger.info(s"Reclaiming block at ${jLong.toHexString(address)}...") reclaimWithMetadata() } @@ -122,6 +120,20 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL protected var _position: Int = 0 protected var _metaPosition: Int = capacity.toInt + /** + * Keeps track of which BlockMemFactory "owns" this block. Set when block is requested by a BMF, + * cleared when the block is reclaimed. + */ + var owner: Option[BlockMemFactory] = None + + def setOwner(bmf: BlockMemFactory): Unit = { + owner = bmf.optionSelf + } + + def clearOwner(): Unit = { + owner = None + } + /** * Marks this memory as free. Also zeroes all the bytes from the beginning address until capacity */ @@ -184,9 +196,10 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL stringBuf.toString } + def debugString: String = f"Block @0x$address%016x canReclaim=$canReclaim remaining=$remaining " + + s"owner: ${owner.map(_.debugString).getOrElse("--")}" + // debug method to set memory to specific value for testing private[memory] def set(value: Byte): Unit = UnsafeUtils.unsafe.setMemory(address, capacity, value) } - - diff --git a/memory/src/main/scala/filodb.memory/BlockDetective.scala b/memory/src/main/scala/filodb.memory/BlockDetective.scala new file mode 100644 index 0000000000..881a1d60e3 --- /dev/null +++ b/memory/src/main/scala/filodb.memory/BlockDetective.scala @@ -0,0 +1,41 @@ +package filodb.memory + +import org.joda.time.DateTime + +/** + * BlockDetective has utilities for recovering blocks and their reclaim histories, given a corrupt pointer. + * This helps to track down ownership and causes for corruption issues. + */ +object BlockDetective { + import collection.JavaConverters._ + + def containsPtr(ptr: BinaryRegion.NativePointer, blocks: Seq[Block]): Seq[Block] = + blocks.filter { blk => ptr >= blk.address && ptr < (blk.address + blk.capacity) } + + def containsPtr(ptr: BinaryRegion.NativePointer, blocks: java.util.List[Block]): Seq[Block] = + containsPtr(ptr, blocks.asScala) + + /** + * Produces a string report containing reclaim history and ownership changes for + * blocks containing a given pointer. + * Reclaim history is limited by MaxReclaimLogSize above, thus + * the bet is that corruption happens soon after reclaim events. + */ + def stringReport(ptr: BinaryRegion.NativePointer, + manager: PageAlignedBlockManager, + pool: BlockMemFactoryPool): String = { + val reclaimEvents = manager.reclaimEventsForPtr(ptr) + val timeBucketBlocks = manager.timeBlocksForPtr(ptr) + val flushBlocks = pool.blocksContainingPtr(ptr) + + f"=== BlockDetective Report for 0x$ptr%016x ===\nReclaim Events:\n" + + reclaimEvents.map { case ReclaimEvent(blk, reclaimTime, oldOwner, remaining) => + f" Block 0x${blk.address}%016x at ${(new DateTime(reclaimTime)).toString()}%s with $remaining%d bytes left" + + oldOwner.map { bmf => s"\tfrom ${bmf.debugString}" }.getOrElse("") + }.mkString("\n") + + "Time bucketed blocks:\n" + + timeBucketBlocks.map(_.debugString).mkString("\n") + + "Flush block lists:\n" + + flushBlocks.map(_.debugString).mkString("\n") + } +} diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 4d1948aa95..67efed1b08 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -1,5 +1,6 @@ package filodb.memory +import java.lang.{Long => jLong} import java.util import java.util.concurrent.locks.ReentrantLock @@ -31,15 +32,17 @@ trait BlockManager { /** * @param memorySize The size of memory in bytes for which blocks are to be allocated * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list + * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return A sequence of blocks totaling up in memory requested or empty if unable to allocate */ - def requestBlocks(memorySize: Long, bucketTime: Option[Long]): Seq[Block] + def requestBlocks(memorySize: Long, bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Seq[Block] /** * @param bucketTime the timebucket from which to allocate block(s), or None for the general list + * @param owner the BlockMemFactory that will be owning this block, until reclaim. Used for debugging. * @return One block of memory */ - def requestBlock(bucketTime: Option[Long]): Option[Block] + def requestBlock(bucketTime: Option[Long], owner: Option[BlockMemFactory] = None): Option[Block] /** * Releases all blocks allocated by this store. @@ -66,6 +69,12 @@ class MemoryStats(tags: Map[String, String]) { val blocksReclaimedMetric = Kamon.counter("blockstore-blocks-reclaimed").refine(tags) } +final case class ReclaimEvent(block: Block, reclaimTime: Long, oldOwner: Option[BlockMemFactory], remaining: Long) + +object PageAlignedBlockManager { + val MaxReclaimLogSize = 10000 +} + /** * Pre Allocates blocks totalling to the passed memory size. * Each block size is the same as the OS page size. @@ -81,6 +90,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, reclaimer: ReclaimListener, numPagesPerBlock: Int) extends BlockManager with StrictLogging { + import PageAlignedBlockManager._ + val mask = PageManager.PROT_READ | PageManager.PROT_EXEC | PageManager.PROT_WRITE import collection.JavaConverters._ @@ -90,6 +101,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, protected val freeBlocks: util.LinkedList[Block] = allocate() protected[memory] val usedBlocks: util.LinkedList[Block] = new util.LinkedList[Block]() protected[memory] val usedBlocksTimeOrdered = new util.TreeMap[Long, util.LinkedList[Block]] + val reclaimLog = new collection.mutable.Queue[ReclaimEvent] protected val lock = new ReentrantLock() @@ -101,8 +113,8 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, override def numFreeBlocks: Int = freeBlocks.size - override def requestBlock(bucketTime: Option[Long]): Option[Block] = { - val blocks = requestBlocks(blockSizeInBytes, bucketTime) + override def requestBlock(bucketTime: Option[Long], bmf: Option[BlockMemFactory] = None): Option[Block] = { + val blocks = requestBlocks(blockSizeInBytes, bucketTime, bmf) blocks.size match { case 0 => None case 1 => Some(blocks.head) @@ -123,7 +135,9 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, * then uses the ReclaimPolicy to check if blocks can be reclaimed * Uses a lock to ensure that concurrent requests are safe. */ - override def requestBlocks(memorySize: Long, bucketTime: Option[Long]): Seq[Block] = { + override def requestBlocks(memorySize: Long, + bucketTime: Option[Long], + bmf: Option[BlockMemFactory] = None): Seq[Block] = { lock.lock() try { val num: Int = Math.ceil(memorySize / blockSizeInBytes).toInt @@ -135,6 +149,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val allocated = new Array[Block](num) (0 until num).foreach { i => val block = freeBlocks.remove() + if (bmf.nonEmpty) block.setOwner(bmf.get) use(block, bucketTime) allocated(i) = block } @@ -177,6 +192,12 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, stats.freeBlocksMetric.set(freeBlocks.size()) } + private def addToReclaimLog(block: Block): Unit = { + val event = ReclaimEvent(block, System.currentTimeMillis, block.owner, block.remaining) + if (reclaimLog.size >= MaxReclaimLogSize) { reclaimLog.dequeue } + reclaimLog += event + } + protected def tryReclaim(num: Int): Unit = { var reclaimed = 0 var currList = 0 @@ -184,8 +205,13 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, while ( reclaimed < num && timeOrderedListIt.hasNext ) { val entry = timeOrderedListIt.next - logger.info(s"timeBlockReclaim: Attempting to reclaim time ordered block list at t=${}") - reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric) + val prevReclaimed = reclaimed + val removed = reclaimFrom(entry.getValue, stats.timeOrderedBlocksReclaimedMetric) + if (removed.nonEmpty) { + logger.info(s"timeBlockReclaim: Reclaimed ${removed.length} time ordered blocks " + + s"from list at t=${entry.getKey} (${(System.currentTimeMillis - entry.getKey)/3600000} hrs ago) " + + s"\nReclaimed blocks: ${removed.map(b => jLong.toHexString(b.address)).mkString(" ")}") + } // If the block list is now empty, remove it from tree map if (entry.getValue.isEmpty) timeOrderedListIt.remove() } @@ -196,19 +222,24 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, s"usedBlocksTimeOrdered=${usedBlocksTimeOrdered.asScala.toList.map{case(n, l) => (n, l.size)}}") } - def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): Unit = { + def reclaimFrom(list: util.LinkedList[Block], reclaimedCounter: Counter): Seq[Block] = { val entries = list.iterator + val removed = new collection.mutable.ArrayBuffer[Block] while (entries.hasNext && reclaimed < num) { val block = entries.next if (block.canReclaim) { entries.remove() + removed += block + addToReclaimLog(block) block.reclaim() + block.clearOwner() freeBlocks.add(block) stats.freeBlocksMetric.set(freeBlocks.size()) reclaimedCounter.increment() reclaimed = reclaimed + 1 } } + removed } } @@ -247,6 +278,18 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, tryReclaim(usedBlocks.size + numTimeOrderedBlocks) } + /** + * Finds all reclaim events in the log whose Block contains the pointer passed in. + * Useful for debugging. O(n) - not performant. + */ + def reclaimEventsForPtr(ptr: BinaryRegion.NativePointer): Seq[ReclaimEvent] = + reclaimLog.filter { ev => ptr >= ev.block.address && ptr < (ev.block.address + ev.block.capacity) } + + def timeBlocksForPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = + usedBlocksTimeOrdered.entrySet.iterator.asScala.flatMap { entry => + BlockDetective.containsPtr(ptr, entry.getValue) + }.toSeq + def releaseBlocks(): Unit = { lock.lock() try { diff --git a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala index a807393529..f6ba89687c 100644 --- a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala +++ b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala @@ -6,24 +6,41 @@ import com.typesafe.scalalogging.StrictLogging * This class allows BlockMemFactory's to be reused so that the blocks can be fully utilized, instead of left stranded * and half empty. It has a checkout and return semantics. Multiple parallel tasks each do their own * checkout and return, thus there should be one blockholder outstanding per task. + * + * @param blockStore the underlying BlockManager to allocate blocks from for each BlockMemFactory + * @param metadataAllocSize size of each metadata set per allocation + * @param baseTags a set of tags to identify each BlockMemFactory, used only for debugging */ -class BlockMemFactoryPool(blockStore: BlockManager, metadataAllocSize: Int) extends StrictLogging { +class BlockMemFactoryPool(blockStore: BlockManager, + metadataAllocSize: Int, + baseTags: Map[String, String]) extends StrictLogging { private val factoryPool = new collection.mutable.Queue[BlockMemFactory]() def poolSize: Int = factoryPool.length - def checkout(): BlockMemFactory = synchronized { - if (factoryPool.nonEmpty) { + /** + * Checks out a BlockMemFactory, optionally adding in extra tags for this particular BMF + */ + def checkout(moreTags: Map[String, String] = Map.empty): BlockMemFactory = synchronized { + val fact = if (factoryPool.nonEmpty) { logger.debug(s"Checking out BlockMemFactory from pool. poolSize=$poolSize") factoryPool.dequeue } else { logger.debug(s"Nothing in BlockMemFactory pool. Creating a new one") - new BlockMemFactory(blockStore, None, metadataAllocSize) + new BlockMemFactory(blockStore, None, metadataAllocSize, baseTags) } + fact.tags = baseTags ++ moreTags + fact } def release(factory: BlockMemFactory): Unit = synchronized { logger.debug(s"Returning factory $factory to the pool. New size ${poolSize + 1}") factoryPool += factory } + + def blocksContainingPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = + factoryPool.flatMap { bmf => + val blocks = bmf.fullBlocks ++ Option(bmf.currentBlock.get).toList + BlockDetective.containsPtr(ptr, blocks) + } } \ No newline at end of file diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 232dfcefef..388d7afdd9 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -197,14 +197,17 @@ class ArrayBackedMemFactory extends MemFactory { * block is full. * @param bucketTime the timebucket (timestamp) from which to allocate block(s), or None for the general list * @param metadataAllocSize the additional size in bytes to ensure is free for writing metadata, per chunk + * @param tags a set of keys/values to identify the purpose of this MemFactory for debugging * @param markFullBlocksAsReclaimable Immediately mark and fully used block as reclaimable. * Typically true during on-demand paging of optimized chunks from persistent store */ class BlockMemFactory(blockStore: BlockManager, bucketTime: Option[Long], metadataAllocSize: Int, + var tags: Map[String, String], markFullBlocksAsReclaimable: Boolean = false) extends MemFactory with StrictLogging { def numFreeBytes: Long = blockStore.numFreeBlocks * blockStore.blockSizeInBytes + val optionSelf = Some(this) // tracks fully populated blocks not marked reclaimable yet (typically waiting for flush) val fullBlocks = ListBuffer[Block]() @@ -215,7 +218,7 @@ class BlockMemFactory(blockStore: BlockManager, // tracks blocks that should share metadata private val metadataSpan: ListBuffer[Block] = ListBuffer[Block]() - currentBlock.set(blockStore.requestBlock(bucketTime).get) + currentBlock.set(blockStore.requestBlock(bucketTime, optionSelf).get) /** * Starts tracking a span of multiple Blocks over which the same metadata should be applied. @@ -254,7 +257,7 @@ class BlockMemFactory(blockStore: BlockManager, currentBlock.get().markReclaimable() } fullBlocks += currentBlock.get() - val newBlock = blockStore.requestBlock(bucketTime).get + val newBlock = blockStore.requestBlock(bucketTime, optionSelf).get currentBlock.set(newBlock) metadataSpan += newBlock } @@ -296,6 +299,9 @@ class BlockMemFactory(blockStore: BlockManager, // We don't free memory, because many BlockHolders will share a single BlockManager, and we rely on // the BlockManager's own shutdown mechanism def shutdown(): Unit = {} + + def debugString: String = + s"BlockMemFactory($bucketTime) ${tags.map { case (k, v) => s"$k=$v" }.mkString(" ")}" } diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 81cffb3e4f..9a181625d5 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -168,4 +168,40 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.releaseBlocks() } + + it should ("allocate blocks using BlockMemFactory with ownership and reclaims") in { + val stats = new MemoryStats(Map("test5" -> "test5")) + // This block manager has 5 blocks capacity + val blockManager = new PageAlignedBlockManager(5 * pageSize, stats, testReclaimer, 1) + + blockManager.usedBlocks.size() shouldEqual 0 + blockManager.numTimeOrderedBlocks shouldEqual 0 + blockManager.usedBlocksTimeOrdered.size shouldEqual 0 + + val factory = new BlockMemFactory(blockManager, Some(10000L), 24, Map("foo" -> "bar"), false) + + // There should be one time ordered block allocated, owned by factory + blockManager.usedBlocks.size shouldEqual 0 + blockManager.numTimeOrderedBlocks shouldEqual 1 + blockManager.hasTimeBucket(10000L) shouldEqual true + + factory.currentBlock.get.owner shouldEqual Some(factory) + + // Now allocate 4 more regular blocks, that will use up all blocks + blockManager.requestBlock(None).isDefined shouldEqual true + blockManager.requestBlock(None).isDefined shouldEqual true + blockManager.requestBlock(None).isDefined shouldEqual true + blockManager.requestBlock(None).isDefined shouldEqual true + blockManager.usedBlocks.size shouldEqual 4 + blockManager.numTimeOrderedBlocks shouldEqual 1 + + // Mark as reclaimable the blockMemFactory's block. Then request more blocks, that one will be reclaimed. + // Check ownership is now cleared. + factory.currentBlock.get.markReclaimable + blockManager.requestBlock(Some(9000L)).isDefined shouldEqual true + blockManager.hasTimeBucket(10000L) shouldEqual false + blockManager.hasTimeBucket(9000L) shouldEqual true + + factory.currentBlock.get.owner shouldEqual None // new requestor did not have owner + } } diff --git a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala index bfb07f70cf..6f619ea609 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/IntBinaryVectorTest.scala @@ -100,7 +100,7 @@ class IntBinaryVectorTest extends NativeVectorTest { val blockStore = new PageAlignedBlockManager(10 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) { freeBlocks.asScala.foreach(_.set(0x55)) // initialize blocks to nonzero value } - val blockFactory = new BlockMemFactory(blockStore, None, 24, true) + val blockFactory = new BlockMemFactory(blockStore, None, 24, Map("foo" -> "bar"), true) // original values will get mixed with nonzero contents if append does not overwrite original memory val builder = IntBinaryVector.appendingVectorNoNA(blockFactory, 10, nbits=4, signed=false) diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index da4e32ad39..d9205067db 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -23,10 +23,12 @@ trait RawDataWindowingSpec extends FunSpec with Matchers with BeforeAndAfterAll private val blockStore = new PageAlignedBlockManager(100 * 1024 * 1024, new MemoryStats(Map("test"-> "test")), null, 16) val storeConf = TestData.storeConf.copy(maxChunksSize = 200) - protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesDataset.blockMetaSize, true) + protected val ingestBlockHolder = new BlockMemFactory(blockStore, None, timeseriesDataset.blockMetaSize, + MMD.dummyContext, true) protected val tsBufferPool = new WriteBufferPool(TestData.nativeMem, timeseriesDataset, storeConf) - protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, None, downsampleDataset.blockMetaSize, true) + protected val ingestBlockHolder2 = new BlockMemFactory(blockStore, None, downsampleDataset.blockMetaSize, + MMD.dummyContext, true) protected val tsBufferPool2 = new WriteBufferPool(TestData.nativeMem, downsampleDataset, storeConf) override def afterAll(): Unit = { From ee945b417e7bcc869d2a8d988ce4608fa8279c1e Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 30 Aug 2019 14:50:41 -0700 Subject: [PATCH 20/28] fix(memory: Fix BlockDetective ConcurrentModificationException (#484) --- .../main/scala/filodb.memory/BlockDetective.scala | 4 ++-- .../main/scala/filodb.memory/BlockManager.scala | 14 ++++++++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/BlockDetective.scala b/memory/src/main/scala/filodb.memory/BlockDetective.scala index 881a1d60e3..be5ce818cf 100644 --- a/memory/src/main/scala/filodb.memory/BlockDetective.scala +++ b/memory/src/main/scala/filodb.memory/BlockDetective.scala @@ -33,9 +33,9 @@ object BlockDetective { f" Block 0x${blk.address}%016x at ${(new DateTime(reclaimTime)).toString()}%s with $remaining%d bytes left" + oldOwner.map { bmf => s"\tfrom ${bmf.debugString}" }.getOrElse("") }.mkString("\n") + - "Time bucketed blocks:\n" + + "\nTime bucketed blocks:\n" + timeBucketBlocks.map(_.debugString).mkString("\n") + - "Flush block lists:\n" + + "\nFlush block lists:\n" + flushBlocks.map(_.debugString).mkString("\n") } } diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index 67efed1b08..e02de2c82f 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -285,10 +285,16 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, def reclaimEventsForPtr(ptr: BinaryRegion.NativePointer): Seq[ReclaimEvent] = reclaimLog.filter { ev => ptr >= ev.block.address && ptr < (ev.block.address + ev.block.capacity) } - def timeBlocksForPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = - usedBlocksTimeOrdered.entrySet.iterator.asScala.flatMap { entry => - BlockDetective.containsPtr(ptr, entry.getValue) - }.toSeq + def timeBlocksForPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = { + lock.lock() + try { + usedBlocksTimeOrdered.entrySet.iterator.asScala.flatMap { entry => + BlockDetective.containsPtr(ptr, entry.getValue) + }.toBuffer + } finally { + lock.unlock() + } + } def releaseBlocks(): Unit = { lock.lock() From a5b1661e14808e8d734b5185832cf5a4831e57b6 Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Thu, 5 Sep 2019 16:03:00 -0700 Subject: [PATCH 21/28] fix(memory): Avoid marking a block as reclaimable twice. Doing so permits the block to be mutated by multiple threads. Also remove atomic reference, which isn't necessary because the enclosing class isn't designed to be thread safe. (#487) --- .../memstore/TimeSeriesPartitionSpec.scala | 2 +- .../filodb.memory/BlockMemFactoryPool.scala | 4 ++-- .../main/scala/filodb.memory/MemFactory.scala | 22 +++++++++---------- .../PageAlignedBlockManagerSpec.scala | 6 ++--- 4 files changed, 16 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala index d2e24dc113..42bc196554 100644 --- a/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala +++ b/core/src/test/scala/filodb.core/memstore/TimeSeriesPartitionSpec.scala @@ -234,7 +234,7 @@ class TimeSeriesPartitionSpec extends MemFactoryCleanupTest with ScalaFutures { chunkSets.head.invokeFlushListener() // update newestFlushedID part.unflushedChunksets shouldEqual 1 - val currBlock = blockHolder.currentBlock.get() // hang on to these; we'll later test reclaiming them manually + val currBlock = blockHolder.currentBlock // hang on to these; we'll later test reclaiming them manually blockHolder.markUsedBlocksReclaimable() // Now, switch buffers and flush again, ingesting 5 more rows diff --git a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala index f6ba89687c..308e7d7928 100644 --- a/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala +++ b/memory/src/main/scala/filodb.memory/BlockMemFactoryPool.scala @@ -40,7 +40,7 @@ class BlockMemFactoryPool(blockStore: BlockManager, def blocksContainingPtr(ptr: BinaryRegion.NativePointer): Seq[Block] = factoryPool.flatMap { bmf => - val blocks = bmf.fullBlocks ++ Option(bmf.currentBlock.get).toList + val blocks = bmf.fullBlocks ++ Option(bmf.currentBlock).toList BlockDetective.containsPtr(ptr, blocks) } -} \ No newline at end of file +} diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index 388d7afdd9..c2512482d6 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -1,7 +1,6 @@ package filodb.memory import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable.ListBuffer @@ -213,20 +212,18 @@ class BlockMemFactory(blockStore: BlockManager, val fullBlocks = ListBuffer[Block]() // tracks block currently being populated - val currentBlock = new AtomicReference[Block]() + var currentBlock = blockStore.requestBlock(bucketTime, optionSelf).get // tracks blocks that should share metadata private val metadataSpan: ListBuffer[Block] = ListBuffer[Block]() - currentBlock.set(blockStore.requestBlock(bucketTime, optionSelf).get) - /** * Starts tracking a span of multiple Blocks over which the same metadata should be applied. * An example would be chunk metadata for chunks written to potentially more than 1 block. */ def startMetaSpan(): Unit = { metadataSpan.clear() - metadataSpan += (currentBlock.get()) + metadataSpan += currentBlock } /** @@ -252,16 +249,17 @@ class BlockMemFactory(blockStore: BlockManager, } protected def ensureCapacity(forSize: Long): Block = { - if (!currentBlock.get().hasCapacity(forSize)) { + if (!currentBlock.hasCapacity(forSize)) { + val newBlock = blockStore.requestBlock(bucketTime, optionSelf).get if (markFullBlocksAsReclaimable) { - currentBlock.get().markReclaimable() + currentBlock.markReclaimable() + } else { + fullBlocks += currentBlock } - fullBlocks += currentBlock.get() - val newBlock = blockStore.requestBlock(bucketTime, optionSelf).get - currentBlock.set(newBlock) + currentBlock = newBlock metadataSpan += newBlock } - currentBlock.get() + currentBlock } /** @@ -294,7 +292,7 @@ class BlockMemFactory(blockStore: BlockManager, /** * @return The capacity of any allocated block */ - def blockAllocationSize(): Long = currentBlock.get().capacity + def blockAllocationSize(): Long = currentBlock.capacity // We don't free memory, because many BlockHolders will share a single BlockManager, and we rely on // the BlockManager's own shutdown mechanism diff --git a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala index 9a181625d5..4e62faa440 100644 --- a/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala +++ b/memory/src/test/scala/filodb.memory/PageAlignedBlockManagerSpec.scala @@ -185,7 +185,7 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA blockManager.numTimeOrderedBlocks shouldEqual 1 blockManager.hasTimeBucket(10000L) shouldEqual true - factory.currentBlock.get.owner shouldEqual Some(factory) + factory.currentBlock.owner shouldEqual Some(factory) // Now allocate 4 more regular blocks, that will use up all blocks blockManager.requestBlock(None).isDefined shouldEqual true @@ -197,11 +197,11 @@ class PageAlignedBlockManagerSpec extends FlatSpec with Matchers with BeforeAndA // Mark as reclaimable the blockMemFactory's block. Then request more blocks, that one will be reclaimed. // Check ownership is now cleared. - factory.currentBlock.get.markReclaimable + factory.currentBlock.markReclaimable blockManager.requestBlock(Some(9000L)).isDefined shouldEqual true blockManager.hasTimeBucket(10000L) shouldEqual false blockManager.hasTimeBucket(9000L) shouldEqual true - factory.currentBlock.get.owner shouldEqual None // new requestor did not have owner + factory.currentBlock.owner shouldEqual None // new requestor did not have owner } } From 997c3eca64a53f74711106f6264b4079a69d275b Mon Sep 17 00:00:00 2001 From: "Brian S. O'Neill" Date: Tue, 17 Sep 2019 13:51:59 -0700 Subject: [PATCH 22/28] fix(memory): Protect blocks from being marked as reclaimable multiple times. (#489) Also, don't load empty chunks, since it screws up the metaspan. --- .../memstore/DemandPagedChunkStore.scala | 33 +++++---- .../src/main/scala/filodb.memory/Block.scala | 10 +++ .../scala/filodb.memory/BlockManager.scala | 2 +- .../main/scala/filodb.memory/MemFactory.scala | 68 +++++++++++++++---- 4 files changed, 87 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala index e98843f516..a2f6aeb793 100644 --- a/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala +++ b/core/src/main/scala/filodb.core/memstore/DemandPagedChunkStore.scala @@ -73,21 +73,28 @@ extends RawToPartitionMaker with StrictLogging { tsShard.shardStats.numChunksPagedIn.increment(rawPartition.chunkSets.size) // One chunkset at a time, load them into offheap and populate the partition rawPartition.chunkSets.foreach { case RawChunkSet(infoBytes, rawVectors) => - val memFactory = getMemFactory(timeBucketForChunkSet(infoBytes)) - val chunkID = ChunkSetInfo.getChunkID(infoBytes) + // If the chunk is empty, skip it. If no call to allocateOffheap is made, then no check + // is made to ensure that the block has room even for metadata. The call to endMetaSpan + // might end up returning 0, because the last block doesn't have any room. It's + // possible to guard against this by forcing an allocation, but it doesn't make sense + // to allocate a block just for storing an unnecessary metadata entry. + if (!rawVectors.isEmpty) { + val memFactory = getMemFactory(timeBucketForChunkSet(infoBytes)) + val chunkID = ChunkSetInfo.getChunkID(infoBytes) - memFactory.startMetaSpan() - val chunkPtrs = copyToOffHeap(rawVectors, memFactory) - val metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), - tsShard.dataset.blockMetaSize.toShort) - require(metaAddr != 0) - val infoAddr = metaAddr + 4 // Important: don't point at partID - val inserted = tsPart.addChunkInfoIfAbsent(chunkID, infoAddr) + memFactory.startMetaSpan() + val chunkPtrs = copyToOffHeap(rawVectors, memFactory) + val metaAddr = memFactory.endMetaSpan(writeMeta(_, tsPart.partID, infoBytes, chunkPtrs), + tsShard.dataset.blockMetaSize.toShort) + require(metaAddr != 0) + val infoAddr = metaAddr + 4 // Important: don't point at partID + val inserted = tsPart.addChunkInfoIfAbsent(chunkID, infoAddr) - if (!inserted) { - logger.info(s"Chunks not copied to partId=${tsPart.partID} ${tsPart.stringPartition}, already has chunk " + - s"$chunkID. Chunk time range (${ChunkSetInfo.getStartTime(infoBytes)}, " + - s"${ChunkSetInfo.getEndTime(infoBytes)}) partition earliestTime=${tsPart.earliestTime}") + if (!inserted) { + logger.info(s"Chunks not copied to partId=${tsPart.partID} ${tsPart.stringPartition}, already has chunk " + + s"$chunkID. Chunk time range (${ChunkSetInfo.getStartTime(infoBytes)}, " + + s"${ChunkSetInfo.getEndTime(infoBytes)}) partition earliestTime=${tsPart.earliestTime}") + } } } tsPart diff --git a/memory/src/main/scala/filodb.memory/Block.scala b/memory/src/main/scala/filodb.memory/Block.scala index b8752befe4..bd9155cebf 100644 --- a/memory/src/main/scala/filodb.memory/Block.scala +++ b/memory/src/main/scala/filodb.memory/Block.scala @@ -134,6 +134,16 @@ class Block(val address: Long, val capacity: Long, val reclaimListener: ReclaimL owner = None } + /** + * Marks this block as reclaimable if unowned, or if the owner hasn't used the block in a while. + */ + def tryMarkReclaimable(): Unit = { + owner match { + case None => markReclaimable + case Some(bmf) => bmf.tryMarkReclaimable + } + } + /** * Marks this memory as free. Also zeroes all the bytes from the beginning address until capacity */ diff --git a/memory/src/main/scala/filodb.memory/BlockManager.scala b/memory/src/main/scala/filodb.memory/BlockManager.scala index e02de2c82f..fbc0ad2fc0 100644 --- a/memory/src/main/scala/filodb.memory/BlockManager.scala +++ b/memory/src/main/scala/filodb.memory/BlockManager.scala @@ -254,7 +254,7 @@ class PageAlignedBlockManager(val totalMemorySizeInBytes: Long, val keys = usedBlocksTimeOrdered.headMap(upTo).keySet.asScala logger.info(s"timeBlockReclaim: Marking lists $keys as reclaimable") usedBlocksTimeOrdered.headMap(upTo).values.asScala.foreach { list => - list.asScala.foreach(_.markReclaimable) + list.asScala.foreach(_.tryMarkReclaimable) } } finally { lock.unlock() diff --git a/memory/src/main/scala/filodb.memory/MemFactory.scala b/memory/src/main/scala/filodb.memory/MemFactory.scala index c2512482d6..7277ecb0df 100644 --- a/memory/src/main/scala/filodb.memory/MemFactory.scala +++ b/memory/src/main/scala/filodb.memory/MemFactory.scala @@ -3,6 +3,7 @@ package filodb.memory import java.nio.ByteBuffer import scala.collection.mutable.ListBuffer +import scala.concurrent.duration._ import com.kenai.jffi.MemoryIO import com.typesafe.scalalogging.StrictLogging @@ -187,6 +188,12 @@ class ArrayBackedMemFactory extends MemFactory { def shutdown(): Unit = {} } +object BlockMemFactory { + // Simple constant to avoid premature reclamation, under the assumption that appending + // metadata to the block is quick. In practice, a few microseconds. Not an ideal solution, + // but it's easier than retrofitting this class to support safe memory ownership. + val USED_THRESHOLD_NANOS = 1.minute.toNanos +} /** * A MemFactory that allocates memory from Blocks obtained from the BlockManager. It @@ -212,18 +219,51 @@ class BlockMemFactory(blockStore: BlockManager, val fullBlocks = ListBuffer[Block]() // tracks block currently being populated - var currentBlock = blockStore.requestBlock(bucketTime, optionSelf).get + var currentBlock = requestBlock() + + private def requestBlock() = blockStore.requestBlock(bucketTime, optionSelf).get // tracks blocks that should share metadata private val metadataSpan: ListBuffer[Block] = ListBuffer[Block]() + // Last time this factory was used for allocation. + private var lastUsedNanos = now + + private def now: Long = System.nanoTime() + + // This should be called to obtain a non-null current block reference. + // Caller should be synchronized. + //scalastyle:off null + private def accessCurrentBlock() = synchronized { + lastUsedNanos = now + if (currentBlock == null) { + currentBlock = requestBlock + } + currentBlock + } + + /** + * Marks all blocks known by this factory as reclaimable, but only if this factory hasn't + * been used recently. + */ + def tryMarkReclaimable(): Unit = synchronized { + if (now - lastUsedNanos > BlockMemFactory.USED_THRESHOLD_NANOS) { + markUsedBlocksReclaimable() + if (currentBlock != null) { + currentBlock.markReclaimable() + currentBlock = null + } + } + } + //scalastyle:on null + /** * Starts tracking a span of multiple Blocks over which the same metadata should be applied. * An example would be chunk metadata for chunks written to potentially more than 1 block. */ def startMetaSpan(): Unit = { metadataSpan.clear() - metadataSpan += currentBlock + metadataSpan += accessCurrentBlock() } /** @@ -243,23 +283,25 @@ class BlockMemFactory(blockStore: BlockManager, metaAddr } - def markUsedBlocksReclaimable(): Unit = { + def markUsedBlocksReclaimable(): Unit = synchronized { fullBlocks.foreach(_.markReclaimable()) fullBlocks.clear() } - protected def ensureCapacity(forSize: Long): Block = { - if (!currentBlock.hasCapacity(forSize)) { - val newBlock = blockStore.requestBlock(bucketTime, optionSelf).get + protected def ensureCapacity(forSize: Long): Block = synchronized { + var block = accessCurrentBlock() + if (!block.hasCapacity(forSize)) { + val newBlock = requestBlock() if (markFullBlocksAsReclaimable) { - currentBlock.markReclaimable() + block.markReclaimable() } else { - fullBlocks += currentBlock + fullBlocks += block } - currentBlock = newBlock - metadataSpan += newBlock + block = newBlock + currentBlock = block + metadataSpan += block } - currentBlock + block } /** @@ -292,7 +334,9 @@ class BlockMemFactory(blockStore: BlockManager, /** * @return The capacity of any allocated block */ - def blockAllocationSize(): Long = currentBlock.capacity + def blockAllocationSize(): Long = synchronized { + accessCurrentBlock().capacity + } // We don't free memory, because many BlockHolders will share a single BlockManager, and we rely on // the BlockManager's own shutdown mechanism From ee6389e59eb872a12186d25fce7e46f0fffdea05 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Wed, 28 Aug 2019 18:29:56 -0700 Subject: [PATCH 23/28] feat(query) : operator precedence (#426) --- .../filodb/prometheus/parse/Parser.scala | 44 +++++++- .../filodb/prometheus/parse/ParserSpec.scala | 103 +++++++++--------- .../main/scala/filodb/query/PlanEnums.scala | 66 ++++++----- 3 files changed, 133 insertions(+), 80 deletions(-) diff --git a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala index 375c1b70f0..926eae5089 100644 --- a/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala +++ b/prometheus/src/main/scala/filodb/prometheus/parse/Parser.scala @@ -346,13 +346,55 @@ object Parser extends Expression { def queryRangeToLogicalPlan(query: String, timeParams: TimeRangeParams): LogicalPlan = { val expression = parseQuery(query) - expression match { + val expressionWithPrecedence = expression match { + case binaryExpression: BinaryExpression => assignPrecedence(binaryExpression.lhs, binaryExpression.operator, + binaryExpression.vectorMatch, binaryExpression.rhs) + case _ => expression + } + + expressionWithPrecedence match { case p: PeriodicSeries => p.toPeriodicSeriesPlan(timeParams) case r: SimpleSeries => r.toRawSeriesPlan(timeParams, isRoot = true) case _ => throw new UnsupportedOperationException() } } + /** + * Recursively assign precedence to BinaryExpression by creating new BinaryExpression with inner expressions + * rearranged based on precedence + */ + def assignPrecedence(lhs: Expression, + operator: Operator, + vectorMatch: Option[VectorMatch], + rhs: Expression): Expression = { + rhs match { + case rhsBE: BinaryExpression => val rhsWithPrecedence = assignPrecedence(rhsBE.lhs, rhsBE.operator, + rhsBE.vectorMatch, rhsBE.rhs) // Assign Precedence to RHS Expression + rhsWithPrecedence match { + case rhsWithPrecBE: BinaryExpression => val rhsOp = rhsWithPrecBE.operator. + getPlanOperator + val precd = rhsOp.precedence - + operator.getPlanOperator.precedence + if ((precd < 0) || (precd == 0 && + !rhsOp.isRightAssociative)) { + // Assign Precedence to LHS Expression + val lhsWithPrecedence = + assignPrecedence(lhs, operator, + vectorMatch, rhsWithPrecBE.lhs) + // Create new BinaryExpression as existing precedence is not correct + // New expression will have "lhs operator rhs.lhs" first as operator.precedence > rhsOp.precedence + BinaryExpression(lhsWithPrecedence, rhsWithPrecBE.operator, + rhsWithPrecBE.vectorMatch, rhsWithPrecBE.rhs) + } else { + BinaryExpression(lhs, operator, vectorMatch, rhsWithPrecedence) + } + case _ => BinaryExpression(lhs, operator, + vectorMatch, rhsWithPrecedence) + } + case _ => BinaryExpression(lhs, operator, vectorMatch, rhs) + } + } + private def handleError(e: Error, input: String) = { val msg = "Cannot parse [" + input + "] because " + e.msg throw new IllegalArgumentException(msg) diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 4aa0b8719c..8556ecd2f2 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -2,6 +2,7 @@ package filodb.prometheus.parse import org.scalatest.{FunSpec, Matchers} import filodb.prometheus.ast.TimeStepParams +import filodb.query.{BinaryJoin, LogicalPlan} //noinspection ScalaStyle // scalastyle:off @@ -242,54 +243,6 @@ class ParserSpec extends FunSpec with Matchers { it("Should be able to make logical plans for Series Expressions") { val queryToLpString = Map( - "primary:instance-001:no.ofrequests{job=\"my-job\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(my-job)), ColumnFilter(__name__,Equals(primary:instance-001:no.ofrequests))),List()),1524855988000,1000000,1524855988000)", - "absent(nonexistent{job=\"myjob\"})" -> - "ApplyInstantFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(myjob)), ColumnFilter(__name__,Equals(nonexistent))),List()),1524855988000,1000000,1524855988000),Absent,List())", - "rate(http_requests_total[5m] offset 1w)" -> - "PeriodicSeriesWithWindowing(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000,300000,Rate,List())", - "http_requests_total{job=\"prometheus\",group=\"canary\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(group,Equals(canary)), ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", - "http_requests_total{job=\"prometheus\",__col__=\"min\"}" -> - "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),List(min)),1524855988000,1000000,1524855988000)", - // Internal FiloDB debug function - "_filodb_chunkmeta_all(http_requests_total{job=\"prometheus\"})" -> - "RawChunkMeta(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),)", - "_filodb_chunkmeta_all(http_requests_total{job=\"prometheus\",__col__=\"avg\"})" -> - "RawChunkMeta(IntervalSelector(1524855988000,1524855988000),List(ColumnFilter(job,Equals(prometheus)), ColumnFilter(__name__,Equals(http_requests_total))),avg)", - - "sum(http_requests_total) by (application, group)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(application, group),List())", - "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(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)" -> - "Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", - "label_join(up{job=\"api-server\",src1=\"a\",src2=\"b\",src3=\"c\"}, \"foo\", \",\", \"src1\", \"src2\", \"src3\")" -> - "ApplyMiscellaneousFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(job,Equals(api-server)), ColumnFilter(src1,Equals(a)), ColumnFilter(src2,Equals(b)), ColumnFilter(src3,Equals(c)), ColumnFilter(__name__,Equals(up))),List()),1524855988000,1000000,1524855988000),LabelJoin,List(foo, ,, src1, src2, src3))", - "histogram_quantile(0.9, sum(rate(http_request_duration_seconds_bucket[10m])) by (le))" -> - "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\"} / 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(),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),List())", - "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(),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))" -> - "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(job, le),List()),HistogramQuantile,List(0.9))", - "http_requests_total" -> - "PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000)", - "http_requests_total ^ 5" -> - "ScalarVectorBinaryOperation(POW,5.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),false)", - - //FIXME Operator precedence is not implemented "10 + http_requests_total * 5" -> "ScalarVectorBinaryOperation(ADD,10.0,ScalarVectorBinaryOperation(MUL,5.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),false),true)", "10 + (http_requests_total * 5)" -> @@ -339,17 +292,69 @@ class ParserSpec extends FunSpec with Matchers { "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())", "sum(http_requests_total) \n \n / \n\n sum(http_requests_total)" -> - "BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),DIV,OneToOne,Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())" + "BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),DIV,OneToOne,Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + + // Binary Expressions should generate Logical Plan according to precedence + // Logical plan generated when expression does not have brackets according to precedence is same as logical plan for expression with brackets which are according to precedence + "(10 % http_requests_total) + 5" -> + "ScalarVectorBinaryOperation(ADD,5.0,ScalarVectorBinaryOperation(MOD,10.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),true),false)", + "10 % http_requests_total + 5" -> + "ScalarVectorBinaryOperation(ADD,5.0,ScalarVectorBinaryOperation(MOD,10.0,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),true),false)", + + "(http_requests_total % http_requests_total) + http_requests_total" -> + "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),MOD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", + "http_requests_total % http_requests_total + http_requests_total" -> + "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),MOD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", + + // "unless" and "and" have same priority but are not right associative so "and" should be evaluated first + "((foo and bar) unless baz) or qux" -> + "BinaryJoin(BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),LAND,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LUnless,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(baz))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LOR,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(qux))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", + "foo and bar unless baz or qux" -> + "BinaryJoin(BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),LAND,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LUnless,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(baz))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LOR,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(qux))),List()),1524855988000,1000000,1524855988000),List(),List(),List())", + + // Pow is right associative so (bar ^ baz) should be evaluated first + "(foo ^ (bar ^ baz))" -> + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),POW,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),POW,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(baz))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + "foo ^ bar ^ baz" -> + "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),POW,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),POW,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(baz))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + + "(foo + bar) or (bla and blub)" -> + "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LOR,ManyToMany,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),LAND,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + "foo + bar or bla and blub" -> + "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LOR,ManyToMany,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),LAND,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + + "bar + on(foo) (bla / on(baz, buz) group_right(test) blub)" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(baz, buz),List(),List(test)),List(foo),List(),List())", + "bar + on(foo) bla / on(baz, buz) group_right(test) blub" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(baz, buz),List(),List(test)),List(foo),List(),List())" ) val qts: Long = 1524855988L queryToLpString.foreach { case (q, e) => info(s"Parsing $q") val lp = Parser.queryToLogicalPlan(q, qts) + if (lp.isInstanceOf[BinaryJoin]) + printBinaryJoin(lp) lp.toString shouldEqual (e) } } + private def printBinaryJoin( lp: LogicalPlan, level: Int = 0) : scala.Unit = { + if (!lp.isInstanceOf[BinaryJoin]) { + info(s"${" "*level}" + lp.toString) + } + else { + val binaryJoin = lp.asInstanceOf[BinaryJoin] + info(s"${" "*level}" + "lhs:" ) + printBinaryJoin(binaryJoin.lhs, level + 1) + info(s"${" "*level}" + "Cardinality: " + binaryJoin.cardinality) + info(s"${" "*level}" + "Operator: " + binaryJoin.operator) + info(s"${" "*level}" + "On labels: " + binaryJoin.on ) + info(s"${" "*level}" + "Include labels: " + binaryJoin.include) + info(s"${" "*level}" + "Ignoring labels: " + binaryJoin.ignoring) + info(s"${" "*level}" + "rhs: ") + printBinaryJoin(binaryJoin.rhs, level + 1) + } + } + private def parseSuccessfully(query: String) = { Parser.parseQuery(query) } diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index 7c5cfc47f2..e542b01757 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -121,64 +121,70 @@ object AggregationOperator extends Enum[AggregationOperator] { } -sealed abstract class BinaryOperator extends EnumEntry +sealed abstract class BinaryOperator extends EnumEntry { + def precedence: Int + def isRightAssociative : Boolean +} -sealed class MathOperator extends BinaryOperator +sealed class MathOperator (val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator -sealed class SetOperator extends BinaryOperator +sealed class SetOperator(val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator -sealed class ComparisonOperator extends BinaryOperator +sealed class ComparisonOperator(val precedence: Int = 0, val isRightAssociative: Boolean = false) extends BinaryOperator object BinaryOperator extends Enum[BinaryOperator] { val values = findValues + case object SUB extends MathOperator(4) - case object SUB extends MathOperator - - case object ADD extends MathOperator - - case object MUL extends MathOperator - - case object MOD extends MathOperator + case object ADD extends MathOperator(4) - case object DIV extends MathOperator + case object MUL extends MathOperator(5) - case object POW extends MathOperator + case object MOD extends MathOperator(5) - case object LAND extends SetOperator + case object DIV extends MathOperator(5) - case object LOR extends SetOperator + case object POW extends MathOperator(6, true) - case object LUnless extends SetOperator + case object LAND extends SetOperator(2) - case object EQL extends ComparisonOperator + case object LOR extends SetOperator(1) - case object NEQ extends ComparisonOperator + case object LUnless extends SetOperator(2) - case object LTE extends ComparisonOperator + case object EQL extends ComparisonOperator(3) - case object LSS extends ComparisonOperator + case object NEQ extends ComparisonOperator(3) - case object GTE extends ComparisonOperator + case object LTE extends ComparisonOperator(3) - case object GTR extends ComparisonOperator + case object LSS extends ComparisonOperator(3) - case object EQL_BOOL extends ComparisonOperator + case object GTE extends ComparisonOperator(3) - case object NEQ_BOOL extends ComparisonOperator + case object GTR extends ComparisonOperator(3) - case object LTE_BOOL extends ComparisonOperator + case object EQL_BOOL extends ComparisonOperator(3) - case object LSS_BOOL extends ComparisonOperator + case object NEQ_BOOL extends ComparisonOperator(3) - case object GTE_BOOL extends ComparisonOperator + case object LTE_BOOL extends ComparisonOperator(3) - case object GTR_BOOL extends ComparisonOperator + case object LSS_BOOL extends ComparisonOperator(3) - case object EQLRegex extends BinaryOperator // FIXME when implemented + case object GTE_BOOL extends ComparisonOperator(3) - case object NEQRegex extends BinaryOperator // FIXME when implemented + case object GTR_BOOL extends ComparisonOperator(3) + case object EQLRegex extends BinaryOperator { // FIXME when implemented + override def precedence: Int = 0 + override def isRightAssociative: Boolean = false + } + case object NEQRegex extends BinaryOperator { // FIXME when implemented + override def precedence: Int = 0 + override def isRightAssociative: Boolean = false + } } sealed trait Cardinality extends EnumEntry From 39993650982f0f7e13575930d0f7a6c336dea535 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Tue, 10 Sep 2019 14:02:37 -0700 Subject: [PATCH 24/28] feat(query): changes function support (#478) --- .../format/vectors/DeltaDeltaVector.scala | 29 ++++++++++++++++ .../format/vectors/DoubleVector.scala | 25 ++++++++++++++ .../format/vectors/LongBinaryVector.scala | 21 ++++++++++++ .../format/vectors/LongVectorTest.scala | 22 ++++++++++++ .../filodb/prometheus/parse/ParserSpec.scala | 4 +++ .../exec/rangefn/AggrOverTimeFunctions.scala | 34 +++++++++++++++++++ .../query/exec/rangefn/RangeFunction.scala | 2 ++ .../rangefn/AggrOverTimeFunctionsSpec.scala | 13 +++++++ scalastyle-test-config.xml | 2 +- 9 files changed, 151 insertions(+), 1 deletion(-) diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala index 5bbfe734d6..e2625d87f5 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala @@ -206,6 +206,26 @@ object DeltaDeltaDataReader extends LongVectorDataReader { val innerIt = IntBinaryVector.simple(inner).iterate(inner, startElement) new DeltaDeltaIterator(innerIt, slope(vector), initValue(vector) + startElement * slope(vector).toLong) } + + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { + if (slope(vector) == 0) { + 0 + } else { + require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") + val itr = iterate(vector, start) + var prev: Long = 0 + var changes = 0 + for {i <- start until end optimized} { + val cur = itr.next + if (i == start) //Initialize prev + prev = cur + if (prev != cur) + changes += 1 + prev = cur + } + changes + } + } } /** @@ -253,6 +273,15 @@ object DeltaDeltaConstDataReader extends LongVectorDataReader { out } } + + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { + if (slope(vector) == 0) { + 0 + } else { + require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") + length(vector) - 1 + } + } } // TODO: validate args, esp base offset etc, somehow. Need to think about this for the many diff classes. diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 806a52dab4..82e7557239 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -143,6 +143,8 @@ trait DoubleVectorDataReader extends CounterVectorReader { */ def count(vector: BinaryVectorPtr, start: Int, end: Int): Int + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double + /** * Converts the BinaryVector to an unboxed Buffer. * Only returns elements that are "available". @@ -244,6 +246,22 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader { } count } + + final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = { + require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") + var prev = Double.NaN + var addr = vector + OffsetData + start * 8 + val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end + var changes = 0d + while (addr < untilAddr) { + val nextDbl = UnsafeUtils.getDouble(addr) + // There are many possible values of NaN. Use a function to ignore them reliably. + if (!java.lang.Double.isNaN(nextDbl) && prev != nextDbl && !java.lang.Double.isNaN(prev)) changes += 1 + addr += 8 + prev = nextDbl + } + changes + } } // Corrects and caches ONE underlying chunk. @@ -257,6 +275,7 @@ extends DoubleVectorDataReader { def sum(vector: BinaryVectorPtr, start: Int, end: Int, ignoreNaN: Boolean = true): Double = inner.sum(vector, start, end, ignoreNaN) def count(vector: BinaryVectorPtr, start: Int, end: Int): Int = inner.count(vector, start, end) + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = inner.changes(vector, start, end) var _correction = 0.0 // Lazily correct - not all queries want corrected data @@ -315,6 +334,9 @@ object MaskedDoubleDataReader extends DoubleVectorDataReader with BitmapMaskVect override def iterate(vector: BinaryVectorPtr, startElement: Int = 0): DoubleIterator = DoubleVector(subvectAddr(vector)).iterate(subvectAddr(vector), startElement) + + override def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = + DoubleVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end) } class DoubleAppendingVector(addr: BinaryRegion.NativePointer, maxBytes: Int, val dispose: () => Unit) @@ -452,4 +474,7 @@ object DoubleLongWrapDataReader extends DoubleVectorDataReader { final def count(vector: BinaryVectorPtr, start: Int, end: Int): Int = end - start + 1 final def iterate(vector: BinaryVectorPtr, startElement: Int = 0): DoubleIterator = new DoubleLongWrapIterator(LongBinaryVector(vector).iterate(vector, startElement)) + + final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = + LongBinaryVector(vector).changes(vector, start, end) } diff --git a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala index a150b7b276..e361b2574e 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala @@ -138,6 +138,7 @@ trait LongVectorDataReader extends VectorDataReader { */ def binarySearch(vector: BinaryVectorPtr, item: Long): Int + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int /** * Searches for the last element # whose element is <= the item, assuming all elements are increasing. * Typically used to find the last timestamp <= item. @@ -227,6 +228,23 @@ object LongVectorDataReader64 extends LongVectorDataReader { } if (element == item) first else first | 0x80000000 } + + final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { + require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") + var prev: Long = 0 + val startAddr = vector + OffsetData + start * 8 + val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end + var changes = 0 + var addr = startAddr + while (addr < untilAddr) { + val cur = UnsafeUtils.getLong(addr) + if (addr == startAddr) prev = cur + if (prev != cur) changes += 1 + prev = cur + addr += 8 + } + changes + } } /** @@ -249,6 +267,9 @@ object MaskedLongDataReader extends LongVectorDataReader with BitmapMaskVector { def binarySearch(vector: BinaryVectorPtr, item: Long): Int = LongBinaryVector(subvectAddr(vector)).binarySearch(subvectAddr(vector), item) + + def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = + LongBinaryVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end) } class LongAppendingVector(addr: BinaryRegion.NativePointer, maxBytes: Int, val dispose: () => Unit) diff --git a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala index fe3d02dc50..0795c798fc 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala @@ -373,4 +373,26 @@ class LongVectorTest extends NativeVectorTest with PropertyChecks { out shouldEqual 0x80000000 } } + + it("should do changes on DeltaDeltaConstVector") { + val builder = LongBinaryVector.appendingVector(memFactory, 100) + val longVal = Int.MaxValue.toLong + 100 + (0 to 4).foreach(n => builder.addData(longVal)) + val ptr = builder.optimize(memFactory) + BinaryVector.majorVectorType(ptr) shouldEqual WireFormat.VECTORTYPE_DELTA2 + val readVect = LongBinaryVector(ptr) + readVect shouldEqual DeltaDeltaConstDataReader + readVect.changes(ptr,0, 4) shouldEqual(0) + } + + it("should do changes on DeltaDeltaDataReader") { + val orig = Seq(1000L, 2001L, 5123L, 5123L, 5250L, 6004L, 6004L) + val builder = LongBinaryVector.appendingVectorNoNA(memFactory, orig.length) + orig.foreach(builder.addData) + builder.length shouldEqual orig.length + val ptr = builder.optimize(memFactory) + val readVect = LongBinaryVector(ptr) + readVect shouldEqual DeltaDeltaDataReader + LongBinaryVector(ptr).changes(ptr, 0, 6) shouldEqual(4) + } } \ No newline at end of file diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index 8556ecd2f2..ce4e54d4a8 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -294,6 +294,10 @@ class ParserSpec extends FunSpec with Matchers { "sum(http_requests_total) \n \n / \n\n sum(http_requests_total)" -> "BinaryJoin(Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),DIV,OneToOne,Aggregate(Sum,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", + "changes(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,Changes,List())", + + // Binary Expressions should generate Logical Plan according to precedence // Logical plan generated when expression does not have brackets according to precedence is same as logical plan for expression with brackets which are according to precedence "(10 % http_requests_total) + 5" -> 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 37f4cb94a5..ca7d8a0a51 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -550,4 +550,38 @@ class StdVarOverTimeChunkedFunctionL extends VarOverTimeChunkedFunctionL() { val stdVar = squaredSum/count - avg*avg sampleToEmit.setValues(endTimestamp, stdVar) } +} + +abstract class ChangesChunkedFunction(var changes: Double = Double.NaN) extends ChunkedRangeFunction[TransientRow] { + override final def reset(): Unit = { changes = Double.NaN } + final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { + sampleToEmit.setValues(endTimestamp, changes) + } +} + +class ChangesChunkedFunctionD() extends ChangesChunkedFunction() with + ChunkedDoubleRangeFunction { + final def addTimeDoubleChunks(doubleVect: BinaryVector.BinaryVectorPtr, + doubleReader: bv.DoubleVectorDataReader, + startRowNum: Int, + endRowNum: Int): Unit = { + if (changes.isNaN) { + changes = 0d + } + changes += doubleReader.changes(doubleVect, startRowNum, endRowNum) + } +} + +// scalastyle:off +class ChangesChunkedFunctionL extends ChangesChunkedFunction with + ChunkedLongRangeFunction{ + final def addTimeLongChunks(longVect: BinaryVector.BinaryVectorPtr, + longReader: bv.LongVectorDataReader, + startRowNum: Int, + endRowNum: Int): Unit = { + if (changes.isNaN) { + changes = 0d + } + changes += longReader.changes(longVect, startRowNum, endRowNum) + } } \ No newline at end of file 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 251627a9bf..774b7913dc 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/RangeFunction.scala @@ -303,6 +303,7 @@ object RangeFunction { case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionL case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionL case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionL + case Some(Changes) => () => new ChangesChunkedFunctionL case _ => iteratingFunction(func, funcParams) } } @@ -331,6 +332,7 @@ object RangeFunction { case Some(MaxOverTime) => () => new MaxOverTimeChunkedFunctionD case Some(StdDevOverTime) => () => new StdDevOverTimeChunkedFunctionD case Some(StdVarOverTime) => () => new StdVarOverTimeChunkedFunctionD + case Some(Changes) => () => new ChangesChunkedFunctionD() case _ => iteratingFunction(func, funcParams) } } diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index d9205067db..9008dfd2e3 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -293,4 +293,17 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { } } + it("should correctly do changes") { + var data = Seq(1.5, 2.5, 3.5, 4.5, 5.5) + val rv = timeValueRV(data) + val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList + + val windowSize = 100 + val step = 20 + + val chunkedIt = new ChunkedWindowIteratorD(rv, 100000, 20000, 150000, 30000, + new ChangesChunkedFunctionD(), queryConfig) + val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList + aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) + } } diff --git a/scalastyle-test-config.xml b/scalastyle-test-config.xml index ccbccd6fdd..dbb4464a70 100644 --- a/scalastyle-test-config.xml +++ b/scalastyle-test-config.xml @@ -91,7 +91,7 @@ - + From 98911a193312a3d693fd04b28cd102491af3815d Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Mon, 16 Sep 2019 15:11:32 -0700 Subject: [PATCH 25/28] feat(query): sort and sort desc function (#490) --- .../queryengine2/QueryEngine.scala | 19 ++++ .../queryengine2/RoutingPlanner.scala | 16 ++-- .../filodb/prometheus/ast/Functions.scala | 10 ++- .../filodb/prometheus/parse/ParserSpec.scala | 6 +- .../main/scala/filodb/query/LogicalPlan.scala | 26 +++++- .../main/scala/filodb/query/PlanEnums.scala | 13 ++- .../query/exec/RangeVectorTransformer.scala | 44 ++++++++- .../exec/rangefn/MiscellaneousFunction.scala | 2 - .../exec/rangefn/InstantFunctionSpec.scala | 9 -- .../query/exec/rangefn/SortFunctionSpec.scala | 89 +++++++++++++++++++ .../query/util/LogicalPlanUtilSpec.scala | 76 ++++++++++++++++ 11 files changed, 278 insertions(+), 32 deletions(-) create mode 100644 query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala create mode 100644 query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala index fbdd351354..dc07d72f54 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/QueryEngine.scala @@ -245,6 +245,8 @@ class QueryEngine(dataset: Dataset, spreadProvider) case lp: ApplyMiscellaneousFunction => materializeApplyMiscellaneousFunction(queryId, submitTime, options, lp, spreadProvider) + case lp: ApplySortFunction => materializeApplySortFunction(queryId, submitTime, options, lp, + spreadProvider) } } @@ -445,6 +447,23 @@ class QueryEngine(dataset: Dataset, vectors } + private def materializeApplySortFunction(queryId: String, + submitTime: Long, + options: QueryOptions, + lp: ApplySortFunction, + spreadProvider: SpreadProvider): PlanResult = { + val vectors = walkLogicalPlanTree(lp.vectors, queryId, submitTime, options, spreadProvider) + if(vectors.plans.length > 1) { + val targetActor = pickDispatcher(vectors.plans) + val topPlan = DistConcatExec(queryId, targetActor, vectors.plans) + topPlan.addRangeVectorTransformer((SortFunctionMapper(lp.function))) + PlanResult(Seq(topPlan), vectors.needsStitch) + } else { + vectors.plans.foreach(_.addRangeVectorTransformer(SortFunctionMapper(lp.function))) + vectors + } + } + /** * Renames Prom AST __name__ metric name filters to one based on the actual metric column of the dataset, * if it is not the prometheus standard diff --git a/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala b/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala index 295b1ef5e8..1554fa4079 100644 --- a/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala +++ b/coordinator/src/main/scala/filodb.coordinator/queryengine2/RoutingPlanner.scala @@ -171,6 +171,7 @@ object QueryRoutingPlanner extends RoutingPlanner { case lp: BinaryJoin => getPeriodicSeriesTimeFromLogicalPlan(lp.lhs) // can assume lhs & rhs have same time case lp: ScalarVectorBinaryOperation => getPeriodicSeriesTimeFromLogicalPlan(lp.vector) case lp: ApplyMiscellaneousFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) + case lp: ApplySortFunction => getPeriodicSeriesTimeFromLogicalPlan(lp.vectors) case _ => throw new BadQueryException("Invalid logical plan") } } @@ -194,7 +195,8 @@ object QueryRoutingPlanner extends RoutingPlanner { lookBackTime)) case lp: ApplyMiscellaneousFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) - case _ => throw new UnsupportedOperationException("Logical plan not supportred for copy") + case lp: ApplySortFunction => lp.copy(vectors = copyWithUpdatedTimeRange(lp.vectors, timeRange, lookBackTime)) + case _ => throw new UnsupportedOperationException("Logical plan not supported for copy") } } @@ -203,7 +205,6 @@ object QueryRoutingPlanner extends RoutingPlanner { */ def copyRawSeriesWithUpdatedTimeRange(rawSeriesPlan: RawSeriesPlan, timeRange: TimeRange, lookBackTime: Long): RawSeries = { - rawSeriesPlan match { case rs: RawSeries => rs.rangeSelector match { case is: IntervalSelector => rs.copy(rangeSelector = is.copy(timeRange.startInMillis - lookBackTime, @@ -219,19 +220,12 @@ object QueryRoutingPlanner extends RoutingPlanner { * NOTE: Plan should be PeriodicSeriesPlan */ def getRawSeriesStartTime(logicalPlan: LogicalPlan): Option[Long] = { - logicalPlan match { + LogicalPlan.findLeafLogicalPlans(logicalPlan).head match { case lp: RawSeries => lp.rangeSelector match { case rs: IntervalSelector => Some(rs.from) case _ => None } - case lp: PeriodicSeries => getRawSeriesStartTime(lp.rawSeries) - case lp: PeriodicSeriesWithWindowing => getRawSeriesStartTime(lp.rawSeries) - case lp: ApplyInstantFunction => getRawSeriesStartTime(lp.vectors) - case lp: Aggregate => getRawSeriesStartTime(lp.vectors) - case lp: BinaryJoin => getRawSeriesStartTime(lp.lhs) // can assume lhs & rhs have same time - case lp: ScalarVectorBinaryOperation => getRawSeriesStartTime(lp.vector) - case lp: ApplyMiscellaneousFunction => getRawSeriesStartTime(lp.vectors) - case _ => None + case _ => throw new BadQueryException("Invalid logical plan") } } } diff --git a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala index 0db0ae778e..eee5f63ba2 100644 --- a/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala +++ b/prometheus/src/main/scala/filodb/prometheus/ast/Functions.scala @@ -11,10 +11,12 @@ trait Functions extends Base with Operators with Vectors { InstantFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && RangeFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && FiloFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && - MiscellaneousFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty) { + MiscellaneousFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty && + SortFunctionId.withNameLowercaseOnlyOption(name.toLowerCase).isEmpty) { throw new IllegalArgumentException(s"Invalid function name [$name]") } + // scalastyle:off def toPeriodicSeriesPlan(timeParams: TimeRangeParams): PeriodicSeriesPlan = { val seriesParam = allParams.filter(_.isInstanceOf[Series]).head.asInstanceOf[Series] val otherParams = allParams.filter(!_.equals(seriesParam)).map { @@ -26,6 +28,7 @@ trait Functions extends Base with Operators with Vectors { val instantFunctionIdOpt = InstantFunctionId.withNameInsensitiveOption(name) val filoFunctionIdOpt = FiloFunctionId.withNameInsensitiveOption(name) val miscellaneousFunctionIdOpt = MiscellaneousFunctionId.withNameInsensitiveOption(name) + val sortFunctionIdOpt = SortFunctionId.withNameInsensitiveOption(name) if (instantFunctionIdOpt.isDefined) { val instantFunctionId = instantFunctionIdOpt.get @@ -49,6 +52,11 @@ trait Functions extends Base with Operators with Vectors { val periodicSeriesPlan = seriesParam.asInstanceOf[PeriodicSeries].toPeriodicSeriesPlan(timeParams) ApplyMiscellaneousFunction(periodicSeriesPlan, miscellaneousFunctionId, otherParams) + } else if (sortFunctionIdOpt.isDefined) { + val sortFunctionId = sortFunctionIdOpt.get + val periodicSeriesPlan = seriesParam.asInstanceOf[PeriodicSeries].toPeriodicSeriesPlan(timeParams) + + ApplySortFunction(periodicSeriesPlan, sortFunctionId, otherParams) } else { val rangeFunctionId = RangeFunctionId.withNameInsensitiveOption(name).get diff --git a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala index ce4e54d4a8..379a3f817d 100644 --- a/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala +++ b/prometheus/src/test/scala/filodb/prometheus/parse/ParserSpec.scala @@ -328,7 +328,11 @@ class ParserSpec extends FunSpec with Matchers { "BinaryJoin(BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(foo))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),LOR,ManyToMany,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),LAND,ManyToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(),List(),List()),List(),List(),List())", "bar + on(foo) (bla / on(baz, buz) group_right(test) blub)" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(baz, buz),List(),List(test)),List(foo),List(),List())", - "bar + on(foo) bla / on(baz, buz) group_right(test) blub" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(baz, buz),List(),List(test)),List(foo),List(),List())" + "bar + on(foo) bla / on(baz, buz) group_right(test) blub" -> "BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bar))),List()),1524855988000,1000000,1524855988000),ADD,OneToOne,BinaryJoin(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(bla))),List()),1524855988000,1000000,1524855988000),DIV,OneToMany,PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(blub))),List()),1524855988000,1000000,1524855988000),List(baz, buz),List(),List(test)),List(foo),List(),List())", + "sort(http_requests_total)" -> + "ApplySortFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),Sort,List())", + "sort_desc(http_requests_total)" -> + "ApplySortFunction(PeriodicSeries(RawSeries(IntervalSelector(1524855688000,1524855988000),List(ColumnFilter(__name__,Equals(http_requests_total))),List()),1524855988000,1000000,1524855988000),SortDesc,List())" ) val qts: Long = 1524855988L diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index 3874b24b95..cc70103fb2 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -135,4 +135,28 @@ case class ApplyInstantFunction(vectors: PeriodicSeriesPlan, */ case class ApplyMiscellaneousFunction(vectors: PeriodicSeriesPlan, function: MiscellaneousFunctionId, - functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan \ No newline at end of file + functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(vectors) +} + +/** + * Apply Sort Function to a collection of RangeVectors + */ +case class ApplySortFunction(vectors: PeriodicSeriesPlan, + function: SortFunctionId, + functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(vectors) +} + +object LogicalPlan { + /** + * Get leaf Logical Plans + */ + def findLeafLogicalPlans (logicalPlan: LogicalPlan) : Seq[LogicalPlan] = { + logicalPlan match { + // Find leaf logical plans for all children and concatenate results + case lp: NonLeafLogicalPlan => lp.children.flatMap(findLeafLogicalPlans(_)) + case _ => Seq(logicalPlan) + } + } +} diff --git a/query/src/main/scala/filodb/query/PlanEnums.scala b/query/src/main/scala/filodb/query/PlanEnums.scala index e542b01757..6c66a3d748 100644 --- a/query/src/main/scala/filodb/query/PlanEnums.scala +++ b/query/src/main/scala/filodb/query/PlanEnums.scala @@ -223,16 +223,21 @@ object MiscellaneousFunctionId extends Enum[MiscellaneousFunctionId] { case object Month extends MiscellaneousFunctionId("month") - case object Sort extends MiscellaneousFunctionId("sort") - - case object SortDesc extends MiscellaneousFunctionId("sort_desc") - case object Timestamp extends MiscellaneousFunctionId("timestamp") case object Year extends MiscellaneousFunctionId("year") } +sealed abstract class SortFunctionId(override val entryName: String) extends EnumEntry + +object SortFunctionId extends Enum[SortFunctionId] { + val values = findValues + case object Sort extends SortFunctionId("sort") + + case object SortDesc extends SortFunctionId("sort_desc") +} + //scalastyle:on diff --git a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala index f5c318b6d9..781c1f3063 100644 --- a/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala +++ b/query/src/main/scala/filodb/query/exec/RangeVectorTransformer.scala @@ -3,12 +3,14 @@ package filodb.query.exec import monix.reactive.Observable import filodb.core.metadata.Column.ColumnType +import filodb.core.metadata.Column.ColumnType.DoubleColumn import filodb.core.metadata.Dataset import filodb.core.query._ import filodb.memory.format.RowReader -import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig} +import filodb.query.{BinaryOperator, InstantFunctionId, MiscellaneousFunctionId, QueryConfig, SortFunctionId} import filodb.query.InstantFunctionId.HistogramQuantile import filodb.query.MiscellaneousFunctionId.{LabelJoin, LabelReplace} +import filodb.query.SortFunctionId.{Sort, SortDesc} import filodb.query.exec.binaryOp.BinaryOperatorFunction import filodb.query.exec.rangefn._ @@ -198,8 +200,8 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, val miscFunction: MiscellaneousFunction = { function match { case LabelReplace => LabelReplaceFunction(funcParams) - case LabelJoin => LabelJoinFunction(funcParams) - case _ => throw new UnsupportedOperationException(s"$function not supported.") + case LabelJoin => LabelJoinFunction(funcParams) + case _ => throw new UnsupportedOperationException(s"$function not supported.") } } @@ -210,4 +212,40 @@ final case class MiscellaneousFunctionMapper(function: MiscellaneousFunctionId, sourceSchema: ResultSchema): Observable[RangeVector] = { miscFunction.execute(source) } +} + +final case class SortFunctionMapper(function: SortFunctionId) extends RangeVectorTransformer { + protected[exec] def args: String = + s"function=$function" + + def apply(dataset: Dataset, + source: Observable[RangeVector], + queryConfig: QueryConfig, + limit: Int, + sourceSchema: ResultSchema): Observable[RangeVector] = { + if (sourceSchema.columns(1).colType == DoubleColumn) { + + val ordering: Ordering[Double] = function match { + case Sort => (Ordering[Double]) + case SortDesc => (Ordering[Double]).reverse + case _ => throw new UnsupportedOperationException(s"$function not supported.") + } + + val resultRv = source.toListL.map { rvs => + rvs.map { rv => + new RangeVector { + override def key: RangeVectorKey = rv.key + + override def rows: Iterator[RowReader] = new BufferableIterator(rv.rows).buffered + } + }.sortBy { rv => rv.rows.asInstanceOf[BufferedIterator[RowReader]].head.getDouble(1) + }(ordering) + + }.map(Observable.fromIterable) + + Observable.fromTask(resultRv).flatten + } else { + source + } + } } \ No newline at end of file diff --git a/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala index ee0366a5d9..3620f21021 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/MiscellaneousFunction.scala @@ -124,5 +124,3 @@ case class LabelJoinFunction(funcParams: Seq[Any]) } } - - diff --git a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala index 9ab285ff41..c0db20cfd5 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/InstantFunctionSpec.scala @@ -157,15 +157,6 @@ class InstantFunctionSpec extends RawDataWindowingSpec with ScalaFutures { applyFunctionAndAssertResult(samples, expected10, InstantFunctionId.Round, Seq(10)) } - it ("should handle unknown functions") { - // sort_desc - the[UnsupportedOperationException] thrownBy { - val miscellaneousVectorFnMapper = exec.MiscellaneousFunctionMapper(MiscellaneousFunctionId.SortDesc) - miscellaneousVectorFnMapper(MetricsTestData.timeseriesDataset, - Observable.fromIterable(sampleBase), queryConfig, 1000, resultSchema) - } should have message "SortDesc not supported." - } - it ("should validate invalid function params") { // clamp_max the[IllegalArgumentException] thrownBy { diff --git a/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala new file mode 100644 index 0000000000..83b031f6bd --- /dev/null +++ b/query/src/test/scala/filodb/query/exec/rangefn/SortFunctionSpec.scala @@ -0,0 +1,89 @@ +package filodb.query.exec.rangefn + +import com.typesafe.config.{Config, ConfigFactory} +import filodb.core.MetricsTestData +import filodb.core.query.{CustomRangeVectorKey, RangeVector, RangeVectorKey, ResultSchema} +import filodb.memory.format.{RowReader, ZeroCopyUTF8String} +import filodb.query.exec.TransientRow +import filodb.query.{QueryConfig, SortFunctionId, exec} +import monix.execution.Scheduler.Implicits.global +import monix.reactive.Observable +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.{FunSpec, Matchers} + +class SortFunctionSpec extends FunSpec with Matchers with ScalaFutures { + val config: Config = ConfigFactory.load("application_test.conf").getConfig("filodb") + val resultSchema = ResultSchema(MetricsTestData.timeseriesDataset.infosFromIDs(0 to 1), 1) + val queryConfig = new QueryConfig(config.getConfig("query")) + val ignoreKey = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("ignore") -> ZeroCopyUTF8String("ignore"))) + + + val testKey1 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-10"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val testKey2 = CustomRangeVectorKey( + Map(ZeroCopyUTF8String("src") -> ZeroCopyUTF8String("source-value-20"), + ZeroCopyUTF8String("dst") -> ZeroCopyUTF8String("original-destination-value"))) + + val testSample: Array[RangeVector] = Array( + new RangeVector { + override def key: RangeVectorKey = testKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 1d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey2 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 5d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 3d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 2d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey2 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 4d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey2 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 6d)).iterator + }, + new RangeVector { + override def key: RangeVectorKey = testKey1 + + override def rows: Iterator[RowReader] = Seq( + new TransientRow(1L, 0d)).iterator + }) + + it("should sort instant vectors in ascending order") { + val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.Sort) + val resultObs = sortFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) + resultRows.shouldEqual(List(0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) + } + + it("should sort instant vectors in descending order") { + val sortFunctionMapper = exec.SortFunctionMapper(SortFunctionId.SortDesc) + val resultObs = sortFunctionMapper(MetricsTestData.timeseriesDataset, + Observable.fromIterable(testSample), queryConfig, 1000, resultSchema) + val resultRows = resultObs.toListL.runAsync.futureValue.flatMap(_.rows.map(_.getDouble(1)).toList) + resultRows.shouldEqual(List(6.0, 5.0, 4.0, 3.0, 2.0, 1.0, 0.0)) + } +} diff --git a/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala b/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala new file mode 100644 index 0000000000..04cdbfcd3d --- /dev/null +++ b/query/src/test/scala/filodb/query/util/LogicalPlanUtilSpec.scala @@ -0,0 +1,76 @@ +package filodb.query.util + +import org.scalatest.{FunSpec, Matchers} + +import filodb.core.query.ColumnFilter +import filodb.core.query.Filter.{Equals, In} +import filodb.query._ +import filodb.query.BinaryOperator.DIV +import filodb.query.Cardinality.OneToOne +import filodb.query.RangeFunctionId.SumOverTime + +class LogicalPlanUtilSpec extends FunSpec with Matchers { + + it("should get MetricName from logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance")) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name_") + res.get.shouldEqual(Set("MetricName")) + } + + it("should get LabelName from logicalPlan with filter In") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", In(Set("Inst-0", "Inst-1")))), Seq("_name_", "instance")) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "instance") + res.get.shouldEqual(Set("Inst-0", "Inst-1")) + } + + it("should get MetricName from BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance")) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance")) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "_name_") + res.get.shouldEqual(Set("MetricName1")) + } + + it("should return None if label value is not present in logicalPlan") { + + val rawSeries = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance")) + val periodicSeriesWithWindowing = PeriodicSeriesWithWindowing(rawSeries, 1000, 500, 5000, 100, SumOverTime) + + val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(periodicSeriesWithWindowing, "_name") + res.isEmpty shouldEqual(true) + } + + it("should concatenate results from lhs and rhs for BinaryJoin LogicalPlan") { + + val rawSeriesLhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("_name_", Equals("MetricName1")), + ColumnFilter("instance", Equals("Inst-0"))), Seq("_name_", "instance")) + val lhs = PeriodicSeries(rawSeriesLhs, 1000, 500, 50000) + + val rawSeriesRhs = RawSeries(IntervalSelector(1000, 3000), Seq(ColumnFilter("job", Equals("MetricName2")), + ColumnFilter("instance", Equals("Inst-1"))), Seq("job", "instance")) + val rhs = PeriodicSeries(rawSeriesRhs, 1000, 500, 50000) + + val binaryJoin = BinaryJoin(lhs, DIV, OneToOne, rhs) + + val res = LogicalPlanUtil.getLabelValueFromLogicalPlan(binaryJoin, "instance") + res.get.shouldEqual(Set("Inst-1", "Inst-0")) + } + +} From 4cdc1e7a797208a930bec6b712b4124ae1619433 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Mon, 26 Aug 2019 17:36:13 -0700 Subject: [PATCH 26/28] feat(util) : util to get value for a columnName from logical plan (#479) --- .../scala/filodb.core/query/KeyFilter.scala | 7 ++++ .../main/scala/filodb/query/LogicalPlan.scala | 28 ++++++++++++--- .../filodb/query/util/LogicalPlanUtil.scala | 34 +++++++++++++++++++ 3 files changed, 64 insertions(+), 5 deletions(-) create mode 100644 query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala diff --git a/core/src/main/scala/filodb.core/query/KeyFilter.scala b/core/src/main/scala/filodb.core/query/KeyFilter.scala index 3191465eb8..4c07934cff 100644 --- a/core/src/main/scala/filodb.core/query/KeyFilter.scala +++ b/core/src/main/scala/filodb.core/query/KeyFilter.scala @@ -7,35 +7,42 @@ import filodb.memory.format.{SingleValueRowReader, UTF8Wrapper, ZeroCopyUTF8Stri sealed trait Filter { def filterFunc: Any => Boolean + def valuesStrings : Set[Any] } object Filter { final case class Equals(value: Any) extends Filter { val filterFunc: Any => Boolean = (item: Any) => value.equals(item) + def valuesStrings: Set[Any] = Set(value) } final case class In(values: Set[Any]) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => values.contains(item) + def valuesStrings: Set[Any] = values } final case class And(left: Filter, right: Filter) extends Filter { private val leftFunc = left.filterFunc private val rightFunc = right.filterFunc val filterFunc: (Any) => Boolean = (item: Any) => leftFunc(item) && rightFunc(item) + def valuesStrings: Set[Any] = left.valuesStrings.union(right.valuesStrings) } final case class NotEquals(value: Any) extends Filter { val filterFunc: (Any) => Boolean = (item: Any) => !value.equals(item) + def valuesStrings: Set[Any] = Set(value) } final case class EqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => pattern.matcher(item.toString).matches() + def valuesStrings: Set[Any] = Set(value) } final case class NotEqualsRegex(value: Any) extends Filter { val pattern = Pattern.compile(value.toString, Pattern.DOTALL) val filterFunc: (Any) => Boolean = (item: Any) => !pattern.matcher(item.toString).matches() + def valuesStrings: Set[Any] = Set(value) } } diff --git a/query/src/main/scala/filodb/query/LogicalPlan.scala b/query/src/main/scala/filodb/query/LogicalPlan.scala index cc70103fb2..e4cd18ad45 100644 --- a/query/src/main/scala/filodb/query/LogicalPlan.scala +++ b/query/src/main/scala/filodb/query/LogicalPlan.scala @@ -9,6 +9,10 @@ sealed trait LogicalPlan */ sealed trait RawSeriesPlan extends LogicalPlan +sealed trait NonLeafLogicalPlan extends LogicalPlan { + def children: Seq[LogicalPlan] +} + /** * Super class for a query that results in range vectors with samples * in regular steps @@ -68,7 +72,9 @@ case class RawChunkMeta(rangeSelector: RangeSelector, case class PeriodicSeries(rawSeries: RawSeriesPlan, start: Long, step: Long, - end: Long) extends PeriodicSeriesPlan + end: Long) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(rawSeries) +} /** * Concrete logical plan to query for data in a given range @@ -83,7 +89,10 @@ case class PeriodicSeriesWithWindowing(rawSeries: RawSeries, end: Long, window: Long, function: RangeFunctionId, - functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan + functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan with NonLeafLogicalPlan +{ + override def children: Seq[LogicalPlan] = Seq(rawSeries) +} /** * Aggregate data across partitions (not in the time dimension). @@ -96,7 +105,9 @@ case class Aggregate(operator: AggregationOperator, vectors: PeriodicSeriesPlan, params: Seq[Any] = Nil, by: Seq[String] = Nil, - without: Seq[String] = Nil) extends PeriodicSeriesPlan + without: Seq[String] = Nil) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(vectors) +} /** * Binary join between collections of RangeVectors. @@ -114,6 +125,9 @@ case class BinaryJoin(lhs: PeriodicSeriesPlan, rhs: PeriodicSeriesPlan, on: Seq[String] = Nil, ignoring: Seq[String] = Nil, include: Seq[String] = Nil) extends PeriodicSeriesPlan + with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(lhs, rhs) +} /** * Apply Scalar Binary operation to a collection of RangeVectors @@ -121,14 +135,18 @@ case class BinaryJoin(lhs: PeriodicSeriesPlan, case class ScalarVectorBinaryOperation(operator: BinaryOperator, scalar: AnyVal, vector: PeriodicSeriesPlan, - scalarIsLhs: Boolean) extends PeriodicSeriesPlan + scalarIsLhs: Boolean) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(vector) +} /** * Apply Instant Vector Function to a collection of RangeVectors */ case class ApplyInstantFunction(vectors: PeriodicSeriesPlan, function: InstantFunctionId, - functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan + functionArgs: Seq[Any] = Nil) extends PeriodicSeriesPlan with NonLeafLogicalPlan { + override def children: Seq[LogicalPlan] = Seq(vectors) +} /** * Apply Miscellaneous Function to a collection of RangeVectors diff --git a/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala b/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala new file mode 100644 index 0000000000..33b78831ca --- /dev/null +++ b/query/src/main/scala/filodb/query/util/LogicalPlanUtil.scala @@ -0,0 +1,34 @@ +package filodb.query.util + +import filodb.core.query.ColumnFilter +import filodb.query._ + +object LogicalPlanUtil { + + private def getLabelValueFromFilters(filters: Seq[ColumnFilter], labelName: String): Option[Set[String]] = { + val matchingFilters = filters.filter(_.column.equals(labelName)) + if (matchingFilters.isEmpty) + None + else + Some(matchingFilters.head.filter.valuesStrings.map(_.toString)) + } + + def getLabelValueFromLogicalPlan(logicalPlan: LogicalPlan, labelName: String): Option[Set[String]] = { + val labelValues = LogicalPlan.findLeafLogicalPlans(logicalPlan).flatMap { lp => + lp match { + case lp: LabelValues => lp.labelConstraints.get(labelName).map(Set(_)) + case lp: RawSeries => getLabelValueFromFilters(lp.filters, labelName) + case lp: RawChunkMeta => getLabelValueFromFilters(lp.filters, labelName) + case lp: SeriesKeysByFilters => getLabelValueFromFilters(lp.filters, labelName) + case _ => throw new BadQueryException("Invalid logical plan") + } + } + if (labelValues.isEmpty) { + None + } else { + var res: Set[String] = Set() + // Concatenate results + Some(labelValues.foldLeft(res) { (acc, i) => i.union(acc) }) + } + } +} From b81906141df4b328e56be1ed456e25e348695745 Mon Sep 17 00:00:00 2001 From: Jackson Jeyapaul Date: Thu, 26 Sep 2019 14:35:42 -0700 Subject: [PATCH 27/28] feat(core): Honor order in copyTags (#507) --- .../scala/filodb.core/metadata/Dataset.scala | 12 +++++++---- .../gateway/conversion/InputRecord.scala | 20 +++++++++---------- .../filodb/prometheus/FormatConversion.scala | 2 +- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/filodb.core/metadata/Dataset.scala b/core/src/main/scala/filodb.core/metadata/Dataset.scala index da1bae4996..f831a334eb 100644 --- a/core/src/main/scala/filodb.core/metadata/Dataset.scala +++ b/core/src/main/scala/filodb.core/metadata/Dataset.scala @@ -166,7 +166,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], ignoreShardKeyColumnSuffixes: Map[String, Seq[String]] = Map.empty, ignoreTagsOnPartitionKeyHash: Seq[String] = Nil, // For each key, copy the tag to the value if the value is absent - copyTags: Map[String, String] = Map.empty) { + copyTags: Seq[(String, String)] = Seq.empty) { override def toString: String = { toConfig.root.render(ConfigRenderOptions.concise) } @@ -180,7 +180,7 @@ case class DatasetOptions(shardKeyColumns: Seq[String], "ignoreShardKeyColumnSuffixes" -> ignoreShardKeyColumnSuffixes.mapValues(_.asJava).asJava, "ignoreTagsOnPartitionKeyHash" -> ignoreTagsOnPartitionKeyHash.asJava, - "copyTags" -> copyTags.asJava) + "copyTags" -> copyTags.groupBy(_._2).map { case (k, v) => (k, v.map(_._1).asJava)}.asJava) ConfigFactory.parseMap(map.asJava) @@ -209,7 +209,10 @@ object DatasetOptions { def fromString(s: String): DatasetOptions = fromConfig(ConfigFactory.parseString(s).withFallback(DefaultOptionsConfig)) - def fromConfig(config: Config): DatasetOptions = + def fromConfig(config: Config): DatasetOptions = { + val copyTagsValue = config.as[Map[String, Seq[String]]]("copyTags") + .toSeq + .flatMap { case (key, value) => value.map (_ -> key) } DatasetOptions(shardKeyColumns = config.as[Seq[String]]("shardKeyColumns"), metricColumn = config.getString("metricColumn"), valueColumn = config.getString("valueColumn"), @@ -217,7 +220,8 @@ object DatasetOptions { ignoreShardKeyColumnSuffixes = config.as[Map[String, Seq[String]]]("ignoreShardKeyColumnSuffixes"), ignoreTagsOnPartitionKeyHash = config.as[Seq[String]]("ignoreTagsOnPartitionKeyHash"), - copyTags = config.as[Map[String, String]]("copyTags")) + copyTags = copyTagsValue) + } } /** diff --git a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala index b31afdcaf8..cdf8a552a3 100644 --- a/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala +++ b/gateway/src/main/scala/filodb/gateway/conversion/InputRecord.scala @@ -100,7 +100,7 @@ object PrometheusInputRecord { Nil } else { val metric = metricTags.head._2 - val transformedTags = transformTags(tags.filterNot(_._1 == dataset.options.metricColumn), dataset).toMap + val transformedTags = transformTags(tags.filterNot(_._1 == dataset.options.metricColumn), dataset) (0 until tsProto.getSamplesCount).map { i => val sample = tsProto.getSamples(i) PrometheusInputRecord(transformedTags, metric, dataset, sample.getTimestampMs, sample.getValue) @@ -113,17 +113,17 @@ object PrometheusInputRecord { * If a tag in copyTags is found and the destination tag is missing, then the destination tag is created * with the value from the source tag. */ - def transformTags(tags: Seq[(String, String)], dataset: Dataset): Seq[(String, String)] = { - val keys = tags.map(_._1).toSet - val extraTags = new collection.mutable.ArrayBuffer[(String, String)]() - tags.foreach { case (k, v) => - if (dataset.options.copyTags contains k) { - val renamedKey = dataset.options.copyTags(k) - if (!(keys contains renamedKey)) - extraTags += renamedKey -> v + def transformTags(tags: Seq[(String, String)], dataset: Dataset): Map[String, String] = { + val extraTags = new collection.mutable.HashMap[String, String]() + val tagsMap = tags.toMap + for ((k, v) <- dataset.options.copyTags) { + if (!extraTags.contains(v) + && !tagsMap.contains(v) + && tagsMap.contains(k)) { + extraTags += v -> tagsMap(k) } } - tags ++ extraTags + tagsMap ++ extraTags } } diff --git a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala index cfd90e07cd..634021ad44 100644 --- a/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala +++ b/prometheus/src/main/scala/filodb/prometheus/FormatConversion.scala @@ -12,7 +12,7 @@ object FormatConversion { val dataset = Dataset("prometheus", Seq("tags:map"), Seq("timestamp:ts", "value:double")) .copy(options = DatasetOptions(Seq("__name__", "_ns"), "__name__", "value", false, Map("__name__" -> Seq("_bucket", "_count", "_sum")), Seq("le"), - Map("exporter" -> "_ns", "job" -> "_ns"))) + Seq("exporter" -> "_ns", "job" -> "_ns"))) /** * Extracts a java ArrayList of labels from the TimeSeries From 47e5dc14379b3556c28e75b8633794b73b04ba79 Mon Sep 17 00:00:00 2001 From: TanviBhavsar Date: Mon, 30 Sep 2019 11:37:53 -0700 Subject: [PATCH 28/28] bug(query): store last element of chunk in ChangesChunkedFunction (#515) --- .../format/vectors/DeltaDeltaVector.scala | 34 ++++++++----------- .../format/vectors/DoubleVector.scala | 33 ++++++++++++------ .../format/vectors/LongBinaryVector.scala | 19 ++++++----- .../format/vectors/LongVectorTest.scala | 8 +++-- .../exec/rangefn/AggrOverTimeFunctions.scala | 14 +++++--- .../rangefn/AggrOverTimeFunctionsSpec.scala | 34 ++++++++++++++++++- 6 files changed, 96 insertions(+), 46 deletions(-) diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala index e2625d87f5..29fb097ad3 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DeltaDeltaVector.scala @@ -207,24 +207,20 @@ object DeltaDeltaDataReader extends LongVectorDataReader { new DeltaDeltaIterator(innerIt, slope(vector), initValue(vector) + startElement * slope(vector).toLong) } - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { - if (slope(vector) == 0) { - 0 - } else { + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Long, ignorePrev: Boolean = false): (Long, Long) = { require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") val itr = iterate(vector, start) - var prev: Long = 0 + var prevVector: Long = prev var changes = 0 - for {i <- start until end optimized} { + for {i <- start until end + 1 optimized} { val cur = itr.next - if (i == start) //Initialize prev - prev = cur - if (prev != cur) + if (i == start && ignorePrev) //Initialize prev + prevVector = cur + if (prevVector != cur) changes += 1 - prev = cur + prevVector = cur } - changes - } + (changes, prevVector) } } @@ -274,13 +270,13 @@ object DeltaDeltaConstDataReader extends LongVectorDataReader { } } - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { - if (slope(vector) == 0) { - 0 - } else { - require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") - length(vector) - 1 - } + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Long, ignorePrev: Boolean = false): (Long, Long) = { + require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") + val firstValue = apply(vector, start) + val lastValue = apply(vector, end) + // compare current element with last element(prev) of previous chunk + val changes = if (!ignorePrev && prev != firstValue) 1 else 0 + if (slope(vector) == 0) (changes, lastValue) else (end - start + changes, lastValue) } } diff --git a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala index 82e7557239..f7f6bd27ba 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/DoubleVector.scala @@ -143,7 +143,8 @@ trait DoubleVectorDataReader extends CounterVectorReader { */ def count(vector: BinaryVectorPtr, start: Int, end: Int): Int - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): + (Double, Double) /** * Converts the BinaryVector to an unboxed Buffer. @@ -238,6 +239,7 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader { var addr = vector + OffsetData + start * 8 val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end var count = 0 + while (addr < untilAddr) { val nextDbl = UnsafeUtils.getDouble(addr) // There are many possible values of NaN. Use a function to ignore them reliably. @@ -247,20 +249,23 @@ object DoubleVectorDataReader64 extends DoubleVectorDataReader { count } - final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = { + final def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): + (Double, Double) = { require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") - var prev = Double.NaN var addr = vector + OffsetData + start * 8 val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end var changes = 0d + var prevVector : Double = prev while (addr < untilAddr) { val nextDbl = UnsafeUtils.getDouble(addr) // There are many possible values of NaN. Use a function to ignore them reliably. - if (!java.lang.Double.isNaN(nextDbl) && prev != nextDbl && !java.lang.Double.isNaN(prev)) changes += 1 + if (!java.lang.Double.isNaN(nextDbl) && prevVector != nextDbl && !java.lang.Double.isNaN(prevVector)) { + changes += 1 + } addr += 8 - prev = nextDbl + prevVector = nextDbl } - changes + (changes, prevVector) } } @@ -275,7 +280,8 @@ extends DoubleVectorDataReader { def sum(vector: BinaryVectorPtr, start: Int, end: Int, ignoreNaN: Boolean = true): Double = inner.sum(vector, start, end, ignoreNaN) def count(vector: BinaryVectorPtr, start: Int, end: Int): Int = inner.count(vector, start, end) - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = inner.changes(vector, start, end) + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): + (Double, Double) = inner.changes(vector, start, end, prev) var _correction = 0.0 // Lazily correct - not all queries want corrected data @@ -335,8 +341,8 @@ object MaskedDoubleDataReader extends DoubleVectorDataReader with BitmapMaskVect override def iterate(vector: BinaryVectorPtr, startElement: Int = 0): DoubleIterator = DoubleVector(subvectAddr(vector)).iterate(subvectAddr(vector), startElement) - override def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = - DoubleVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end) + override def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): + (Double, Double) = DoubleVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end, prev) } class DoubleAppendingVector(addr: BinaryRegion.NativePointer, maxBytes: Int, val dispose: () => Unit) @@ -475,6 +481,11 @@ object DoubleLongWrapDataReader extends DoubleVectorDataReader { final def iterate(vector: BinaryVectorPtr, startElement: Int = 0): DoubleIterator = new DoubleLongWrapIterator(LongBinaryVector(vector).iterate(vector, startElement)) - final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Double = - LongBinaryVector(vector).changes(vector, start, end) + final def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Double, ignorePrev: Boolean = false): + (Double, Double) = { + val ignorePrev = if (prev.isNaN) true + else false + val changes = LongBinaryVector(vector).changes(vector, start, end, prev.toLong, ignorePrev) + (changes._1.toDouble, changes._1.toDouble) + } } diff --git a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala index e361b2574e..b262e00e52 100644 --- a/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala +++ b/memory/src/main/scala/filodb.memory/format/vectors/LongBinaryVector.scala @@ -138,7 +138,7 @@ trait LongVectorDataReader extends VectorDataReader { */ def binarySearch(vector: BinaryVectorPtr, item: Long): Int - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Long, ignorePrev: Boolean = false): (Long, Long) /** * Searches for the last element # whose element is <= the item, assuming all elements are increasing. * Typically used to find the last timestamp <= item. @@ -229,21 +229,22 @@ object LongVectorDataReader64 extends LongVectorDataReader { if (element == item) first else first | 0x80000000 } - final def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = { + final def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Long, ignorePrev: Boolean = false): + (Long, Long) = { require(start >= 0 && end < length(vector), s"($start, $end) is out of bounds, length=${length(vector)}") - var prev: Long = 0 + var prevVector: Long = prev val startAddr = vector + OffsetData + start * 8 val untilAddr = vector + OffsetData + end * 8 + 8 // one past the end var changes = 0 var addr = startAddr while (addr < untilAddr) { val cur = UnsafeUtils.getLong(addr) - if (addr == startAddr) prev = cur - if (prev != cur) changes += 1 - prev = cur + if (addr == startAddr) prevVector = cur + if (prevVector != cur) changes += 1 + prevVector = cur addr += 8 } - changes + (changes, prevVector) } } @@ -268,8 +269,8 @@ object MaskedLongDataReader extends LongVectorDataReader with BitmapMaskVector { def binarySearch(vector: BinaryVectorPtr, item: Long): Int = LongBinaryVector(subvectAddr(vector)).binarySearch(subvectAddr(vector), item) - def changes(vector: BinaryVectorPtr, start: Int, end: Int): Int = - LongBinaryVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end) + def changes(vector: BinaryVectorPtr, start: Int, end: Int, prev: Long, ignorePrev: Boolean = false): (Long, Long) = + LongBinaryVector(subvectAddr(vector)).changes(subvectAddr(vector), start, end, prev) } class LongAppendingVector(addr: BinaryRegion.NativePointer, maxBytes: Int, val dispose: () => Unit) diff --git a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala index 0795c798fc..53661ca69e 100644 --- a/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala +++ b/memory/src/test/scala/filodb.memory/format/vectors/LongVectorTest.scala @@ -382,7 +382,9 @@ class LongVectorTest extends NativeVectorTest with PropertyChecks { BinaryVector.majorVectorType(ptr) shouldEqual WireFormat.VECTORTYPE_DELTA2 val readVect = LongBinaryVector(ptr) readVect shouldEqual DeltaDeltaConstDataReader - readVect.changes(ptr,0, 4) shouldEqual(0) + val changesResult = readVect.changes(ptr,0, 4,0, true) + changesResult._1 shouldEqual(0) + changesResult._2 shouldEqual(Int.MaxValue.toLong + 100) } it("should do changes on DeltaDeltaDataReader") { @@ -393,6 +395,8 @@ class LongVectorTest extends NativeVectorTest with PropertyChecks { val ptr = builder.optimize(memFactory) val readVect = LongBinaryVector(ptr) readVect shouldEqual DeltaDeltaDataReader - LongBinaryVector(ptr).changes(ptr, 0, 6) shouldEqual(4) + val changesResult = LongBinaryVector(ptr).changes(ptr, 0, 6, 0, true) + changesResult._1 shouldEqual(4) + changesResult._2 shouldEqual(6004) } } \ No newline at end of file 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 ca7d8a0a51..1d63f57825 100644 --- a/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala +++ b/query/src/main/scala/filodb/query/exec/rangefn/AggrOverTimeFunctions.scala @@ -552,8 +552,9 @@ class StdVarOverTimeChunkedFunctionL extends VarOverTimeChunkedFunctionL() { } } -abstract class ChangesChunkedFunction(var changes: Double = Double.NaN) extends ChunkedRangeFunction[TransientRow] { - override final def reset(): Unit = { changes = Double.NaN } +abstract class ChangesChunkedFunction(var changes: Double = Double.NaN, var prev: Double = Double.NaN) + extends ChunkedRangeFunction[TransientRow] { + override final def reset(): Unit = { changes = Double.NaN; prev = Double.NaN } final def apply(endTimestamp: Long, sampleToEmit: TransientRow): Unit = { sampleToEmit.setValues(endTimestamp, changes) } @@ -568,7 +569,10 @@ class ChangesChunkedFunctionD() extends ChangesChunkedFunction() with if (changes.isNaN) { changes = 0d } - changes += doubleReader.changes(doubleVect, startRowNum, endRowNum) + + val changesResult = doubleReader.changes(doubleVect, startRowNum, endRowNum, prev) + changes += changesResult._1 + prev = changesResult._2 } } @@ -582,6 +586,8 @@ class ChangesChunkedFunctionL extends ChangesChunkedFunction with if (changes.isNaN) { changes = 0d } - changes += longReader.changes(longVect, startRowNum, endRowNum) + val changesResult = longReader.changes(longVect, startRowNum, endRowNum, prev.toLong) + changes += changesResult._1 + prev = changesResult._2 } } \ No newline at end of file diff --git a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala index 9008dfd2e3..4e44b03fc6 100644 --- a/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala +++ b/query/src/test/scala/filodb/query/exec/rangefn/AggrOverTimeFunctionsSpec.scala @@ -153,7 +153,6 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val windowSize = rand.nextInt(100) + 10 val step = rand.nextInt(75) + 5 info(s" iteration $x windowSize=$windowSize step=$step") - val slidingIt = slidingWindowIt(data, rv, new SumOverTimeFunction(), windowSize, step) val aggregated = slidingIt.map(_.getDouble(1)).toBuffer // drop first sample because of exclusive start @@ -260,6 +259,10 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val avgChunked = chunkedWindowIt(data, rv, new AvgOverTimeChunkedFunctionD(), windowSize, step) val aggregated4 = avgChunked.map(_.getDouble(1)).toBuffer aggregated4 shouldEqual data.sliding(windowSize, step).map(a => avg(a drop 1)).toBuffer + + val changesChunked = chunkedWindowIt(data, rv, new ChangesChunkedFunctionD(), windowSize, step) + val aggregated5 = changesChunked.map(_.getDouble(1)).toBuffer + aggregated5.drop(0) shouldEqual data.sliding(windowSize, step).map(_.length - 2).drop(0).toBuffer } } @@ -306,4 +309,33 @@ class AggrOverTimeFunctionsSpec extends RawDataWindowingSpec { val aggregated = chunkedIt.map(x => (x.getLong(0), x.getDouble(1))).toList aggregated shouldEqual List((100000, 0.0), (120000, 2.0), (140000, 2.0)) } + + it("should correctly do changes for DoubleVectorDataReader and DeltaDeltaDataReader when window has more " + + "than one chunks") { + val data1= (1 to 240).map(_.toDouble) + val data2 : Seq[Double]= Seq[Double]( 1.1, 1.5, 2.5, 3.5, 4.5, 5.5) + + (0 until numIterations).foreach { x => + val windowSize = rand.nextInt(100) + 10 + val step = rand.nextInt(50) + 5 + info(s" iteration $x windowSize=$windowSize step=$step") + // Append double data and shuffle so that it becomes DoubleVectorDataReader + val data = scala.util.Random.shuffle(data2 ++ data1) + + val rv = timeValueRV(data) + val list = rv.rows.map(x => (x.getLong(0), x.getDouble(1))).toList + + val stepTimeMillis = step.toLong * pubFreq + val changesChunked = chunkedWindowIt(data, rv, new ChangesChunkedFunctionD(), windowSize, step) + val aggregated2 = changesChunked.map(_.getDouble(1)).toBuffer + + data.sliding(windowSize, step).map(_.length - 2).toBuffer.zip(aggregated2).foreach { + case(val1, val2) => if (val1 == -1 ) { + val2.isNaN shouldEqual (true) // window does not have any element so changes will be NaN + } else { + val1 shouldEqual (val2) + } + } + } + } }